diff --git a/bin/functions/hibench_prop_env_mapping.py b/bin/functions/hibench_prop_env_mapping.py index 1ef946491..97739afcc 100644 --- a/bin/functions/hibench_prop_env_mapping.py +++ b/bin/functions/hibench_prop_env_mapping.py @@ -123,36 +123,31 @@ MODEL="hibench.nweight.model", # For streaming bench - STREAMING_TESTCASE="hibench.streambench.testCase", - + # zkHelper + STREAMING_ZKHELPER_JAR="hibench.streamingbench.zkhelper.jar", # prepare - STREAMING_TOPIC_NAME="hibench.streambench.kafka.topic", - STREAMING_KAFKA_HOME="hibench.streambench.kafka.home", - STREAMING_ZKADDR="hibench.streambench.zkHost", - STREAMING_CONSUMER_GROUP="hibench.streambench.kafka.consumerGroup", - STREAMING_DATA_DIR="hibench.streambench.datagen.dir", - STREAMING_DATA1_NAME="hibench.streambench.datagen.data1.name", - STREAMING_DATA1_DIR="hibench.streambench.datagen.data1.dir", - STREAMING_DATA1_LENGTH="hibench.streambench.datagen.recordLength", - STREAMING_DATA2_SAMPLE_DIR="hibench.streambench.datagen.data2_samples.dir", - STREAMING_DATA2_CLUSTER_DIR="hibench.streambench.datagen.data2_cluster.dir", - STREAMING_PARTITIONS="hibench.streambench.kafka.topicPartitions", - DATA_GEN_JAR="hibench.streambench.datagen.jar", + STREAMING_TOPIC_NAME="hibench.streamingbench.topic_name", + STREAMING_KAFKA_HOME="hibench.streamingbench.kafka.home", + STREAMING_ZKADDR="hibench.streamingbench.zookeeper.host", + STREAMING_CONSUMER_GROUP="hibench.streamingbench.consumer_group", + STREAMING_DATA_SCALE_FACTOR="hibench.streamingbench.datagen.scale_factor", + STREAMING_DATA_DIR="hibench.streamingbench.datagen.dir", + STREAMING_DATA1_NAME="hibench.streamingbench.datagen.data1.name", + STREAMING_DATA1_DIR="hibench.streamingbench.datagen.data1.dir", + STREAMING_DATA1_LENGTH="hibench.streamingbench.datagen.data1.length", + STREAMING_DATA2_SAMPLE_DIR="hibench.streamingbench.datagen.data2_samples.dir", + STREAMING_DATA2_CLUSTER_DIR="hibench.streamingbench.datagen.data2_cluster.dir", + STREAMING_PARTITIONS="hibench.streamingbench.partitions", + DATA_GEN_JAR="hibench.streamingbench.datagen.jar", + + STREAMING_DATAGEN_MODE="hibench.streamingbench.prepare.mode", + STREAMING_DATAGEN_RECORDS="hibench.streamingbench.prepare.push.records", # sparkstreaming - STREAMBENCH_SPARK_JAR="hibench.streambench.sparkbench.jar", - STREAMBENCH_STORM_JAR="hibench.streambench.stormbench.jar", - - # gearpump - GEARPUMP_HOME="hibench.streambench.gearpump.home", - STREAMBENCH_GEARPUMP_JAR="hibench.streambench.gearpump.jar", - STREAMBENCH_GEARPUMP_EXECUTORS="hibench.streambench.gearpump.executors", - - # flinkstreaming - HIBENCH_FLINK_MASTER="hibench.flink.master", - FLINK_HOME="hibench.streambench.flink.home", - STREAMBENCH_FLINK_JAR="hibench.streambench.flinkbench.jar", - STREAMBENCH_FLINK_PARALLELISM="hibench.streambench.flink.parallelism", + STREAMINGBENCH_JARS="hibench.streamingbench.jars", + STREAMBENCH_STORM_JAR="hibench.streamingbench.stormbench.jar", + STORM_BIN_HOME="hibench.streamingbench.storm.bin", + STREAMING_BENCHNAME="hibench.streamingbench.benchname", # samza STREAMING_SAMZA_WORDCOUNT_INTERNAL_TOPIC="samza_internal.wordcount.kafka.input.name", diff --git a/bin/functions/load-config.py b/bin/functions/load-config.py index b1874f4a9..675bdb10e 100755 --- a/bin/functions/load-config.py +++ b/bin/functions/load-config.py @@ -239,9 +239,9 @@ def wildcard_replacement(key, value): finish = False - # wildcard_rules = [(key, HibenchConf[key]) for key in HibenchConf if "*" in key] + wildcard_rules = [(key, HibenchConf[key]) for key in HibenchConf if "*" in key] # now, let's check wildcard replacement rules - # for key, value in wildcard_rules: + for key, value in wildcard_rules: # check if we found a rule like: aaa.*.ccc.*.ddd -> bbb.*.* # wildcard replacement is useful for samza conf, which @@ -253,12 +253,12 @@ def wildcard_replacement(key, value): # switch the order of two wildcards, something like the # first wildcard in key to match the second wildcard in # value. I just don't think it'll be needed. - # if not wildcard_replacement(key, value): # not wildcard rules? re-add + if not wildcard_replacement(key, value): # not wildcard rules? re-add HibenchConf[key] = value - # if wildcard_rules: # need try again - # wildcard_rules = [] - # else: break - break + if wildcard_rules: # need try again + wildcard_rules = [] + else: break + # all finished, remove values contains no_value_sign for key in [x for x in HibenchConf if no_value_sign in HibenchConf[x]]: del HibenchConf[key] diff --git a/bin/functions/workload-functions.sh b/bin/functions/workload-functions.sh index 6a56c87a9..2f9ec27cf 100644 --- a/bin/functions/workload-functions.sh +++ b/bin/functions/workload-functions.sh @@ -271,27 +271,15 @@ function run-spark-job() { } function run-streaming-job (){ - run-spark-job --jars ${STREAMBENCH_SPARK_JAR} $@ + run-spark-job --jars ${STREAMINGBENCH_JARS} $@ } function run-storm-job(){ - CMD="${STORM_HOME}/bin/storm jar ${STREAMBENCH_STORM_JAR} $@" + CMD="${STORM_BIN_HOME}/storm jar ${STREAMBENCH_STORM_JAR} $@" echo -e "${BGreen}Submit Storm Job: ${Green}$CMD${Color_Off}" execute_withlog $CMD } -function run-gearpump-app(){ - CMD="${GEARPUMP_HOME}/bin/gear app -executors ${STREAMBENCH_GEARPUMP_EXECUTORS} -jar ${STREAMBENCH_GEARPUMP_JAR} $@" - echo -e "${BGreen}Submit Gearpump Application: ${Green}$CMD${Color_Off}" - execute_withlog $CMD -} - -function run-flink-job(){ - CMD="${FLINK_HOME}/bin/flink run -p ${STREAMBENCH_FLINK_PARALLELISM} -m ${HIBENCH_FLINK_MASTER} $@ ${STREAMBENCH_FLINK_JAR} ${SPARKBENCH_PROPERTIES_FILES}" - echo -e "${BGreen}Submit Flink Job: ${Green}$CMD${Color_Off}" - execute_withlog $CMD -} - function run-hadoop-job(){ ENABLE_MONITOR=1 if [ "$1" = "--without-monitor" ]; then diff --git a/conf/00-default-properties.conf b/conf/00-default-properties.conf index 5cbf7eab9..0fee0959d 100644 --- a/conf/00-default-properties.conf +++ b/conf/00-default-properties.conf @@ -257,9 +257,3 @@ spark.default.parallelism ${hibench.default.map.parallelism} # set spark sql's default shuffle partitions according to hibench's parallelism value spark.sql.shuffle.partitions ${hibench.default.map.parallelism} - - -#======================================================= -# Flink -#======================================================= -hibench.flink.master FLINK_JM_HOST:PORT diff --git a/conf/01-default-streamingbench.conf b/conf/01-default-streamingbench.conf index 2a2b4919c..c28e0e8ab 100644 --- a/conf/01-default-streamingbench.conf +++ b/conf/01-default-streamingbench.conf @@ -1,152 +1,146 @@ -######################################################### -# General Stream Config -######################################################### +# Two data sets(text and numeric) are available, app argument indicates to use which +#app=micro-sketch #use text dataset, avg record size: 60 bytes +#app=micro-statistics #use numeric dataset, avg record size: 200 bytes +hibench.streamingbench.app micro-sketch -# Note to ensure benchName to be consistent with datagen type. Numeric data for statistics and text data for others -# (available benchname: identity, repartition) TDB: sample project grep wordcount distinctcount statistics -hibench.streambench.testCase identity +# Text dataset can be scaled in terms of record size +hibench.streamingbench.prepare.textdataset_recordsize_factor -# zookeeper address for Kakfa serverce, (default: HOSTNAME:HOSTPORT) -hibench.streambench.zkHost HOSTNAME:HOSTPORT +# Two modes of generator: push,periodic +# Push means to send data to kafka cluster as fast as it could +# Periodic means sending data according to sending rate specification +#hibench.streamingbench.prepare.mode push +hibench.streamingbench.prepare.mode periodic -# Probability used in sample test case -hibench.streambench.sampleProbability 0.1 +# Under push mode: number of total records that will be generated +hibench.streamingbench.prepare.push.records 900000000 -# Indicate whether in debug mode for correctness verfication (default: false) -hibench.streambench.debugMode false +# Following three params are under periodic mode +# Bytes to push per interval +hibench.streamingbench.prepare.periodic.recordPerInterval 600000 -# JARS -hibench.streambench.datagen.jar ${hibench.home}/src/streambench/datagen/target/streaming-bench-datagen-5.0-SNAPSHOT-jar-with-dependencies.jar -hibench.streambench.sparkbench.jar ${hibench.home}/src/streambench/sparkbench/target/streaming-bench-spark-5.0-SNAPSHOT-${hibench.spark.version}-jar-with-dependencies.jar -hibench.streambench.stormbench.jar ${hibench.home}/src/streambench/stormbench/target/streaming-bench-storm-5.0-SNAPSHOT.jar -hibench.streambench.gearpump.jar ${hibench.home}/src/streambench/gearpumpbench/target/streaming-bench-gearpump-5.0-SNAPSHOT-jar-with-dependencies.jar -hibench.streambench.flinkbench.jar ${hibench.home}/src/streambench/flinkbench/target/streaming-bench-flink-5.0-SNAPSHOT-jar-with-dependencies.jar +# Interval time (in ms) +hibench.streamingbench.prepare.periodic.intervalSpan 5000 -######################################################### -# Kafka Config -######################################################### +# Total round count of data send +hibench.streamingbench.prepare.periodic.totalRound 100 -# Kafka home -hibench.streambench.kafka.home /PATH/TO/KAFKA/HOME - -# the topic that spark will receive input data (default: ${hibench.streambench.testCase}) -hibench.streambench.kafka.topic ${hibench.streambench.testCase} - -# number of partitions of generated topic (default 20) -hibench.streambench.kafka.topicPartitions 20 +# zookeeper host:port of kafka cluster -# consumer group of the consumer for kafka (default: HiBench) -hibench.streambench.kafka.consumerGroup HiBench +#example: hostname:9092 +hibench.streamingbench.zookeeper.host HOSTNAME:HOSTPORT -# Kafka broker lists, written in mode "host:port,host:port,..." (default: HOSTNAME:HOSTPORT) -hibench.streambench.kafka.brokerList HOSTNAME:HOSTPORT +#Parallel config +# number of nodes that will receive kafka input +hibench.streamingbench.receiver_nodes 4 -# Set the starting offset of kafkaConsumer (default: largest) -hibench.streambench.kafka.offsetReset largest -######################################################### -# Data Generator Config -######################################################### +############### +#Benchmark args +#Note to ensure benchName to be consistent with datagen type. Numeric data for statistics and text data for others +# available benchname: identity sample project grep wordcount distinctcount statistics -# Interval span in millisecond (default: 50) -hibench.streambench.datagen.intervalSpan 50 +hibench.streamingbench.benchname identity -# Number of records to generate per interval span (default: 5) -hibench.streambench.datagen.recordsPerInterval 5 +#common args +# the topic that spark will receive input data +hibench.streamingbench.topic_name ${hibench.streamingbench.benchname} -# Number of total records that will be generated (default: -1 means infinity) -hibench.streambench.datagen.totalRecords -1 +# Spark stream batch interval (in seconds) +hibench.streamingbench.batch_interval 10 -# Total round count of data send (default: -1 means infinity) -hibench.streambench.datagen.totalRounds -1 +# consumer group of the spark consumer for kafka +hibench.streamingbench.consumer_group HiBench -# default path to store seed files (default: ${hibench.hdfs.data.dir}/Streaming) -hibench.streambench.datagen.dir ${hibench.hdfs.data.dir}/Streaming +# expected number of records to be processed +hibench.streamingbench.record_count 900000000 -# fixed length of record (default: 200) -hibench.streambench.datagen.recordLength 200 +#sketch/distinctcount/statistics arg +# the field index of the record that will be extracted +hibench.streamingbench.field_index 1 -# Number of KafkaProducer running on different thread (default: 1) -# The limitation of a single KafkaProducer is about 100Mb/s -hibench.streambench.datagen.producerNumber 1 +#sketch/wordcount/distinctcount/statistics arg +# the seperator between fields of a single record +hibench.streamingbench.separator \\s+ -hibench.streambench.fixWindowDuration 30000 +#sample arg +# probability that a record will be taken as a sample +hibench.streamingbench.prob 0.1 -hibench.streambench.fixWindowSlideStep 30000 -######################################################### -# Spark Streaming Config -######################################################### +#grep arg +# the substring that will be checked to see if contained in a record +hibench.streamingbench.pattern the -# Number of nodes that will receive kafka input (default: 4) -hibench.streambench.spark.receiverNumber 4 +#common arg +# indicate RDD storage level. +# 1 for memory only 1 copy. Others for default mem_disk_ser 2 copies +hibench.streamingbench.copies 2 -# Spark streaming Batchnterval in millisecond (default 100) -hibench.streambench.spark.batchInterval 100 +# indicate whether to test the write ahead log new feature +# set true to test WAL feature +hibench.streamingbench.testWAL false -# Indicate RDD storage level. (default: 2) -# 0 = StorageLevel.MEMORY_ONLY -# 1 = StorageLevel.MEMORY_AND_DISK_SER -# other = StorageLevel.MEMORY_AND_DISK_SER_2 -hibench.streambench.spark.storageLevel 2 +# if testWAL is true, this path to store stream context in hdfs shall be specified. If false, it can be empty +hibench.streamingbench.checkpoint_path -# indicate whether to test the write ahead log new feature (default: false) -hibench.streambench.spark.enableWAL false +#common arg +# indicate whether in debug mode for correctness verfication +hibench.streamingbench.debug false -# if testWAL is true, this path to store stream context in hdfs shall be specified. If false, it can be empty (default: /var/tmp) -hibench.streambench.spark.checkpointPath /var/tmp +# whether to use direct approach or not ( sparkstreaming only ) +hibench.streamingbench.direct_mode true -# whether to use direct approach or not (dafault: true) -hibench.streambench.spark.useDirectMode true +# Kafka broker lists, used for direct mode, written in mode "host:port,host:port,..." -######################################################### -# Flink Config -######################################################### -hibench.streambench.flink.home /PATH/TO/FLINK/HOME +# example: hostname:9092 +hibench.streamingbench.brokerList HOSTNAME:HOSTPORT -# default parallelism of flink job -hibench.streambench.flink.parallelism 20 +hibench.streamingbench.broker_list_with_quote "${hibench.streamingbench.brokerList}" -hibench.streambench.flink.bufferTimeout 5 +# storm bench conf -hibench.streambench.flink.checkpointDuration 1000 +# STORM_BIN_HOME +hibench.streamingbench.storm.home /PATH/TO/STORM/HOME -######################################################### -# Storm Config -######################################################### +# Kafka home +hibench.streamingbench.kafka.home /PATH/TO/KAFKA/HOME -# STORM_BIN_HOME -hibench.streambench.storm.home /PATH/TO/STORM/HOME +#Cluster config # nimbus of storm cluster -hibench.streambench.storm.nimbus HOSTNAME_OF_STORM_NIMBUS -hibench.streambench.storm.nimbusAPIPort 6627 +hibench.streamingbench.storm.nimbus HOSTNAME_OF_STORM +hibench.streamingbench.storm.nimbusAPIPort 6627 # time interval to contact nimbus to judge if finished -hibench.streambench.storm.nimbusContactInterval 10 +hibench.streamingbench.storm.nimbusContactInterval 10 + + +#Parallel config # number of workers of Storm. Number of most bolt threads is also equal to this param. -hibench.streambench.storm.worker_count 12 +hibench.streamingbench.storm.worker_count 12 # number of kafka spout threads of Storm -hibench.streambench.storm.spout_threads 12 +hibench.streamingbench.storm.spout_threads 12 # number of bolt threads altogether -hibench.streambench.storm.bolt_threads 12 +hibench.streamingbench.storm.bolt_threads 12 # kafka arg indicating whether to read data from kafka from the start or go on to read from last position -hibench.streambench.storm.read_from_start true +hibench.streamingbench.storm.read_from_start true # whether to turn on ack -hibench.streambench.storm.ackon true - -######################################################### -# Gearpump Config -######################################################### - -hibench.streambench.gearpump.home /PATH/TO/GEARPUMP/HOME - -hibench.streambench.gearpump.executors 1 - -hibench.streambench.gearpump.parallelism 1 +hibench.streamingbench.storm.ackon true +# Added for default rules: +hibench.streamingbench.jars ${hibench.streamingbench.sparkbench.jar} +hibench.streamingbench.sparkbench.jar ${hibench.home}/src/streambench/sparkbench/target/streaming-bench-spark_0.1-5.0-SNAPSHOT-${hibench.spark.version}-jar-with-dependencies.jar +hibench.streamingbench.stormbench.jar ${hibench.home}/src/streambench/stormbench/target/streaming-bench-storm-0.1-SNAPSHOT-jar-with-dependencies.jar +hibench.streamingbench.datagen.jar ${hibench.home}/src/streambench/datagen/target/datagen-0.0.1-jar-with-dependencies.jar +hibench.streamingbench.storm.bin ${hibench.streamingbench.storm.home}/bin +hibench.streamingbench.zkhelper.jar ${hibench.home}/src/streambench/zkHelper/target/streaming-bench-zkhelper-0.1-SNAPSHOT-jar-with-dependencies.jar +# default path setting for store of data1 & data2 +hibench.streamingbench.datagen.dir ${hibench.hdfs.data.dir}/Streaming +# partition size settings +hibench.streamingbench.partitions 1 diff --git a/conf/20-samza-common.conf b/conf/20-samza-common.conf index 2c8d7d3a5..28c449d5c 100644 --- a/conf/20-samza-common.conf +++ b/conf/20-samza-common.conf @@ -1,6 +1,6 @@ # YARN samza_internal.yarn.package.path ${hibench.home}/src/streambench/samzabench/target/streaming-bench-samza-0.1-SNAPSHOT-dist.tar.gz -samza.yarn.package.path ${hibench.streambench.datagen.dir}/samza_package/streaming-bench-samza-0.1-SNAPSHOT-dist.tar.gz +samza.yarn.package.path ${hibench.streamingbench.datagen.dir}/samza_package/streaming-bench-samza-0.1-SNAPSHOT-dist.tar.gz samza.yarn.container.count 1 # Metrics @@ -18,14 +18,14 @@ samza.serializers.registry.integer.class org.apache.samza.serializers samza.systems.kafka.samza.factory org.apache.samza.system.kafka.KafkaSystemFactory samza.systems.kafka.samza.msg.serde string samza.systems.kafka.streams.metrics.samza.msg.serde metrics -samza.systems.kafka.consumer.zookeeper.connect ${hibench.streambench.zookeeper.host}/ +samza.systems.kafka.consumer.zookeeper.connect ${hibench.streamingbench.zookeeper.host}/ samza.systems.kafka.consumer.auto.offset.reset smallest -samza.systems.kafka.producer.bootstrap.servers ${hibench.streambench.brokerList} -samza.systems.kafka.producer.metadata.broker.list ${hibench.streambench.brokerList} +samza.systems.kafka.producer.bootstrap.servers ${hibench.streamingbench.brokerList} +samza.systems.kafka.producer.metadata.broker.list ${hibench.streamingbench.brokerList} # Override replication factor (NOTE: Your number of brokers must be greater than replication factor!) samza.task.checkpoint.replication.factor 1 -samza_internal.partitions ${hibench.streambench.partitions} +samza_internal.partitions ${hibench.streamingbench.partitions} # Tasks samza.task.checkpoint.factory org.apache.samza.checkpoint.kafka.KafkaCheckpointManagerFactory @@ -35,6 +35,6 @@ samza.task.checkpoint.system kafka samza.job.factory.class org.apache.samza.job.yarn.YarnJobFactory # mapping task properties -samza_workload_conf.${hibench.streambench.benchname}.* samza.* +samza_workload_conf.${hibench.streamingbench.benchname}.* samza.* diff --git a/conf/30-samza-workloads.conf b/conf/30-samza-workloads.conf index e6c46dcaa..444772e8c 100644 --- a/conf/30-samza-workloads.conf +++ b/conf/30-samza-workloads.conf @@ -5,10 +5,10 @@ samza_workload_conf.identity.job.name identity # Task samza_workload_conf.identity.task.class com.intel.hibench.streambench.samza.micro.Identity -samza_workload_conf.identity.task.inputs kafka.${hibench.streambench.topic_name} +samza_workload_conf.identity.task.inputs kafka.${hibench.streamingbench.topic_name} # Systems -samza_workload_conf.identity.systems.kafka.streams.${hibench.streambench.topic_name}.samza.offset.default oldest +samza_workload_conf.identity.systems.kafka.streams.${hibench.streamingbench.topic_name}.samza.offset.default oldest # wordcount @@ -69,7 +69,7 @@ samza_workload_conf.split.job.name split # Task samza_workload_conf.split.task.class com.intel.hibench.streambench.samza.micro.Split samza_workload_conf.split.task.inputs kafka.${samza_internal.split.inputs} -samza_internal.split.inputs ${hibench.streambench.topic_name} +samza_internal.split.inputs ${hibench.streamingbench.topic_name} # Systems samza_workload_conf.split.systems.kafka.streams.${samza_internal.split.inputs}.samza.offset.default oldest @@ -86,7 +86,7 @@ samza_workload_conf.extractfordistinctcount.job.name extra # Task samza_workload_conf.extractfordistinctcount.task.class com.intel.hibench.streambench.samza.micro.ExtractForDistinctcount samza_workload_conf.extractfordistinctcount.task.inputs kafka.${samza_internal.extractfordistinctcount.inputs} -samza_internal.extractfordistinctcount.inputs ${hibench.streambench.topic_name} +samza_internal.extractfordistinctcount.inputs ${hibench.streamingbench.topic_name} # Systems @@ -104,7 +104,7 @@ samza_workload_conf.extractforstatistics.job.name extra # Task samza_workload_conf.extractforstatistics.task.class com.intel.hibench.streambench.samza.micro.ExtractForStatistics samza_workload_conf.extractforstatistics.task.inputs kafka.${samza_internal.extractforstatistics.inputs} -samza_internal.extractforstatistics.inputs ${hibench.streambench.topic_name} +samza_internal.extractforstatistics.inputs ${hibench.streamingbench.topic_name} # Systems @@ -121,10 +121,10 @@ samza_workload_conf.project.job.name project # Task samza_workload_conf.project.task.class com.intel.hibench.streambench.samza.micro.Projection -samza_workload_conf.project.task.inputs kafka.${hibench.streambench.topic_name} +samza_workload_conf.project.task.inputs kafka.${hibench.streamingbench.topic_name} # Systems -samza_workload_conf.project.systems.kafka.streams.${hibench.streambench.topic_name}.samza.offset.default oldest +samza_workload_conf.project.systems.kafka.streams.${hibench.streamingbench.topic_name}.samza.offset.default oldest # metrics @@ -147,7 +147,7 @@ samza_workload_conf.metrics.serializers.registry.string.class org.apac samza_workload_conf.metrics.systems.kafka.samza.factory org.apache.samza.system.kafka.KafkaSystemFactory #systems.kafka.samza.msg.serde=string samza_workload_conf.metrics.systems.kafka.streams.metrics.samza.msg.serde metrics -samza_workload_conf.metrics.systems.kafka.consumer.zookeeper.connect ${hibench.streambench.zookeeper.host} +samza_workload_conf.metrics.systems.kafka.consumer.zookeeper.connect ${hibench.streamingbench.zookeeper.host} samza_workload_conf.metrics.systems.kafka.consumer.auto.offset.reset largest -samza_workload_conf.metrics.systems.kafka.producer.bootstrap.servers ${hibench.streambench.brokerList} -samza_workload_conf.metrics.systems.kafka.producer.metadata.broker.list ${hibench.streambench.brokerList} +samza_workload_conf.metrics.systems.kafka.producer.bootstrap.servers ${hibench.streamingbench.brokerList} +samza_workload_conf.metrics.systems.kafka.producer.metadata.broker.list ${hibench.streamingbench.brokerList} diff --git a/conf/99-user_defined_properties.conf.template b/conf/99-user_defined_properties.conf.template index 3495fabb0..aa7d39211 100644 --- a/conf/99-user_defined_properties.conf.template +++ b/conf/99-user_defined_properties.conf.template @@ -90,41 +90,31 @@ hibench.compress.profile disable # Compression codec profile selection: snappy, lzo, default hibench.compress.codec.profile snappy -#====================================================== # Streaming settings -#====================================================== #------------------- # Available benchname: identity sample project grep wordcount distinctcount statistics -hibench.streambench.testCase identity +hibench.streamingbench.benchname identity # data scale of streaming data -hibench.streambench.scale.profile ${hibench.scale.profile} +hibench.streamingbench.scale.profile ${hibench.scale.profile} # zookeeper host:port of kafka cluster #example hostname:2181 -hibench.streambench.zkHost HOSTNAME:HOSTPORT +hibench.streamingbench.zookeeper.host HOSTNAME:HOSTPORT # Kafka broker lists, used for direct mode, written in mode "host:port,host:port,..." #example hostname:9092 -hibench.streambench.kafka.brokerList HOSTNAME:HOSTPORT +hibench.streamingbench.brokerList HOSTNAME:HOSTPORT # Storm home -hibench.streambench.storm.home /PATH/TO/STORM/HOME - -# Flink home -hibench.streambench.flink.home /PATH/TO/FLINK/HOME +hibench.streamingbench.storm.home /PATH/TO/STORM/HOME # Kafka home -hibench.streambench.kafka.home /PATH/TO/KAFKA/HOME +hibench.streamingbench.kafka.home /PATH/TO/KAFKA/HOME #Cluster config # nimbus of storm cluster -hibench.streambench.storm.nimbus HOSTNAME_TO_STORM_NIMBUS +hibench.streamingbench.storm.nimbus HOSTNAME_TO_STORM_NIMBUS # partition size settings -hibench.streambench.kafka.topicPartitions 20 - -## gearpump bench conf -hibench.streambench.gearpump.home /PATH/TO/GEARPUMP/HOME -hibench.streambench.gearpump.executors 1 -hibench.streambench.gearpump.parallelism 1 +hibench.streamingbench.partitions 1 diff --git a/src/pom.xml b/src/pom.xml index 2d581ced4..9b897922e 100644 --- a/src/pom.xml +++ b/src/pom.xml @@ -1,13 +1,13 @@ - 4.0.0 + 4.0.0 - com.intel.hibench - hibench - 5.0-SNAPSHOT - pom - hibench - http://maven.apache.org + com.intel.hibench + hibench + 5.0-SNAPSHOT + pom + hibench + http://maven.apache.org 1.6 @@ -35,195 +35,179 @@ 6.5.15 - - - central - - Maven Repository - https://repo1.maven.org/maven2 - - true - - - false - - - - apache-repo - Apache Repository - https://repository.apache.org/content/repositories/releases - - true - - - false - - - - scala-tools.org - Scala-tools Maven 2 Repository - https://oss.sonatype.org/content/groups/scala-tools/ - - - - - scala-tools.org - Scala-tools Maven2 Repository - https://oss.sonatype.org/content/groups/scala-tools/ - - + + + central + + Maven Repository + https://repo1.maven.org/maven2 + + true + + + false + + + + apache-repo + Apache Repository + https://repository.apache.org/content/repositories/releases + + true + + + false + + + + scala-tools.org + Scala-tools Maven 2 Repository + https://oss.sonatype.org/content/groups/scala-tools/ + + + + + scala-tools.org + Scala-tools Maven2 Repository + https://oss.sonatype.org/content/groups/scala-tools/ + + - - - - - net.alchim31.maven - scala-maven-plugin - ${scala-maven-plugin.version} - - - org.apache.maven.plugins - maven-compiler-plugin - ${maven-compiler-plugin.version} - - - - - - org.apache.maven.plugins - maven-jar-plugin - ${maven-jar-plugin.version} - - - default-jar - package - - jar - - - - - - + + + + + net.alchim31.maven + scala-maven-plugin + ${scala-maven-plugin.version} + + + org.apache.maven.plugins + maven-compiler-plugin + ${maven-compiler-plugin.version} + + + + + + org.apache.maven.plugins + maven-jar-plugin + ${maven-jar-plugin.version} + + + default-jar + package + + jar + + + + + + - - nutchindexing - mahout - hivebench - sparkbench - pegasus - autogen - + + nutchindexing + mahout + hivebench + sparkbench + pegasus + autogen + - - - MR1 - - 1 - - - - MR1 - - - + + + MR1 + + 1 + + + MR1 + + - - MR2 - - - - 2 - - - true - - MR2 - - - + + MR2 + + + + 2 + + + !MR1 + + - - spark1.2 - - 1.2.1 - 1.2 - - - - spark1.2 - - - + + spark1.2 + + 1.2.1 + 1.2 + + + spark1.2 + + - - spark1.3 - - spark1.3 - 1.3.0 - 1.3 - - - - spark1.3 - - - - streambench - - + + spark1.3 + + spark1.3 + 1.3.0 + 1.3 + + + !spark1.2 + + + streambench + - - spark1.4 - - spark1.4 - 1.4.0 - 1.4 - - - - spark1.4 - - - - streambench - - + + + spark1.4 + + spark1.4 + 1.4.0 + 1.4 + + + spark1.4 + + + streambench + - - spark1.5 - - spark1.5 - 1.5.0 - 1.5 - - - - spark1.5 - - - - streambench - - + + + spark1.5 + + spark1.5 + 1.5.0 + 1.5 + + + spark1.5 + + + streambench + - - spark1.6 - - spark1.6 - 1.6.1 - 1.6 - - - true - - spark1.6 - - - - streambench - - + + + spark1.6 + + spark1.6 + 1.6.0 + 1.6 + + + spark1.6 + + + streambench + - + + diff --git a/src/streambench/common/pom.xml b/src/streambench/common/pom.xml deleted file mode 100644 index 06489fd44..000000000 --- a/src/streambench/common/pom.xml +++ /dev/null @@ -1,121 +0,0 @@ - - - - - 4.0.0 - - com.intel.hibench - streambench - 5.0-SNAPSHOT - - - com.intel.hibench.streambench - streaming-bench-common - jar - 5.0-SNAPSHOT - Streaming Benchmark Common - - - - org.apache.logging.log4j - log4j-api - 2.5 - - - org.apache.logging.log4j - log4j-core - 2.5 - - - log4j - log4j - 1.2.16 - - - org.slf4j - slf4j-api - 1.7.5 - - - org.slf4j - slf4j-log4j12 - 1.7.5 - - - com.codahale.metrics - metrics-jvm - 3.0.2 - - - org.apache.kafka - kafka_2.11 - 0.8.2.1 - - - - - src/main - - - net.alchim31.maven - scala-maven-plugin - - ${scala.binary.version} - ${scala.version} - - - - scala-compile-first - process-resources - - add-source - compile - - - - scala-test-compile - process-test-resources - - testCompile - - - - - - maven-assembly-plugin - ${maven-assembly-plugin.version} - - ${project.build.finalName} - - jar-with-dependencies - - - - - make-assembly - package - - single - - - - - - - diff --git a/src/streambench/common/src/main/java/com/intel/hibench/streambench/common/KMeansDataParser.java b/src/streambench/common/src/main/java/com/intel/hibench/streambench/common/KMeansDataParser.java deleted file mode 100644 index 36d1d036e..000000000 --- a/src/streambench/common/src/main/java/com/intel/hibench/streambench/common/KMeansDataParser.java +++ /dev/null @@ -1,43 +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.intel.hibench.streambench.common; - -// raw numeric data format: -// 8 {0:-60.196392992004334,5:620.4421901009101,14:420.4220612785746,13:185.21083185702275,15:483.72692251215295,1:594.7827813502976,3:140.3239790342253,16:3.104707691856035,9:635.8535653005378,19:322.0711157700041,11:87.66295667498484,18:857.7858889856491,17:101.49594891724111,2:921.839749304954,6:697.4655671122938,7:367.3720748762538,8:855.4795500704753,10:564.4074585413068,4:913.7870598326768,12:275.71369666459043} -// 9 {0:53.780307992655864,5:670.9608085434543,14:427.8278718060577,13:-42.1599560546298,15:509.38987065684455,1:575.0478527061222,3:111.01989708300927,16:48.39876690814693,9:546.0244129369196,19:344.88758399392515,11:35.63727678698427,18:826.8387868256459,17:100.39105575653751,2:972.7568962232599,6:743.3101817500838,7:367.5321255830725,8:897.5852428056947,10:705.1143980643583,4:891.1293114411877,12:364.63401807787426} -public class KMeansDataParser { - // assume maximum dimension of k means data is 2048. Should be large enough. - private static final int MAXIMUM_NUMERIC_COLUMNS = 2048; - - static public KMeansData parse(String line) { - double[] data = new double[MAXIMUM_NUMERIC_COLUMNS]; - - String[] elements = line.split("[{}:,\\s]+"); - long id = Long.parseLong(elements[0]); - - for (int i = 1; i < elements.length; i +=2) { - int index = Integer.parseInt(elements[i]); - - if (index < MAXIMUM_NUMERIC_COLUMNS) { - double value = Double.parseDouble(elements[i+1]); - data[index] = value; - } - } - return new KMeansData(id, data); - } -} diff --git a/src/streambench/common/src/main/java/com/intel/hibench/streambench/common/StreamBenchConfig.java b/src/streambench/common/src/main/java/com/intel/hibench/streambench/common/StreamBenchConfig.java deleted file mode 100644 index 1d2484c02..000000000 --- a/src/streambench/common/src/main/java/com/intel/hibench/streambench/common/StreamBenchConfig.java +++ /dev/null @@ -1,105 +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.intel.hibench.streambench.common; - -/** - * All name of configurations used in StreamBench are defined here. Later I plan to refactor - * property name. With this mapping layer, the underlying Java/Scala code don't need to be - * changed. - */ -public class StreamBenchConfig { - // ===================================== - // General StreamBench Conf - // ===================================== - public static String TESTCASE = "hibench.streambench.testCase"; - - public static String ZK_HOST = "hibench.streambench.zkHost"; - - public static String CONSUMER_GROUP = "hibench.streambench.kafka.consumerGroup"; - - public static String KAFKA_TOPIC = "hibench.streambench.kafka.topic"; - - public static String KAFKA_BROKER_LIST = "hibench.streambench.kafka.brokerList"; - - public static String KAFKA_OFFSET_RESET = "hibench.streambench.kafka.offsetReset"; - - public static String KAFKA_TOPIC_PARTITIONS = "hibench.streambench.kafka.topicPartitions"; - - public static String DEBUG_MODE = "hibench.streambench.debugMode"; - - // ===================================== - // TestCase related - // ===================================== - // TODO: Once we remove all sample testcases, this config could be removed. - public static String SAMPLE_PROBABILITY = "hibench.streambench.sampleProbability"; - - public static String FixWINDOW_DURATION = "hibench.streambench.fixWindowDuration"; - - public static String FixWINDOW_SLIDESTEP = "hibench.streambench.fixWindowSlideStep"; - - // ===================================== - // Data Generator Related Conf - // ===================================== - public static String DATAGEN_RECORDS_PRE_INTERVAL = "hibench.streambench.datagen.recordsPerInterval"; - - public static String DATAGEN_INTERVAL_SPAN = "hibench.streambench.datagen.intervalSpan"; - - public static String DATAGEN_TOTAL_RECORDS = "hibench.streambench.datagen.totalRecords"; - - public static String DATAGEN_TOTAL_ROUNDS = "hibench.streambench.datagen.totalRounds"; - - public static String DATAGEN_RECORD_LENGTH = "hibench.streambench.datagen.recordLength"; - - public static String DATAGEN_PRODUCER_NUMBER = "hibench.streambench.datagen.producerNumber"; - // ===================================== - // Spark Streaming Related Conf - // ===================================== - public static String SPARK_BATCH_INTERVAL = "hibench.streambench.spark.batchInterval"; - - public static String SPARK_CHECKPOINT_PATH = "hibench.streambench.spark.checkpointPath"; - - public static String SPARK_ENABLE_WAL = "hibench.streambench.spark.enableWAL"; - - public static String SPARK_USE_DIRECT_MODE = "hibench.streambench.spark.useDirectMode"; - - public static String SPARK_STORAGE_LEVEL = "hibench.streambench.spark.storageLevel"; - - public static String SPARK_RECEIVER_NUMBER = "hibench.streambench.spark.receiverNumber"; - - // ====================================== - // Flink Related Conf - // ====================================== - public static String FLINK_BUFFERTIMEOUT = "hibench.streambench.flink.bufferTimeout"; - - public static String FLINK_CHECKPOINTDURATION = "hibench.streambench.flink.checkpointDuration"; - - // ====================================== - // Storm Related Conf - // ====================================== - public static String STORM_WORKERCOUNT = "hibench.streambench.storm.worker_count"; - public static String STORM_SPOUT_THREADS = "hibench.streambench.storm.spout_threads"; - public static String STORM_BOLT_THREADS = "hibench.streambench.storm.bolt_threads"; - public static String STORM_ACKON = "hibench.streambench.storm.ackon"; - public static String STORM_LOCAL_SHUFFLE = "hibench.streambench.storm.localshuffle"; - - // ===================================== - // Gearpump Related Conf - // ===================================== - public static String GEARPUMP_PARALLELISM = "hibench.streambench.gearpump.parallelism"; - -} diff --git a/src/streambench/common/src/main/java/com/intel/hibench/streambench/common/TestCase.java b/src/streambench/common/src/main/java/com/intel/hibench/streambench/common/TestCase.java deleted file mode 100644 index e561f34cb..000000000 --- a/src/streambench/common/src/main/java/com/intel/hibench/streambench/common/TestCase.java +++ /dev/null @@ -1,55 +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.intel.hibench.streambench.common; - -public enum TestCase { - - // Do nothing with input events. It's useful to test the native schedule cost - IDENTITY("identity"), - - // Repartition input events to ensure data shuffle happening - REPARTITION("repartition"), - - // Wordcount is used to test the performance of state operator - WORDCOUNT("wordcount"), - - // FixWindow is used to test the performance of window operator - FIXWINDOW("fixwindow"), - - - // ====== Following TestCase hasn't been finalized ====== - PROJECT("project"), - - SAMPLE("sample"), - - GREP("grep"), - - DISTINCTCOUNT("distinctCount"), - - STATISTICS("statistics"); - // ========================================================= - - private String name; - - TestCase(String name) { - this.name = name; - } - - // Convert input name to uppercase and return related value of TestCase type - public static TestCase withValue(String name) {return TestCase.valueOf(name.toUpperCase()); } -} diff --git a/src/streambench/common/src/main/java/com/intel/hibench/streambench/common/UserVisitParser.java b/src/streambench/common/src/main/java/com/intel/hibench/streambench/common/UserVisitParser.java deleted file mode 100644 index f332f8e24..000000000 --- a/src/streambench/common/src/main/java/com/intel/hibench/streambench/common/UserVisitParser.java +++ /dev/null @@ -1,33 +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.intel.hibench.streambench.common; - -// raw uservisit table format: -// 0 227.209.164.46,nbizrgdziebsaecsecujfjcqtvnpcnxxwiopmddorcxnlijdizgoi,1991-06-10,0.115967035,Mozilla/5.0 (iPhone; U; CPU like Mac OS X)AppleWebKit/420.1 (KHTML like Gecko) Version/3.0 Mobile/4A93Safari/419.3,YEM,YEM-AR,snowdrops,1 -// 0 35.143.225.164,nbizrgdziebsaecsecujfjcqtvnpcnxxwiopmddorcxnlijdizgoi,1996-05-31,0.8792629,Mozilla/5.0 (Windows; U; Windows NT 5.2) AppleWebKit/525.13 (KHTML like Gecko) Chrome/0.2.149.27 Safari/525.13,PRT,PRT-PT,fraternally,8 -// 0 34.57.45.175,nbizrgdziebtsaecsecujfjcqtvnpcnxxwiopmddorcxnlijdizgoi,2001-06-29,0.14202267,Mozilla/4.0 (compatible; MSIE 6.0; Windows NT 5.1),DOM,DOM-ES,Gaborone's,7 -public class UserVisitParser { - - public static UserVisit parse(String line) { - String[] elements = line.split("[\\s,]+"); - String ip = elements[1]; - String sessionId = elements[2]; - String browser = elements[5]; - return new UserVisit(ip, sessionId, browser); - } -} diff --git a/src/streambench/common/src/main/scala/com/intel/hibench/streambench/common/metrics/FetchJob.scala b/src/streambench/common/src/main/scala/com/intel/hibench/streambench/common/metrics/FetchJob.scala deleted file mode 100644 index d22feae11..000000000 --- a/src/streambench/common/src/main/scala/com/intel/hibench/streambench/common/metrics/FetchJob.scala +++ /dev/null @@ -1,58 +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.intel.hibench.streambench.common.metrics - -import java.util.concurrent.Callable - -import com.codahale.metrics.Histogram - -class FetchJob(zkConnect: String, topic: String, partition: Int, - histogram: Histogram) extends Callable[FetchJobResult] { - - override def call(): FetchJobResult = { - val result = new FetchJobResult() - val consumer = new KafkaConsumer(zkConnect, topic, partition) - while (consumer.hasNext) { - val times = new String(consumer.next(), "UTF-8").split(":") - val startTime = times(0).toLong - val endTime = times(1).toLong - // correct negative value which might be caused by difference of system time - histogram.update(Math.max(0, endTime - startTime)) - result.update(startTime, endTime) - } - println(s"Collected ${result.count} results for partition: ${partition}") - result - } -} - -class FetchJobResult(var minTime: Long, var maxTime: Long, var count: Long) { - - def this() = this(Long.MaxValue, Long.MinValue, 0) - - def update(startTime: Long ,endTime: Long): Unit = { - count += 1 - - if(startTime < minTime) { - minTime = startTime - } - - if(endTime > maxTime) { - maxTime = endTime - } - } -} diff --git a/src/streambench/common/src/main/scala/com/intel/hibench/streambench/common/metrics/KafkaCollector.scala b/src/streambench/common/src/main/scala/com/intel/hibench/streambench/common/metrics/KafkaCollector.scala deleted file mode 100644 index 70a5f0f83..000000000 --- a/src/streambench/common/src/main/scala/com/intel/hibench/streambench/common/metrics/KafkaCollector.scala +++ /dev/null @@ -1,114 +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.intel.hibench.streambench.common.metrics - -import java.io.{FileWriter, File} -import java.util.Date -import java.util.concurrent.{TimeUnit, Future, Executors} - -import com.codahale.metrics.{UniformReservoir, Histogram} -import kafka.utils.{ZKStringSerializer, ZkUtils} -import org.I0Itec.zkclient.ZkClient - -import scala.collection.mutable.ArrayBuffer - - -class KafkaCollector(zkConnect: String, metricsTopic: String, - outputDir: String, sampleNumber: Int, desiredThreadNum: Int) extends LatencyCollector { - - private val histogram = new Histogram(new UniformReservoir(sampleNumber)) - private val threadPool = Executors.newFixedThreadPool(desiredThreadNum) - private val fetchResults = ArrayBuffer.empty[Future[FetchJobResult]] - - def start(): Unit = { - val partitions = getPartitions(metricsTopic, zkConnect) - - println("Starting MetricsReader for kafka topic: " + metricsTopic) - - partitions.foreach(partition => { - val job = new FetchJob(zkConnect, metricsTopic, partition, histogram) - val fetchFeature = threadPool.submit(job) - fetchResults += fetchFeature - }) - - threadPool.shutdown() - threadPool.awaitTermination(30, TimeUnit.MINUTES) - - val finalResults = fetchResults.map(_.get()).reduce((a, b) => { - val minTime = Math.min(a.minTime, b.minTime) - val maxTime = Math.max(a.maxTime, b.maxTime) - val count = a.count + b.count - new FetchJobResult(minTime, maxTime, count) - }) - - report(finalResults.minTime, finalResults.maxTime, finalResults.count) - } - - private def getPartitions(topic: String, zkConnect: String): Seq[Int] = { - val zkClient = new ZkClient(zkConnect, 6000, 6000, ZKStringSerializer) - try { - ZkUtils.getPartitionsForTopics(zkClient, Seq(topic)).flatMap(_._2).toSeq - } finally { - zkClient.close() - } - } - - - private def report(minTime: Long, maxTime: Long, count: Long): Unit = { - val outputFile = new File(outputDir, metricsTopic + ".csv") - println(s"written out metrics to ${outputFile.getCanonicalPath}") - val header = "time,count,throughput(msgs/s),max_latency(ms),mean_latency(ms),min_latency(ms)," + - "stddev_latency(ms),p50_latency(ms),p75_latency(ms),p95_latency(ms),p98_latency(ms)," + - "p99_latency(ms),p999_latency(ms)\n" - val fileExists = outputFile.exists() - if (!fileExists) { - val parent = outputFile.getParentFile - if (!parent.exists()) { - parent.mkdirs() - } - outputFile.createNewFile() - } - val outputFileWriter = new FileWriter(outputFile, true) - if (!fileExists) { - outputFileWriter.append(header) - } - val time = new Date(System.currentTimeMillis()).toString - val count = histogram.getCount - val snapshot = histogram.getSnapshot - val throughput = count * 1000 / (maxTime - minTime) - outputFileWriter.append(s"$time,$count,$throughput," + - s"${formatDouble(snapshot.getMax)}," + - s"${formatDouble(snapshot.getMean)}," + - s"${formatDouble(snapshot.getMin)}," + - s"${formatDouble(snapshot.getStdDev)}," + - s"${formatDouble(snapshot.getMedian)}," + - s"${formatDouble(snapshot.get75thPercentile())}," + - s"${formatDouble(snapshot.get95thPercentile())}," + - s"${formatDouble(snapshot.get98thPercentile())}," + - s"${formatDouble(snapshot.get99thPercentile())}," + - s"${formatDouble(snapshot.get999thPercentile())}\n") - outputFileWriter.close() - } - - private def formatDouble(d: Double): String = { - "%.3f".format(d) - } - -} - - diff --git a/src/streambench/common/src/main/scala/com/intel/hibench/streambench/common/metrics/KafkaConsumer.scala b/src/streambench/common/src/main/scala/com/intel/hibench/streambench/common/metrics/KafkaConsumer.scala deleted file mode 100644 index 252d473f2..000000000 --- a/src/streambench/common/src/main/scala/com/intel/hibench/streambench/common/metrics/KafkaConsumer.scala +++ /dev/null @@ -1,98 +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.intel.hibench.streambench.common.metrics - -import java.util.Properties - -import kafka.api.{OffsetRequest, FetchRequestBuilder} -import kafka.common.ErrorMapping._ -import kafka.common.TopicAndPartition -import kafka.consumer.{ConsumerConfig, SimpleConsumer} -import kafka.message.MessageAndOffset -import kafka.utils.{ZKStringSerializer, ZkUtils, Utils} -import org.I0Itec.zkclient.ZkClient - -class KafkaConsumer(zookeeperConnect: String, topic: String, partition: Int) { - - private val CLIENT_ID = "metrics_reader" - private val props = new Properties() - props.put("zookeeper.connect", zookeeperConnect) - props.put("group.id", CLIENT_ID) - private val config = new ConsumerConfig(props) - private val consumer = createConsumer - - private val earliestOffset = consumer - .earliestOrLatestOffset(TopicAndPartition(topic, partition), OffsetRequest.EarliestTime, -1) - private var nextOffset: Long = earliestOffset - private var iterator: Iterator[MessageAndOffset] = getIterator(nextOffset) - - def next(): Array[Byte] = { - val mo = iterator.next() - val message = mo.message - - nextOffset = mo.nextOffset - - Utils.readBytes(message.payload) - } - - def hasNext: Boolean = { - @annotation.tailrec - def hasNextHelper(iter: Iterator[MessageAndOffset], newIterator: Boolean): Boolean = { - if (iter.hasNext) true - else if (newIterator) false - else { - iterator = getIterator(nextOffset) - hasNextHelper(iterator, newIterator = true) - } - } - hasNextHelper(iterator, newIterator = false) - } - - def close(): Unit = { - consumer.close() - } - - private def createConsumer: SimpleConsumer = { - val zkClient = new ZkClient(zookeeperConnect, 6000, 6000, ZKStringSerializer) - try { - val leader = ZkUtils.getLeaderForPartition(zkClient, topic, partition) - .getOrElse(throw new RuntimeException( - s"leader not available for TopicAndPartition($topic, $partition)")) - val broker = ZkUtils.getBrokerInfo(zkClient, leader) - .getOrElse(throw new RuntimeException(s"broker info not found for leader $leader")) - new SimpleConsumer(broker.host, broker.port, - config.socketTimeoutMs, config.socketReceiveBufferBytes, CLIENT_ID) - } catch { - case e: Exception => - throw e - } finally { - zkClient.close() - } - } - - private def getIterator(offset: Long): Iterator[MessageAndOffset] = { - val request = new FetchRequestBuilder() - .addFetch(topic, partition, offset, config.fetchMessageMaxBytes) - .build() - - val response = consumer.fetch(request) - response.errorCode(topic, partition) match { - case NoError => response.messageSet(topic, partition).iterator - case error => throw exceptionFor(error) - } - } -} diff --git a/src/streambench/common/src/main/scala/com/intel/hibench/streambench/common/metrics/KafkaReporter.scala b/src/streambench/common/src/main/scala/com/intel/hibench/streambench/common/metrics/KafkaReporter.scala deleted file mode 100644 index 65111218b..000000000 --- a/src/streambench/common/src/main/scala/com/intel/hibench/streambench/common/metrics/KafkaReporter.scala +++ /dev/null @@ -1,51 +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.intel.hibench.streambench.common.metrics - -import java.util.Properties - -import org.apache.kafka.clients.producer.{ProducerRecord, KafkaProducer} -import org.apache.kafka.common.serialization.StringSerializer - -/** - * This has to be created at server side - */ -class KafkaReporter(topic: String, bootstrapServers: String) extends LatencyReporter { - - private val producer = ProducerSingleton.getInstance(bootstrapServers) - - override def report(startTime: Long, endTime: Long): Unit = { - producer.send(new ProducerRecord[String, String](topic, null, s"$startTime:$endTime")) - } -} - -object ProducerSingleton { - @volatile private var instance : Option[KafkaProducer[String, String]] = None - - def getInstance(bootstrapServers: String): KafkaProducer[String, String] = synchronized { - if (!instance.isDefined) { - synchronized { - if(!instance.isDefined) { - val props = new Properties() - props.put("bootstrap.servers", bootstrapServers) - instance = Some(new KafkaProducer(props, new StringSerializer, new StringSerializer)) - } - } - } - instance.get - } -} \ No newline at end of file diff --git a/src/streambench/common/src/main/scala/com/intel/hibench/streambench/common/metrics/MetricsUtil.scala b/src/streambench/common/src/main/scala/com/intel/hibench/streambench/common/metrics/MetricsUtil.scala deleted file mode 100644 index 8e7bf757b..000000000 --- a/src/streambench/common/src/main/scala/com/intel/hibench/streambench/common/metrics/MetricsUtil.scala +++ /dev/null @@ -1,50 +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.intel.hibench.streambench.common.metrics - -import com.intel.hibench.streambench.common.Platform -import kafka.admin.AdminUtils -import kafka.utils.ZKStringSerializer -import org.I0Itec.zkclient.ZkClient - -object MetricsUtil { - - val TOPIC_CONF_FILE_NAME = "metrics_topic.conf" - - def getTopic(platform: Platform, sourceTopic: String, producerNum: Int, - recordPerInterval: Long, intervalSpan: Int): String = { - val topic = s"${platform}_${sourceTopic}_${producerNum}_${recordPerInterval}" + - s"_${intervalSpan}_${System.currentTimeMillis()}" - println(s"metrics is being written to kafka topic $topic") - topic - } - - def createTopic(zkConnect: String, topic: String, partitions: Int): Unit = { - val zkClient = new ZkClient(zkConnect, 6000, 6000, ZKStringSerializer) - try { - AdminUtils.createTopic(zkClient, topic, partitions, 1) - while (!AdminUtils.topicExists(zkClient, topic)) { - Thread.sleep(100) - } - } catch { - case e: Exception => - throw e - } finally { - zkClient.close() - } - } -} diff --git a/src/streambench/datagen/lib/kafka-clients-0.8.1.jar b/src/streambench/datagen/lib/kafka-clients-0.8.1.jar new file mode 100644 index 000000000..d77c0fef8 Binary files /dev/null and b/src/streambench/datagen/lib/kafka-clients-0.8.1.jar differ diff --git a/src/streambench/datagen/pom.xml b/src/streambench/datagen/pom.xml index e8b4e9fd4..b2b435b54 100644 --- a/src/streambench/datagen/pom.xml +++ b/src/streambench/datagen/pom.xml @@ -1,82 +1,60 @@ - - - - 4.0.0 - - - com.intel.hibench - streambench - 5.0-SNAPSHOT - + xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> + 4.0.0 - com.intel.hibench.streambench - streaming-bench-datagen - 5.0-SNAPSHOT - Streaming Benchmark Data Generation Tool + + com.intel.hibench + streambench + 5.0-SNAPSHOT + - - - org.apache.kafka - kafka-clients - 0.8.2.2 - - - org.apache.hadoop - hadoop-hdfs - ${hadoop.mr2.version} - - - org.apache.hadoop - hadoop-client - ${hadoop.mr2.version} - + com.intel.hibench.streambench + datagen + 0.0.1 + Streamingbench data generation tools + + + org.apache.kafka + kafka-clients + 0.8.1 + system + ${basedir}/lib/kafka-clients-0.8.1.jar + - com.intel.hibench.streambench - streaming-bench-common - 5.0-SNAPSHOT + org.apache.hadoop + hadoop-hdfs + ${hadoop.mr2.version} - + + org.apache.hadoop + hadoop-client + ${hadoop.mr2.version} + + - - - - maven-assembly-plugin - ${maven-assembly-plugin.version} - - ${project.build.finalName} - - jar-with-dependencies - - - - - make-assembly - package - - single - - - - - - + + + + maven-assembly-plugin + ${maven-assembly-plugin.version} + + ${project.build.finalName} + + jar-with-dependencies + + + + + make-assembly + package + + single + + + + + + diff --git a/src/streambench/datagen/src/main/java/com/intel/hibench/streambench/DataGenerator.java b/src/streambench/datagen/src/main/java/com/intel/hibench/streambench/DataGenerator.java deleted file mode 100644 index 523131927..000000000 --- a/src/streambench/datagen/src/main/java/com/intel/hibench/streambench/DataGenerator.java +++ /dev/null @@ -1,126 +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.intel.hibench.streambench; - -import com.intel.hibench.common.HiBenchConfig; -import com.intel.hibench.streambench.common.ConfigLoader; -import com.intel.hibench.streambench.common.StreamBenchConfig; -import com.intel.hibench.streambench.util.DataGeneratorConfig; -import com.intel.hibench.streambench.util.KafkaSender; -import com.intel.hibench.streambench.util.RecordSendTask; - -import java.util.Timer; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; - -public class DataGenerator { - - public static void main(String[] args) { - if (args.length < 5) { - System.err.println("args: need to be specified!"); - System.exit(1); - } - - // initialize variable from configuration and input parameters. - ConfigLoader configLoader = new ConfigLoader(args[0]); - - String userVisitsFile = args[1]; - long userVisitsFileOffset = Long.parseLong(args[2]); - String kMeansFile = args[3]; - long kMeansFileOffset = Long.parseLong(args[4]); - - // load properties from config file - String testCase = configLoader.getProperty(StreamBenchConfig.TESTCASE).toLowerCase(); - String topic = configLoader.getProperty(StreamBenchConfig.KAFKA_TOPIC); - String brokerList = configLoader.getProperty(StreamBenchConfig.KAFKA_BROKER_LIST); - int intervalSpan = Integer.parseInt(configLoader.getProperty(StreamBenchConfig.DATAGEN_INTERVAL_SPAN)); - long recordsPerInterval = Long.parseLong(configLoader.getProperty(StreamBenchConfig.DATAGEN_RECORDS_PRE_INTERVAL)); - long totalRecords = Long.parseLong(configLoader.getProperty(StreamBenchConfig.DATAGEN_TOTAL_RECORDS)); - int totalRounds = Integer.parseInt(configLoader.getProperty(StreamBenchConfig.DATAGEN_TOTAL_ROUNDS)); - int recordLength = Integer.parseInt(configLoader.getProperty(StreamBenchConfig.DATAGEN_RECORD_LENGTH)); - String dfsMaster = configLoader.getProperty(HiBenchConfig.DFS_MASTER); - boolean debugMode = Boolean.getBoolean(configLoader.getProperty(StreamBenchConfig.DEBUG_MODE)); - - DataGeneratorConfig dataGeneratorConf = new DataGeneratorConfig(testCase, brokerList, kMeansFile, kMeansFileOffset, - userVisitsFile, userVisitsFileOffset, dfsMaster, recordLength, intervalSpan, topic, recordsPerInterval, - totalRounds, totalRecords, debugMode); - - // Create thread pool and submit producer task - int producerNumber = Integer.parseInt(configLoader.getProperty(StreamBenchConfig.DATAGEN_PRODUCER_NUMBER)); - ExecutorService pool = Executors.newFixedThreadPool(producerNumber); - for(int i = 0; i < producerNumber; i++) { - pool.execute(new DataGeneratorJob(dataGeneratorConf)); - } - - // Print out some basic information - System.out.println("============ StreamBench Data Generator ============"); - System.out.println(" Interval Span : " + intervalSpan + " ms"); - System.out.println(" Record Per Interval : " + recordsPerInterval); - System.out.println(" Record Length : " + recordLength + " bytes"); - System.out.println(" Producer Number : " + producerNumber); - if(totalRecords == -1) { - System.out.println(" Total Records : -1 [Infinity]"); - } else { - System.out.println(" Total Records : " + totalRecords); - } - - if (totalRounds == -1) { - System.out.println(" Total Rounds : -1 [Infinity]"); - } else { - System.out.println(" Total Rounds : " + totalRounds); - } - System.out.println(" Kafka Topic : " + topic); - System.out.println("===================================================="); - System.out.println("Estimated Speed : "); - long recordsPreSecond = recordsPerInterval * 1000 * producerNumber / intervalSpan ; - System.out.println(" " + recordsPreSecond + " records/second"); - double mbPreSecond = (double)recordsPreSecond * recordLength / 1000000; - System.out.println(" " + mbPreSecond + " Mb/second"); - System.out.println("===================================================="); - - pool.shutdown(); - } - - static class DataGeneratorJob implements Runnable { - DataGeneratorConfig conf; - - // Constructor - public DataGeneratorJob(DataGeneratorConfig conf) { - this.conf = conf; - } - - @Override - public void run() { - // instantiate KafkaSender - KafkaSender sender; - if(conf.getTestCase().contains("statistics")) { - sender = new KafkaSender(conf.getBrokerList(), conf.getkMeansFile(), conf.getkMeansFileOffset(), - conf.getDfsMaster(), conf.getRecordLength(), conf.getIntervalSpan()); - } else { - sender = new KafkaSender(conf.getBrokerList(), conf.getUserVisitsFile(), conf.getUserVisitsFileOffset(), - conf.getDfsMaster(), conf.getRecordLength(), conf.getIntervalSpan()); - } - - // Schedule timer task - Timer timer = new Timer(); - timer.scheduleAtFixedRate( - new RecordSendTask(sender, conf.getTopic(), conf.getRecordsPerInterval(), - conf.getTotalRounds(), conf.getTotalRecords(), conf.getDebugMode(), timer), 0, conf.getIntervalSpan()); - } - } -} diff --git a/src/streambench/datagen/src/main/java/com/intel/hibench/streambench/FileDataGenNew.java b/src/streambench/datagen/src/main/java/com/intel/hibench/streambench/FileDataGenNew.java new file mode 100644 index 000000000..f20e700cb --- /dev/null +++ b/src/streambench/datagen/src/main/java/com/intel/hibench/streambench/FileDataGenNew.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 com.intel.hibench.streambench; + +import java.io.*; +import java.util.Enumeration; +import java.util.Vector; + +import org.apache.hadoop.fs.*; +import org.apache.hadoop.conf.*; + +public class FileDataGenNew { + Configuration fsConf = new Configuration(); + + FileDataGenNew(String HDFSMaster) { + fsConf.set("fs.default.name", HDFSMaster); + } + + public BufferedReader loadDataFromFile(String filepath, long offset) { + try { + Path pt = new Path(filepath); + FileSystem fs = FileSystem.get(fsConf); + InputStreamReader isr; + if (fs.isDirectory(pt)) { // multiple parts + isr = new InputStreamReader(OpenMultiplePartsWithOffset(fs, pt, offset)); + } else { // single file + FSDataInputStream fileHandler = fs.open(pt); + if (offset > 0) fileHandler.seek(offset); + isr = new InputStreamReader(fileHandler); + } + + BufferedReader reader = new BufferedReader(isr); + if (offset > 0) reader.readLine(); // skip first line in case of seek + return reader; + } catch (FileNotFoundException e) { + e.printStackTrace(); + } catch (IOException e) { + e.printStackTrace(); + } + return null; + } + + private InputStream OpenMultiplePartsWithOffset(FileSystem fs, Path pt, long offset) throws IOException { + System.out.println("Opening files, path:" + pt + " offset:" + offset); + RemoteIterator rit = fs.listFiles(pt, false); + Vector fileHandleList = new Vector(); + while (rit.hasNext()) { + Path path = rit.next().getPath(); + String filename = path.toString().substring(path.getParent().toString().length(), path.toString().length()); + + if (filename.startsWith("/part-")) { + long filesize = fs.getFileStatus(path).getLen(); + if (offset < filesize) { + FSDataInputStream handle = fs.open(path); + if (offset > 0) { + handle.seek(offset); + } + fileHandleList.add(handle); + } + offset -= filesize; + } + } + if (fileHandleList.size() == 1) return fileHandleList.get(0); + else if (fileHandleList.size() > 1) { + Enumeration enu = fileHandleList.elements(); + return new SequenceInputStream(enu); + } else { + System.err.println("Error, no source file loaded. run genSeedDataset.sh first!"); + return null; + } + } +} diff --git a/src/streambench/datagen/src/main/java/com/intel/hibench/streambench/NewKafkaConnector.java b/src/streambench/datagen/src/main/java/com/intel/hibench/streambench/NewKafkaConnector.java new file mode 100644 index 000000000..26d77638d --- /dev/null +++ b/src/streambench/datagen/src/main/java/com/intel/hibench/streambench/NewKafkaConnector.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.intel.hibench.streambench; + +import java.io.BufferedReader; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.util.Properties; + +import com.intel.hibench.streambench.utils.ConfigLoader; + +import org.apache.kafka.clients.producer.Callback; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.RecordMetadata; + +public class NewKafkaConnector { + + KafkaProducer producer; + private static final int MAXIMUM_NUMERIC_COLUMNS = 2048; // assume maximum dimension of k means data is 2048. Should be large enough. + private Integer[] NumericData = new Integer[MAXIMUM_NUMERIC_COLUMNS]; + private int Data1Length; + + public NewKafkaConnector(String brokerList, ConfigLoader cl) { + Properties props = new Properties(); + props.setProperty(ProducerConfig.REQUIRED_ACKS_CONFIG, "1"); + props.setProperty(ProducerConfig.BROKER_LIST_CONFIG, brokerList); + props.setProperty(ProducerConfig.METADATA_FETCH_TIMEOUT_CONFIG, Integer.toString(5 * 1000)); + props.setProperty(ProducerConfig.REQUEST_TIMEOUT_CONFIG, Integer.toString(Integer.MAX_VALUE)); + producer = new KafkaProducer(props); + Data1Length = Integer.parseInt(cl.getProperty("hibench.streamingbench.datagen.data1.length")); + } + + private String parseUserVisitTable(String line, int MaximumLength) { + // raw uservisit table format: + // 0 227.209.164.46,nbizrgdziebsaecsecujfjcqtvnpcnxxwiopmddorcxnlijdizgoi,1991-06-10,0.115967035,Mozilla/5.0 (iPhone; U; CPU like Mac OS X)AppleWebKit/420.1 (KHTML like Gecko) Version/3.0 Mobile/4A93Safari/419.3,YEM,YEM-AR,snowdrops,1 + // 0 35.143.225.164,nbizrgdziebsaecsecujfjcqtvnpcnxxwiopmddorcxnlijdizgoi,1996-05-31,0.8792629,Mozilla/5.0 (Windows; U; Windows NT 5.2) AppleWebKit/525.13 (KHTML like Gecko) Chrome/0.2.149.27 Safari/525.13,PRT,PRT-PT,fraternally,8 + // 0 34.57.45.175,nbizrgdziebtsaecsecujfjcqtvnpcnxxwiopmddorcxnlijdizgoi,2001-06-29,0.14202267,Mozilla/4.0 (compatible; MSIE 6.0; Windows NT 5.1),DOM,DOM-ES,Gaborone's,7 + + String[] elements = line.split("[\\s,]+"); + StringBuffer sb = new StringBuffer(); + sb.append(elements[0]); + sb.append(elements[1]); + sb.append(elements[3] + " 00:00:00"); + sb.append(elements[4]); + sb.append(elements[2]); + + String result = sb.toString(); + return String.format("%s%n", (result.length() < MaximumLength) ? result : result.substring(0, MaximumLength)); + } + + private String parseNumeric(String line) { + // raw numeric data format: + // 8 {0:-60.196392992004334,5:620.4421901009101,14:420.4220612785746,13:185.21083185702275,15:483.72692251215295,1:594.7827813502976,3:140.3239790342253,16:3.104707691856035,9:635.8535653005378,19:322.0711157700041,11:87.66295667498484,18:857.7858889856491,17:101.49594891724111,2:921.839749304954,6:697.4655671122938,7:367.3720748762538,8:855.4795500704753,10:564.4074585413068,4:913.7870598326768,12:275.71369666459043} + // 9 {0:53.780307992655864,5:670.9608085434543,14:427.8278718060577,13:-42.1599560546298,15:509.38987065684455,1:575.0478527061222,3:111.01989708300927,16:48.39876690814693,9:546.0244129369196,19:344.88758399392515,11:35.63727678698427,18:826.8387868256459,17:100.39105575653751,2:972.7568962232599,6:743.3101817500838,7:367.5321255830725,8:897.5852428056947,10:705.1143980643583,4:891.1293114411877,12:364.63401807787426} + + String[] elements = line.split("[{}:,\\s]+"); + int idx = -1; + int maxidx = -1; + for (int count = 0; count < elements.length; count++) { + if (count == 0) continue; //omit first element + if (count % 2 == 1) idx = Integer.parseInt(elements[count]); + else { + int val = (int) Float.parseFloat(elements[count]); + assert idx >= 0 : String.format("index value should be greater than zero!, got:%d", idx); + assert idx < MAXIMUM_NUMERIC_COLUMNS : String.format("index value %d exceed range of %d", idx, MAXIMUM_NUMERIC_COLUMNS); + NumericData[idx] = val; + if (maxidx < idx) maxidx = idx; + } + } + + StringBuilder sb = new StringBuilder(); + for (int i = 0; i < NumericData.length; i++) { + int val = NumericData[i]; + sb.append(val + " "); + } + String result = sb.toString(); + return String.format("%s%n", result.substring(0, result.length() - 1)); + } + + /** + * Returns the actual number of sent records + */ + public long sendRecords(BufferedReader reader, String topic, long totalRecord, boolean isNumericData) { + long start = System.currentTimeMillis(); + long bytesSent = 0L; + long recordsSent = 0L; + + Callback callback = new Callback() { + public void onCompletion(RecordMetadata metadata, Exception e) { + if (e != null) + e.printStackTrace(); + } + }; + + ByteArrayOutputStream ous = new ByteArrayOutputStream(); + try { + while (recordsSent < totalRecord) { + String line = reader.readLine(); + if (line == null) { + break; + } + if (isNumericData) { + ous.write(parseNumeric(line).getBytes()); + } else { + ous.write(parseUserVisitTable(line, Data1Length).getBytes()); + } + if (ous.size() == 0) { + break; // no more data got, let's break + } + ProducerRecord record = new ProducerRecord(topic, ous.toByteArray()); + producer.send(record, callback); + + recordsSent ++; + bytesSent += ous.size(); + ous.reset(); + } + ous.close(); + } catch (IOException e) { + e.printStackTrace(); + } + + long end = System.currentTimeMillis(); + System.out.println("Records sent: " + recordsSent); + System.out.println("Bytes sent: " + bytesSent); + System.out.println("Time consumed(sec):" + (end - start) / 1000.0); + double seconds = (double) (end - start) / (double) 1000; + double throughput = ((double) bytesSent / seconds) / 1000000; + System.out.println("Throughput: " + throughput + "MB/s"); + + return recordsSent; + } + + public void close() { + producer.close(); + } +} diff --git a/src/streambench/datagen/src/main/java/com/intel/hibench/streambench/StartNew.java b/src/streambench/datagen/src/main/java/com/intel/hibench/streambench/StartNew.java new file mode 100644 index 000000000..17cc065b8 --- /dev/null +++ b/src/streambench/datagen/src/main/java/com/intel/hibench/streambench/StartNew.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.intel.hibench.streambench; + +import com.intel.hibench.streambench.utils.ConfigLoader; + +import java.io.BufferedReader; + +//Data generators are deployed in different nodes and run by launching them near simultaneously in different nodes. +public class StartNew { + + private static String benchName; + private static String HDFSMaster; + private static String dataFile1; + private static long dataFile1Offset; + private static String dataFile2; + private static long dataFile2Offset; + + public static void main(String[] args) { + if (args.length < 5) { + System.err.println("args: need to be specified!"); + System.exit(1); + } + + ConfigLoader cl = new ConfigLoader(args[0]); + + benchName = cl.getProperty("hibench.streamingbench.benchname").toLowerCase(); + String topic = cl.getProperty("hibench.streamingbench.topic_name"); + String brokerList = cl.getProperty("hibench.streamingbench.brokerList"); + HDFSMaster = cl.getProperty("hibench.hdfs.master"); + long totalCount = Long.parseLong(cl.getProperty("hibench.streamingbench.prepare.push.records")); + dataFile1 = args[1]; + dataFile1Offset = Long.parseLong(args[2]); + dataFile2 = args[3]; + dataFile2Offset = Long.parseLong(args[4]); + boolean isNumericData = false; + if (benchName.contains("statistics")) { + isNumericData = true; + } + + NewKafkaConnector con = new NewKafkaConnector(brokerList, cl); + + long recordsSent = 0L; + while (recordsSent < totalCount) { + recordsSent += con.sendRecords(getReader(), topic, totalCount - recordsSent, isNumericData); + } + + con.close(); + } + + public static BufferedReader getReader() { + FileDataGenNew files = new FileDataGenNew(HDFSMaster); + if (benchName.contains("statistics")) { + return files.loadDataFromFile(dataFile2, dataFile2Offset); + } else { + return files.loadDataFromFile(dataFile1, dataFile1Offset); + } + } +} diff --git a/src/streambench/datagen/src/main/java/com/intel/hibench/streambench/StartPeriodic.java b/src/streambench/datagen/src/main/java/com/intel/hibench/streambench/StartPeriodic.java new file mode 100644 index 000000000..06db678a9 --- /dev/null +++ b/src/streambench/datagen/src/main/java/com/intel/hibench/streambench/StartPeriodic.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.intel.hibench.streambench; + +import com.intel.hibench.streambench.utils.ConfigLoader; + +import java.io.BufferedReader; +import java.util.Timer; +import java.util.TimerTask; + + +public class StartPeriodic { + + private static String benchName; + private static String HDFSMaster; + private static String dataFile1; + private static long dataFile1Offset; + private static String dataFile2; + private static long dataFile2Offset; + + public static void main(String[] args) { + + if (args.length < 5) { + System.err.println("args: need to be specified!"); + System.exit(1); + } + + ConfigLoader cl = new ConfigLoader(args[0]); + + benchName = cl.getProperty("hibench.streamingbench.benchname").toLowerCase(); + String topic = cl.getProperty("hibench.streamingbench.topic_name"); + String brokerList = cl.getProperty("hibench.streamingbench.brokerList"); + int recordPerInterval = Integer.parseInt(cl.getProperty("hibench.streamingbench.prepare.periodic.recordPerInterval")); + int intervalSpan = Integer.parseInt(cl.getProperty("hibench.streamingbench.prepare.periodic.intervalSpan")); + int totalRound = Integer.parseInt(cl.getProperty("hibench.streamingbench.prepare.periodic.totalRound")); + HDFSMaster = cl.getProperty("hibench.hdfs.master"); + dataFile1 = args[1]; + dataFile1Offset = Long.parseLong(args[2]); + dataFile2 = args[3]; + dataFile2Offset = Long.parseLong(args[4]); + boolean isNumericData = false; + if (benchName.contains("statistics")) { + isNumericData = true; + } + + NewKafkaConnector con = new NewKafkaConnector(brokerList, cl); + + Timer timer = new Timer(); + timer.schedule(new SendTask(totalRound, recordPerInterval, con, topic, isNumericData), 0, intervalSpan); + System.out.println("Timer scheduled."); + } + + public static BufferedReader getReader() { + FileDataGenNew files = new FileDataGenNew(HDFSMaster); + if (benchName.contains("statistics")) { + return files.loadDataFromFile(dataFile2, dataFile2Offset); + } else { + return files.loadDataFromFile(dataFile1, dataFile1Offset); + } + } + + static class SendTask extends TimerTask { + int leftTimes; + int recordCount; + int totalTimes; + NewKafkaConnector kafkaCon; + String topic; + long totalRecords; + boolean isNumericData; + + public SendTask(int times, int count, NewKafkaConnector con, String topic, boolean isNumericData) { + leftTimes = times; + recordCount = count; + totalTimes = times; + kafkaCon = con; + this.topic = topic; + totalRecords = 0; + this.isNumericData = isNumericData; + } + + @Override + public void run() { + System.out.println("Task run, remains:" + leftTimes); + if (leftTimes > 0) { + long recordsSent = 0L; + while (recordsSent < recordCount) { + recordsSent += kafkaCon.sendRecords(getReader(), topic, recordCount - recordsSent, isNumericData); + } + totalRecords += recordsSent; + leftTimes--; + } else { + System.out.println("Time's up! Total records sent:" + totalRecords); + kafkaCon.close(); + System.exit(0); + } + } + } +} diff --git a/src/streambench/datagen/src/main/java/com/intel/hibench/streambench/util/CachedData.java b/src/streambench/datagen/src/main/java/com/intel/hibench/streambench/util/CachedData.java deleted file mode 100644 index dc712d5b2..000000000 --- a/src/streambench/datagen/src/main/java/com/intel/hibench/streambench/util/CachedData.java +++ /dev/null @@ -1,80 +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.intel.hibench.streambench.util; - -import org.apache.hadoop.conf.Configuration; - -import java.io.BufferedReader; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; - -/** - * Cache the total records in memory. - */ -public class CachedData { - - private volatile static CachedData cachedData; - - private List data; - - private int next; - private int totalRecords; - - public static CachedData getInstance(String seedFile, long fileOffset, String dfsMaster) { - if(cachedData == null) { - synchronized (CachedData.class) { - if (cachedData == null) { - cachedData = new CachedData(seedFile, fileOffset, dfsMaster); - } - } - } - return cachedData; - } - - private CachedData(String seedFile, long fileOffset, String dfsMaster){ - Configuration dfsConf = new Configuration(); - dfsConf.set("fs.default.name", dfsMaster); - - // read records from seedFile and cache into "data" - data = new ArrayList(); - BufferedReader reader = SourceFileReader.getReader(dfsConf, seedFile, fileOffset); - String line = null; - try { - while ((line = reader.readLine()) != null) { - data.add(line); - } - } catch (IOException e) { - System.err.println("Failed read records from Path: " + seedFile); - e.printStackTrace(); - } - - this.next = 0; - this.totalRecords = data.size(); - } - - /** - * Loop get record. - * @return the record. - */ - public String getRecord() { - int current = next; - next = (next + 1) % totalRecords; - return data.get(current); - } -} diff --git a/src/streambench/datagen/src/main/java/com/intel/hibench/streambench/util/DataGeneratorConfig.java b/src/streambench/datagen/src/main/java/com/intel/hibench/streambench/util/DataGeneratorConfig.java deleted file mode 100644 index e900817f0..000000000 --- a/src/streambench/datagen/src/main/java/com/intel/hibench/streambench/util/DataGeneratorConfig.java +++ /dev/null @@ -1,107 +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.intel.hibench.streambench.util; - -// A POJO class to contain necessary configuration -public class DataGeneratorConfig { - String testCase; - String brokerList; - String kMeansFile; - long kMeansFileOffset; - String userVisitsFile; - long userVisitsFileOffset; - String dfsMaster; - int recordLength; - int intervalSpan; - String topic; - long recordsPerInterval; - int totalRounds; - long totalRecords; - boolean debugMode; - - public DataGeneratorConfig(String testCase, String brokerList, String kMeansFile, long kMeansFileOffset, - String userVisitsFile, long userVisitsFileOffset, String dfsMaster, int recordLength, int intervalSpan, - String topic, long recordsPerInterval, int totalRounds, long totalRecords, boolean debugMode) { - this.testCase = testCase; - this.brokerList = brokerList; - this.kMeansFile = kMeansFile; - this.kMeansFileOffset = kMeansFileOffset; - this.userVisitsFile = userVisitsFile; - this.userVisitsFileOffset = userVisitsFileOffset; - this.dfsMaster = dfsMaster; - this.recordLength = recordLength; - this.intervalSpan = intervalSpan; - this.topic = topic; - this.recordsPerInterval = recordsPerInterval; - this.totalRounds = totalRounds; - this.totalRecords = totalRecords; - this.debugMode = debugMode; - } - - public String getTestCase() { - return testCase; - } - - public String getBrokerList() { - return brokerList; - } - - public String getkMeansFile() { - return kMeansFile; - } - - public long getkMeansFileOffset() { - return kMeansFileOffset; - } - - public String getUserVisitsFile() { - return userVisitsFile; - } - - public long getUserVisitsFileOffset() { - return userVisitsFileOffset; - } - - public String getDfsMaster() { - return dfsMaster; - } - - public int getRecordLength() { - return recordLength; - } - - public int getIntervalSpan() { return intervalSpan; } - - public String getTopic() { - return topic; - } - - public long getRecordsPerInterval() { - return recordsPerInterval; - } - - public int getTotalRounds() { - return totalRounds; - } - - public long getTotalRecords() { - return totalRecords; - } - - public boolean getDebugMode() { return debugMode; } -} diff --git a/src/streambench/datagen/src/main/java/com/intel/hibench/streambench/util/KafkaSender.java b/src/streambench/datagen/src/main/java/com/intel/hibench/streambench/util/KafkaSender.java deleted file mode 100644 index 571314029..000000000 --- a/src/streambench/datagen/src/main/java/com/intel/hibench/streambench/util/KafkaSender.java +++ /dev/null @@ -1,121 +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.intel.hibench.streambench.util; - -import java.util.Arrays; -import java.util.Properties; - -import org.apache.kafka.clients.producer.*; -import org.apache.kafka.common.serialization.StringSerializer; - -/** - * KafkaSender hold an kafka producer. It gets content from input parameter, generates records and - * sends records to kafka. - */ -public class KafkaSender { - - KafkaProducer kafkaProducer; - CachedData cachedData; - int recordLength; - int intervalSpan; - - StringSerializer serializer = new StringSerializer(); - - // Constructor - public KafkaSender(String brokerList, String seedFile, - long fileOffset, String dfsMaster, int recordLength, int intervalSpan) { - - // Details of KafkaProducerConfig could be find from: - // http://kafka.apache.org/documentation.html#producerconfigs - Properties props = new Properties(); - props.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList); - props.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, - "org.apache.kafka.common.serialization.ByteArraySerializer"); - props.setProperty(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, - "org.apache.kafka.common.serialization.ByteArraySerializer"); - props.setProperty(ProducerConfig.ACKS_CONFIG, "1"); - props.getProperty(ProducerConfig.CLIENT_ID_CONFIG, "DataGenerator"); - this.kafkaProducer = new KafkaProducer(props); - - this.cachedData = CachedData.getInstance(seedFile, fileOffset, dfsMaster); - this.recordLength = recordLength; - this.intervalSpan = intervalSpan; - } - - // The callback function will be triggered when receive ack from kafka. - // Print error message if exception exist. - Callback callback = new Callback() { - public void onCompletion(RecordMetadata metadata, Exception e) { - if (e != null) - e.printStackTrace(); - } - }; - - // send content to Kafka - public long send (String topic, long targetRecords, boolean debugMode) { - - long sentRecords = 0L; - long sentBytes = 0L; - - while (sentRecords < targetRecords) { - String line = cachedData.getRecord(); - String currentTime = Long.toString(System.currentTimeMillis()); - - // Key and Value will be serialized twice. - // 1. in producer.send method - // 2. explicitly serialize here to count byte size. - byte[] keyByte = serializer.serialize(topic, currentTime); - byte[] valueByte = fillArray(keyByte, serializer.serialize(topic, line)); - - ProducerRecord serializedRecord = new ProducerRecord(topic, keyByte, valueByte); - kafkaProducer.send(serializedRecord, callback); - - //update counter - sentRecords++; - sentBytes = sentBytes + keyByte.length + valueByte.length; - } - - return sentRecords; - } - - // Get byte array with fixed length (value length + key length = recordLength) - private byte[] fillArray(byte[] key, byte[] line) { - - int valueLength = recordLength - key.length; - byte[] valueByte; - if (valueLength > 0) { - valueByte = new byte[valueLength]; - if (line.length < valueLength) { - // There is no enough content in line, fill rest space with 0 - System.arraycopy(line, 0, valueByte, 0, line.length); - Arrays.fill(valueByte, line.length, valueLength, (byte)0); - } else { - System.arraycopy(line, 0, valueByte, 0, valueLength); - } - } else { - // recordLength is smaller than the length of key, return empty array. - valueByte = new byte[0]; - } - return valueByte; - } - - // close kafka producer - public void close() { - kafkaProducer.close(); - } -} diff --git a/src/streambench/datagen/src/main/java/com/intel/hibench/streambench/util/RecordSendTask.java b/src/streambench/datagen/src/main/java/com/intel/hibench/streambench/util/RecordSendTask.java deleted file mode 100644 index a6ef6ecbd..000000000 --- a/src/streambench/datagen/src/main/java/com/intel/hibench/streambench/util/RecordSendTask.java +++ /dev/null @@ -1,95 +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.intel.hibench.streambench.util; - -import java.util.Timer; -import java.util.TimerTask; -import java.util.concurrent.atomic.AtomicLong; - -public class RecordSendTask extends TimerTask { - KafkaSender sender; - private String topic; - private boolean debugMode; - - private long recordsPerInterval; // Define how many records will be sent on each round - private int totalRounds; // Total times this task plan to run. -1 means run infinity - private long totalRecords; // Total records this task plan to sent. -1 means no limit. - - // Count how many rounds has run pre producer - private int roundCounter = 0; - - // Count how many records has sent globally - private static AtomicLong recordsCounter = new AtomicLong(0L); - private Timer timer; // The timer which executor this task - - private Boolean stopped = false; - - // Constructors - public RecordSendTask(KafkaSender sender, String topic, - long recordsPerInterval, int totalRounds, long totalRecords, boolean debugMode, Timer timer) { - - this.sender = sender; - this.topic = topic; - this.recordsPerInterval = recordsPerInterval; - this.totalRounds = totalRounds; - this.totalRecords = totalRecords; - this.debugMode = debugMode; - this.timer = timer; - System.out.println(Thread.currentThread().getName() + " - starting generate data ... "); - } - - @Override - public void run() { - synchronized(this) { - if (debugMode) { - String threadName = Thread.currentThread().getName(); - System.out.println( threadName + " - RecordSendTask run, " + - roundCounter + " round, " + recordsCounter + " records sent"); - } - - if (isRecordValid() && isRoundValid()) { - // Send records to Kafka - long sentRecords = sender.send(topic, recordsPerInterval, debugMode); - - // Update counter - roundCounter++; - recordsCounter.getAndAdd(sentRecords); - } else { - if(!stopped) { - sender.close(); - timer.cancel(); - timer.purge(); - stopped = true; - // exit timer thread - System.out.println("DataGenerator stop, " + - roundCounter + " round, " + recordsCounter + " records sent"); - } - } - } - } - - // Check round times, if it's bigger than total rounds, terminate data generator - private boolean isRoundValid() { - return (-1 == totalRounds) || (roundCounter < totalRounds); - } - - // Check sent record number, if it's bigger than total records, terminate data generator - private boolean isRecordValid() { - return (-1 == totalRecords) || (recordsCounter.get() < totalRecords); - } -} diff --git a/src/streambench/datagen/src/main/java/com/intel/hibench/streambench/util/SourceFileReader.java b/src/streambench/datagen/src/main/java/com/intel/hibench/streambench/util/SourceFileReader.java deleted file mode 100644 index b51ee24ff..000000000 --- a/src/streambench/datagen/src/main/java/com/intel/hibench/streambench/util/SourceFileReader.java +++ /dev/null @@ -1,88 +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.intel.hibench.streambench.util; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.*; - -import java.io.*; -import java.util.Vector; - -public class SourceFileReader { - - // get file reader according to given path and offset. - static public BufferedReader getReader (Configuration dfsConf, String path, long offset) { - BufferedReader reader = null; - try { - Path pt = new Path(path); - FileSystem fs = FileSystem.get(dfsConf); - InputStreamReader isr; - if (fs.isDirectory(pt)) { - //give path is an directory - isr = new InputStreamReader(openMultipleParts(fs, pt, offset)); - } else { - //give path is an file - FSDataInputStream inputStream = fs.open(pt); - if (offset > 0) { - inputStream.seek(offset); - } - isr = new InputStreamReader(inputStream); - } - - reader = new BufferedReader(isr); - } catch (IOException e) { - System.err.println("Fail to get reader from path: " + path); - e.printStackTrace(); - } - return reader; - } - - // Open all files start with "part-", those files are generated by genSeedDataset.sh - static private InputStream openMultipleParts ( - FileSystem fs, Path pt, long offset) throws IOException { - - System.out.println("opening all parts in path: " + pt + ", from offset: " + offset ); - // list all files in given path - RemoteIterator rit = fs.listFiles(pt, false); - Vector fileHandleList = new Vector(); - while (rit.hasNext()) { - Path path = rit.next().getPath(); - - // Only read those files start with "part-" - if (path.getName().startsWith("part-")) { - long fileSize = fs.getFileStatus(path).getLen(); - if (offset < fileSize) { - FSDataInputStream inputStream = fs.open(path); - if (offset > 0) { - inputStream.seek(offset); - } - fileHandleList.add(inputStream); - } - offset -= fileSize; - } - } - - if (!fileHandleList.isEmpty()) { - return new SequenceInputStream(fileHandleList.elements()); - } else { - System.err.println("Error, no source file loaded. run genSeedDataset.sh first!"); - return null; - } - - } -} diff --git a/src/streambench/common/src/main/java/com/intel/hibench/streambench/common/ConfigLoader.java b/src/streambench/datagen/src/main/java/com/intel/hibench/streambench/utils/ConfigLoader.java similarity index 98% rename from src/streambench/common/src/main/java/com/intel/hibench/streambench/common/ConfigLoader.java rename to src/streambench/datagen/src/main/java/com/intel/hibench/streambench/utils/ConfigLoader.java index 7f7c2188f..bb4ad16a5 100644 --- a/src/streambench/common/src/main/java/com/intel/hibench/streambench/common/ConfigLoader.java +++ b/src/streambench/datagen/src/main/java/com/intel/hibench/streambench/utils/ConfigLoader.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package com.intel.hibench.streambench.common; +package com.intel.hibench.streambench.utils; import java.io.BufferedReader; import java.io.FileNotFoundException; diff --git a/src/streambench/flinkbench/pom.xml b/src/streambench/flinkbench/pom.xml deleted file mode 100644 index c2059b297..000000000 --- a/src/streambench/flinkbench/pom.xml +++ /dev/null @@ -1,106 +0,0 @@ - - - - - 4.0.0 - - streambench - com.intel.hibench - 5.0-SNAPSHOT - ../pom.xml - - - - com.intel.hibench.streambench - streaming-bench-flink - jar - Streaming Benchmark Flink - - - 1.0.3 - 2.4.2 - - - - - apache.snapshots - Apache Development Snapshot Repository - https://repository.apache.org/content/repositories/snapshots/ - - false - - - true - - - - - - - org.apache.flink - flink-core - ${flinkVersion} - compile - - - - org.apache.flink - flink-connector-kafka-0.8_2.11 - ${flinkVersion} - compile - - - - org.apache.flink - flink-streaming-java_2.11 - ${flinkVersion} - compile - - - - com.intel.hibench.streambench - streaming-bench-common - ${project.version} - - - - - - - maven-assembly-plugin - ${maven-assembly-plugin.version} - - - jar-with-dependencies - - - - - make-assembly - package - - single - - - - - - - \ No newline at end of file diff --git a/src/streambench/flinkbench/src/main/java/com/intel/flinkbench/RunBench.java b/src/streambench/flinkbench/src/main/java/com/intel/flinkbench/RunBench.java deleted file mode 100644 index b4f95764e..000000000 --- a/src/streambench/flinkbench/src/main/java/com/intel/flinkbench/RunBench.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.intel.flinkbench; - -import com.intel.flinkbench.microbench.*; -import com.intel.flinkbench.util.BenchLogUtil; -import com.intel.hibench.streambench.common.ConfigLoader; -import com.intel.flinkbench.util.FlinkBenchConfig; -import com.intel.hibench.streambench.common.metrics.KafkaReporter; -import com.intel.hibench.streambench.common.metrics.MetricsUtil; -import com.intel.hibench.streambench.common.StreamBenchConfig; - -import com.intel.hibench.streambench.common.Platform; - -public class RunBench { - public static void main(String[] args) throws Exception { - runAll(args); - } - - public static void runAll(String[] args) throws Exception { - - if (args.length < 1) - BenchLogUtil.handleError("Usage: RunBench "); - - ConfigLoader cl = new ConfigLoader(args[0]); - - // Prepare configuration - FlinkBenchConfig conf = new FlinkBenchConfig(); - conf.brokerList = cl.getProperty(StreamBenchConfig.KAFKA_BROKER_LIST); - conf.zkHost = cl.getProperty(StreamBenchConfig.ZK_HOST); - conf.testCase = cl.getProperty(StreamBenchConfig.TESTCASE); - conf.topic = cl.getProperty(StreamBenchConfig.KAFKA_TOPIC); - conf.consumerGroup = cl.getProperty(StreamBenchConfig.CONSUMER_GROUP); - conf.bufferTimeout = Long.parseLong(cl.getProperty(StreamBenchConfig.FLINK_BUFFERTIMEOUT)); - conf.offsetReset = cl.getProperty(StreamBenchConfig.KAFKA_OFFSET_RESET); - conf.windowDuration = cl.getProperty(StreamBenchConfig.FixWINDOW_DURATION); - conf.windowSlideStep = cl.getProperty(StreamBenchConfig.FixWINDOW_SLIDESTEP); - - conf.checkpointDuration = Long.parseLong(cl.getProperty(StreamBenchConfig.FLINK_CHECKPOINTDURATION)); - int producerNum = Integer.parseInt(cl.getProperty(StreamBenchConfig.DATAGEN_PRODUCER_NUMBER)); - long recordsPerInterval = Long.parseLong(cl.getProperty(StreamBenchConfig.DATAGEN_RECORDS_PRE_INTERVAL)); - int intervalSpan = Integer.parseInt(cl.getProperty(StreamBenchConfig.DATAGEN_INTERVAL_SPAN)); - conf.reportTopic = MetricsUtil.getTopic(Platform.FLINK, conf.testCase, producerNum, recordsPerInterval, intervalSpan); - int reportTopicPartitions = Integer.parseInt(cl.getProperty(StreamBenchConfig.KAFKA_TOPIC_PARTITIONS)); - MetricsUtil.createTopic(conf.zkHost, conf.reportTopic, reportTopicPartitions); - - // Main testcase logic - String testCase = conf.testCase; - - if (testCase.equals("wordcount")) { - WordCount wordCount = new WordCount(); - wordCount.processStream(conf); - } else if (testCase.equals("identity")) { - Identity identity = new Identity(); - identity.processStream(conf); - } else if (testCase.equals("repartition")) { - Repartition repartition = new Repartition(); - repartition.processStream(conf); - } else if (testCase.equals("fixwindow")) { - FixedWindow window = new FixedWindow(); - window.processStream(conf); - } - } -} diff --git a/src/streambench/flinkbench/src/main/java/com/intel/flinkbench/datasource/StreamBase.java b/src/streambench/flinkbench/src/main/java/com/intel/flinkbench/datasource/StreamBase.java deleted file mode 100644 index b7d73d8ba..000000000 --- a/src/streambench/flinkbench/src/main/java/com/intel/flinkbench/datasource/StreamBase.java +++ /dev/null @@ -1,52 +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.intel.flinkbench.datasource; - -import com.intel.flinkbench.util.KeyedTupleSchema; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.streaming.api.functions.source.SourceFunction; -import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer08; - -import com.intel.flinkbench.util.FlinkBenchConfig; - -import java.util.Properties; - -public abstract class StreamBase { - - private SourceFunction> dataStream; - - public SourceFunction> getDataStream() { - return this.dataStream; - } - - public void createDataStream(FlinkBenchConfig config) throws Exception{ - - Properties properties = new Properties(); - properties.setProperty("zookeeper.connect", config.zkHost); - properties.setProperty("group.id", config.consumerGroup); - properties.setProperty("bootstrap.servers", config.brokerList); - properties.setProperty("auto.offset.reset", config.offsetReset); - - this.dataStream = new FlinkKafkaConsumer08>( - config.topic, - new KeyedTupleSchema(), - properties); - } - - public void processStream(FlinkBenchConfig config) throws Exception {} -} diff --git a/src/streambench/flinkbench/src/main/java/com/intel/flinkbench/metrics/Metrics.java b/src/streambench/flinkbench/src/main/java/com/intel/flinkbench/metrics/Metrics.java deleted file mode 100644 index d57de954d..000000000 --- a/src/streambench/flinkbench/src/main/java/com/intel/flinkbench/metrics/Metrics.java +++ /dev/null @@ -1,22 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.intel.flinkbench.metrics; - -public class Metrics { - //TODO get the timestamp of original data and calculate the latency -} diff --git a/src/streambench/flinkbench/src/main/java/com/intel/flinkbench/microbench/FixedWindow.java b/src/streambench/flinkbench/src/main/java/com/intel/flinkbench/microbench/FixedWindow.java deleted file mode 100644 index a8dd06e29..000000000 --- a/src/streambench/flinkbench/src/main/java/com/intel/flinkbench/microbench/FixedWindow.java +++ /dev/null @@ -1,58 +0,0 @@ -package com.intel.flinkbench.microbench; - -import com.intel.flinkbench.datasource.StreamBase; -import com.intel.flinkbench.util.FlinkBenchConfig; -import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.api.common.functions.ReduceFunction; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import com.intel.hibench.streambench.common.metrics.KafkaReporter; -import com.intel.hibench.streambench.common.UserVisitParser; -import org.apache.flink.streaming.api.windowing.time.Time; - -public class FixedWindow extends StreamBase{ - - @Override - public void processStream(final FlinkBenchConfig config) throws Exception{ - - final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setBufferTimeout(config.bufferTimeout); - env.enableCheckpointing(config.checkpointDuration); - - createDataStream(config); - DataStream> dataStream = env.addSource(getDataStream()); - long windowDuration = Long.parseLong(config.windowDuration); - long windowSlideStep = Long.parseLong(config.windowSlideStep); - - dataStream.map(new MapFunction, Tuple2>>() { - - @Override - public Tuple2> map(Tuple2 value) throws Exception { - - String ip = UserVisitParser.parse(value.f1).getIp(); - return new Tuple2>(ip, new Tuple2(Long.parseLong(value.f0), 1)); - } - }) - .keyBy(0) - .timeWindow(Time.milliseconds(windowDuration), Time.milliseconds(windowSlideStep)) - .reduce(new ReduceFunction>>() { - @Override - public Tuple2> reduce(Tuple2> v1, Tuple2> v2) throws Exception { - return new Tuple2>(v1.f0, new Tuple2(Math.min(v1.f1.f0, v2.f1.f0), v1.f1.f1 + v2.f1.f1)); - } - }).map(new MapFunction>, String>() { - - @Override - public String map(Tuple2> value) throws Exception { - KafkaReporter kafkaReporter = new KafkaReporter(config.reportTopic, config.brokerList); - for (int i = 0; i< value.f1.f1; i++) { - kafkaReporter.report(value.f1.f0, System.currentTimeMillis()); - } - return value.f0; - } - }); - - env.execute("Fixed Window Job"); - } -} diff --git a/src/streambench/flinkbench/src/main/java/com/intel/flinkbench/microbench/Identity.java b/src/streambench/flinkbench/src/main/java/com/intel/flinkbench/microbench/Identity.java deleted file mode 100644 index 6a131cff7..000000000 --- a/src/streambench/flinkbench/src/main/java/com/intel/flinkbench/microbench/Identity.java +++ /dev/null @@ -1,52 +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.intel.flinkbench.microbench; - -import com.intel.flinkbench.datasource.StreamBase; -import com.intel.flinkbench.util.FlinkBenchConfig; -import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import com.intel.hibench.streambench.common.metrics.KafkaReporter; - -public class Identity extends StreamBase { - - @Override - public void processStream(final FlinkBenchConfig config) throws Exception{ - - final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setBufferTimeout(config.bufferTimeout); - - createDataStream(config); - DataStream> dataStream = env.addSource(getDataStream()); - - dataStream.map(new MapFunction, Tuple2>() { - - @Override - public Tuple2 map(Tuple2 value) throws Exception { - KafkaReporter kafkaReporter = new KafkaReporter(config.reportTopic, config.brokerList); - - kafkaReporter.report(Long.parseLong(value.f0), System.currentTimeMillis()); - return value; - } - }); - - env.execute("Identity Job"); - } -} diff --git a/src/streambench/flinkbench/src/main/java/com/intel/flinkbench/microbench/Repartition.java b/src/streambench/flinkbench/src/main/java/com/intel/flinkbench/microbench/Repartition.java deleted file mode 100644 index 69dd8edc1..000000000 --- a/src/streambench/flinkbench/src/main/java/com/intel/flinkbench/microbench/Repartition.java +++ /dev/null @@ -1,54 +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.intel.flinkbench.microbench; - -import com.intel.flinkbench.datasource.StreamBase; -import com.intel.flinkbench.util.FlinkBenchConfig; -import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import com.intel.hibench.streambench.common.metrics.KafkaReporter; - -public class Repartition extends StreamBase { - - @Override - public void processStream(final FlinkBenchConfig config) throws Exception{ - - final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setBufferTimeout(config.bufferTimeout); - - createDataStream(config); - DataStream> dataStream = env.addSource(getDataStream()); - - dataStream.rebalance().map( - new MapFunction, Tuple2>() { - - @Override - public Tuple2 map(Tuple2 value) throws Exception { - KafkaReporter kafkaReporter = new KafkaReporter(config.reportTopic, config.brokerList); - - kafkaReporter.report(Long.parseLong(value.f0), System.currentTimeMillis()); - return value; - } - }); - - - env.execute("Repartition Job"); - } -} diff --git a/src/streambench/flinkbench/src/main/java/com/intel/flinkbench/microbench/WordCount.java b/src/streambench/flinkbench/src/main/java/com/intel/flinkbench/microbench/WordCount.java deleted file mode 100644 index 7e1ad1283..000000000 --- a/src/streambench/flinkbench/src/main/java/com/intel/flinkbench/microbench/WordCount.java +++ /dev/null @@ -1,80 +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.intel.flinkbench.microbench; - -import com.intel.flinkbench.datasource.StreamBase; -import com.intel.flinkbench.util.FlinkBenchConfig; -import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.api.common.functions.RichMapFunction; -import org.apache.flink.api.common.state.ValueState; -import org.apache.flink.api.common.state.ValueStateDescriptor; -import org.apache.flink.api.common.typeinfo.TypeHint; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import com.intel.hibench.streambench.common.metrics.KafkaReporter; -import com.intel.hibench.streambench.common.UserVisitParser; - -public class WordCount extends StreamBase { - - @Override - public void processStream(final FlinkBenchConfig config) throws Exception { - final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setBufferTimeout(config.bufferTimeout); - env.enableCheckpointing(config.checkpointDuration); - createDataStream(config); - DataStream> dataStream = env.addSource(getDataStream()); - dataStream - .map(new MapFunction, Tuple2>>() { - @Override - public Tuple2> map(Tuple2 input) throws Exception { - String ip = UserVisitParser.parse(input.f1).getIp(); - //map record to > type - return new Tuple2>(ip, new Tuple2(input.f0, 1)); - } - }) - .keyBy(0) - .map(new RichMapFunction>, Tuple2>>() { - private transient ValueState sum; - - @Override - public Tuple2> map(Tuple2> input) throws Exception { - int currentSum = sum.value(); - currentSum += input.f1.f1; - sum.update(currentSum); - KafkaReporter kafkaReporter = new KafkaReporter(config.reportTopic, config.brokerList); - kafkaReporter.report(Long.parseLong(input.f1.f0), System.currentTimeMillis()); - return new Tuple2>(input.f0, new Tuple2(input.f1.f0, currentSum)); - } - - @Override - public void open(Configuration config) { - ValueStateDescriptor descriptor = - new ValueStateDescriptor( - "count", // the state name - TypeInformation.of(new TypeHint() { - }), // type information - 0); // default value of the state, if nothing was set - sum = getRuntimeContext().getState(descriptor); - } - }); - env.execute("Word Count Job"); - } -} diff --git a/src/streambench/flinkbench/src/main/java/com/intel/flinkbench/util/BenchLogUtil.java b/src/streambench/flinkbench/src/main/java/com/intel/flinkbench/util/BenchLogUtil.java deleted file mode 100644 index cc0369e32..000000000 --- a/src/streambench/flinkbench/src/main/java/com/intel/flinkbench/util/BenchLogUtil.java +++ /dev/null @@ -1,54 +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.intel.flinkbench.util; - -import java.io.File; -import java.io.PrintWriter; - -public class BenchLogUtil { - private static PrintWriter out; - - public static void init() throws Exception { - File file = new File("/tmp/benchlog-flink.txt"); - out = new PrintWriter(file); - } - - public static void logMsg(String msg) { - try { - if (out == null) { - init(); - } - } catch (Exception e) { - e.printStackTrace(); - } - out.println(msg); - out.flush(); - System.out.println(msg); - } - - public static void close() { - if (out != null) { - out.close(); - } - } - - public static void handleError(String msg) { - System.err.println(msg); - System.exit(1); - } -} diff --git a/src/streambench/flinkbench/src/main/java/com/intel/flinkbench/util/KeyedTupleSchema.java b/src/streambench/flinkbench/src/main/java/com/intel/flinkbench/util/KeyedTupleSchema.java deleted file mode 100644 index 2e61e2988..000000000 --- a/src/streambench/flinkbench/src/main/java/com/intel/flinkbench/util/KeyedTupleSchema.java +++ /dev/null @@ -1,39 +0,0 @@ -package com.intel.flinkbench.util; - -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.typeutils.TupleTypeInfo; -import org.apache.flink.api.java.typeutils.TypeExtractor; -import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; -import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema; - -import java.io.IOException; - -public class KeyedTupleSchema implements KeyedSerializationSchema>, KeyedDeserializationSchema> { - - @Override - public byte[] serializeKey(Tuple2 element) { - return element.f0.getBytes(); - } - - @Override - public byte[] serializeValue(Tuple2 element) { - return element.f1.getBytes(); - } - - @Override - public Tuple2 deserialize(byte[] messageKey, byte[] message, String topic, int partition, long offset) throws IOException { - return new Tuple2(new String(messageKey), new String(message)); - } - - @Override - public boolean isEndOfStream(Tuple2 nextElement) { - return false; - } - - @Override - public TypeInformation> getProducedType() { - return new TupleTypeInfo>(TypeExtractor.createTypeInfo(String.class), TypeExtractor.createTypeInfo(String.class)); - } -} - diff --git a/src/streambench/flinkbench/src/main/java/com/intel/flinkbench/util/StringTupleSchema.java b/src/streambench/flinkbench/src/main/java/com/intel/flinkbench/util/StringTupleSchema.java deleted file mode 100644 index b19e4208a..000000000 --- a/src/streambench/flinkbench/src/main/java/com/intel/flinkbench/util/StringTupleSchema.java +++ /dev/null @@ -1,60 +0,0 @@ -package com.intel.flinkbench.util; - -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.typeutils.TupleTypeInfo; -import org.apache.flink.api.java.typeutils.TypeExtractor; -import org.apache.flink.streaming.util.serialization.DeserializationSchema; -import org.apache.flink.streaming.util.serialization.SerializationSchema; - -/** - * Very simple serialization schema for strings of tuple. - */ -public class StringTupleSchema implements DeserializationSchema>, SerializationSchema> { - - private static final long serialVersionUID = 1L; - - @Override - public Tuple2 deserialize(byte[] message) { - byte[] str1 = new byte[message.length]; - byte[] str2 = new byte[message.length]; - int index = 0; - for (int i = 0; i < message.length; i++) { - - if (message[i] == ' ') { - index = i; - break; - } - str1[i] = message[i]; - } - for (int i = index + 1; i < message.length; i++) { - str2[i - index - 1] = message[i]; - } - return new Tuple2(new String(str1, 0, index), new String(str2, 0, message.length - index - 1)); - } - - @Override - public boolean isEndOfStream(Tuple2 nextElement) { - return false; - } - - @Override - public byte[] serialize(Tuple2 element) { - byte[] str1 = element.f0.getBytes(); - byte[] str2 = element.f1.getBytes(); - int len1 = str1.length; - int len2 = str2.length; - byte[] result = new byte[len1 + len2 + 1]; - System.arraycopy(str1, 0, result, 0, len1); - result[len1] = ' '; - for (int i = len1 + 1; i <= len1 + len2; i++) { - result[i] = str2[i - len1 - 1]; - } - return result; - } - - @Override - public TypeInformation> getProducedType() { - return new TupleTypeInfo>(TypeExtractor.createTypeInfo(String.class), TypeExtractor.createTypeInfo(String.class)); - } -} diff --git a/src/streambench/gearpumpbench/pom.xml b/src/streambench/gearpumpbench/pom.xml deleted file mode 100644 index d5cf7730e..000000000 --- a/src/streambench/gearpumpbench/pom.xml +++ /dev/null @@ -1,143 +0,0 @@ - - - - - 4.0.0 - - com.intel.hibench - streambench - 5.0-SNAPSHOT - ../pom.xml - - - com.intel.hibench.streambench - streaming-bench-gearpump - jar - Streaming Benchmark Gearpump - - 0.8.1 - 2.11.8 - 2.11 - 7.1.0 - - - - - apache.snapshots - Apache Development Snapshot Repository - https://repository.apache.org/content/repositories/snapshots/ - - false - - - true - - - - gearpump-shaded-repo - Vincent at Bintray - http://dl.bintray.com/fvunicorn/maven - - - - - - org.apache.gearpump - gearpump-core_2.11 - ${gearpumpVersion} - provided - - - org.apache.gearpump - gearpump-streaming_2.11 - ${gearpumpVersion} - provided - - - org.apache.gearpump - gearpump-external-kafka_2.11 - ${gearpumpVersion} - - - org.eclipse.collections - eclipse-collections-api - ${eclipseCollectionsVersion} - - - org.eclipse.collections - eclipse-collections - ${eclipseCollectionsVersion} - - - com.intel.hibench.streambench - streaming-bench-common - ${project.version} - - - - - target/scala-${scala.binary.version}/classes - target/scala-${scala.binary.version}/test-classes - - - net.alchim31.maven - scala-maven-plugin - - ${scala.binary.version} - ${scala.version} - - - - scala-compile-first - process-resources - - add-source - compile - - - - scala-test-compile - process-test-resources - - testCompile - - - - - - - maven-assembly-plugin - ${maven-assembly-plugin.version} - - - jar-with-dependencies - - - - - make-assembly - package - - single - - - - - - - diff --git a/src/streambench/gearpumpbench/src/main/scala/com/intel/hibench/streambench/gearpump/RunBench.scala b/src/streambench/gearpumpbench/src/main/scala/com/intel/hibench/streambench/gearpump/RunBench.scala deleted file mode 100644 index 76ae42be5..000000000 --- a/src/streambench/gearpumpbench/src/main/scala/com/intel/hibench/streambench/gearpump/RunBench.scala +++ /dev/null @@ -1,77 +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.intel.hibench.streambench.gearpump - -import com.intel.hibench.streambench.common.metrics.MetricsUtil -import com.intel.hibench.streambench.common.{ConfigLoader, Platform, StreamBenchConfig, TestCase} -import com.intel.hibench.streambench.gearpump.application._ -import com.intel.hibench.streambench.gearpump.source.KafkaSourceProvider -import com.intel.hibench.streambench.gearpump.util.GearpumpConfig -import org.apache.gearpump.cluster.UserConfig -import org.apache.gearpump.cluster.client.ClientContext - -object RunBench { - def main(args: Array[String]) { - this.run(args) - } - - def run(args: Array[String]) { - val context = ClientContext() - implicit val system = context.system - implicit val sourceProvider = new KafkaSourceProvider() - val confLoader = new ConfigLoader(args(0)) - val gearConf = getConfig(confLoader) - - val benchmark = TestCase.withValue(gearConf.benchName) match { - case TestCase.WORDCOUNT => new WordCount(gearConf) - case TestCase.IDENTITY => new IdentityApp(gearConf) - case TestCase.FIXWINDOW => new WindowCount(gearConf) - } - - val benchConf = UserConfig.empty - .withValue(GearpumpConfig.BENCH_CONFIG, gearConf) - context.submit(benchmark.application(benchConf)) - context.close() - } - - private def getConfig(conf: ConfigLoader): GearpumpConfig = { - val benchName = conf.getProperty(StreamBenchConfig.TESTCASE) - val topic = conf.getProperty(StreamBenchConfig.KAFKA_TOPIC) - val zkHost = conf.getProperty(StreamBenchConfig.ZK_HOST) - val consumerGroup = conf.getProperty(StreamBenchConfig.CONSUMER_GROUP) - val parallelism = conf.getProperty(StreamBenchConfig.GEARPUMP_PARALLELISM).toInt - val brokerList = conf.getProperty(StreamBenchConfig.KAFKA_BROKER_LIST) - val prob = conf.getProperty(StreamBenchConfig.SAMPLE_PROBABILITY).toDouble - val reporterTopic = getReporterTopic(conf) - val reporterTopicPartitions = conf.getProperty(StreamBenchConfig.KAFKA_TOPIC_PARTITIONS).toInt - MetricsUtil.createTopic(zkHost, reporterTopic, reporterTopicPartitions) - - val windowDuration = conf.getProperty(StreamBenchConfig.FixWINDOW_DURATION).toLong - val windowStep = conf.getProperty(StreamBenchConfig.FixWINDOW_SLIDESTEP).toLong - - GearpumpConfig(benchName, zkHost, brokerList, consumerGroup, topic, - parallelism, prob, reporterTopic, windowDuration, windowStep) - } - - private def getReporterTopic(conf: ConfigLoader): String = { - val topic = conf.getProperty(StreamBenchConfig.KAFKA_TOPIC) - val producerNum: Int = conf.getProperty(StreamBenchConfig.DATAGEN_PRODUCER_NUMBER).toInt - val recordPerInterval = conf.getProperty(StreamBenchConfig.DATAGEN_RECORDS_PRE_INTERVAL).toLong - val intervalSpan: Int = conf.getProperty(StreamBenchConfig.DATAGEN_INTERVAL_SPAN).toInt - MetricsUtil.getTopic(Platform.GEARPUMP, topic, producerNum, recordPerInterval, intervalSpan) - } -} diff --git a/src/streambench/gearpumpbench/src/main/scala/com/intel/hibench/streambench/gearpump/application/BasicApplication.scala b/src/streambench/gearpumpbench/src/main/scala/com/intel/hibench/streambench/gearpump/application/BasicApplication.scala deleted file mode 100644 index d09153bad..000000000 --- a/src/streambench/gearpumpbench/src/main/scala/com/intel/hibench/streambench/gearpump/application/BasicApplication.scala +++ /dev/null @@ -1,35 +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.intel.hibench.streambench.gearpump.application - -import com.intel.hibench.streambench.common.TestCase -import com.intel.hibench.streambench.common.metrics.{KafkaReporter, LatencyReporter} -import com.intel.hibench.streambench.gearpump.source.SourceProvider -import com.intel.hibench.streambench.gearpump.util.GearpumpConfig -import org.apache.gearpump.cluster.UserConfig -import org.apache.gearpump.streaming.task.Task -import org.apache.gearpump.streaming.{Processor, StreamApplication} - -abstract class BasicApplication(conf: GearpumpConfig)(implicit sourceProvider: SourceProvider){ - val benchName: TestCase - - def getSource(): Processor[_ <: Task] = { - sourceProvider.getSourceProcessor(conf) - } - - def application(benchConfig: UserConfig): StreamApplication -} diff --git a/src/streambench/gearpumpbench/src/main/scala/com/intel/hibench/streambench/gearpump/application/IdentityApp.scala b/src/streambench/gearpumpbench/src/main/scala/com/intel/hibench/streambench/gearpump/application/IdentityApp.scala deleted file mode 100644 index 459dbfd59..000000000 --- a/src/streambench/gearpumpbench/src/main/scala/com/intel/hibench/streambench/gearpump/application/IdentityApp.scala +++ /dev/null @@ -1,39 +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.intel.hibench.streambench.gearpump.application - -import com.intel.hibench.streambench.common.TestCase -import com.intel.hibench.streambench.gearpump.source.SourceProvider -import com.intel.hibench.streambench.gearpump.task.Identity -import com.intel.hibench.streambench.gearpump.util.GearpumpConfig -import org.apache.gearpump.cluster.UserConfig -import org.apache.gearpump.partitioner.ShufflePartitioner -import org.apache.gearpump.streaming.{Processor, StreamApplication} -import org.apache.gearpump.util.Graph -import org.apache.gearpump.util.Graph._ - -class IdentityApp(conf: GearpumpConfig)(implicit sourceProvider: SourceProvider) extends BasicApplication(conf) { - override val benchName = TestCase.IDENTITY - - override def application(benchConfig: UserConfig): StreamApplication = { - val source = getSource() - val partitioner = new ShufflePartitioner - val identity = Processor[Identity](conf.parallelism) - StreamApplication("identity", Graph(source ~ partitioner ~> identity), benchConfig) - } -} - diff --git a/src/streambench/gearpumpbench/src/main/scala/com/intel/hibench/streambench/gearpump/application/WindowCount.scala b/src/streambench/gearpumpbench/src/main/scala/com/intel/hibench/streambench/gearpump/application/WindowCount.scala deleted file mode 100644 index 630c826b3..000000000 --- a/src/streambench/gearpumpbench/src/main/scala/com/intel/hibench/streambench/gearpump/application/WindowCount.scala +++ /dev/null @@ -1,39 +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.intel.hibench.streambench.gearpump.application - -import com.intel.hibench.streambench.common.TestCase -import com.intel.hibench.streambench.gearpump.source.SourceProvider -import com.intel.hibench.streambench.gearpump.task.{Parser, SlidingWindow} -import com.intel.hibench.streambench.gearpump.util.GearpumpConfig -import org.apache.gearpump.cluster.UserConfig -import org.apache.gearpump.partitioner.ShufflePartitioner -import org.apache.gearpump.streaming.{Processor, StreamApplication} -import org.apache.gearpump.util.Graph -import org.apache.gearpump.util.Graph._ - -class WindowCount(conf: GearpumpConfig)(implicit sourceProvider: SourceProvider) extends BasicApplication(conf) { - override val benchName = TestCase.FIXWINDOW - - override def application(benchConfig: UserConfig): StreamApplication = { - val source = getSource() - val partitioner = new ShufflePartitioner - val parser = Processor[Parser](conf.parallelism) - val slidingWindow = Processor[SlidingWindow](conf.parallelism) - StreamApplication("windowCount", Graph(source ~ partitioner ~> parser ~> slidingWindow), benchConfig) - } -} diff --git a/src/streambench/gearpumpbench/src/main/scala/com/intel/hibench/streambench/gearpump/application/WordCount.scala b/src/streambench/gearpumpbench/src/main/scala/com/intel/hibench/streambench/gearpump/application/WordCount.scala deleted file mode 100644 index 7b731bba6..000000000 --- a/src/streambench/gearpumpbench/src/main/scala/com/intel/hibench/streambench/gearpump/application/WordCount.scala +++ /dev/null @@ -1,40 +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.intel.hibench.streambench.gearpump.application - -import com.intel.hibench.streambench.common.TestCase -import com.intel.hibench.streambench.gearpump.source.SourceProvider -import com.intel.hibench.streambench.gearpump.task.{Parser, Sum} -import com.intel.hibench.streambench.gearpump.util.GearpumpConfig -import org.apache.gearpump.cluster.UserConfig -import org.apache.gearpump.partitioner.ShufflePartitioner -import org.apache.gearpump.streaming.{Processor, StreamApplication} -import org.apache.gearpump.util.Graph -import org.apache.gearpump.util.Graph._ - -class WordCount(conf: GearpumpConfig)(implicit sourceProvider: SourceProvider) extends BasicApplication(conf) { - override val benchName = TestCase.WORDCOUNT - - override def application(benchConfig: UserConfig): StreamApplication = { - val source = getSource() - val partitioner = new ShufflePartitioner - val parser = Processor[Parser](conf.parallelism) - val sum = Processor[Sum](conf.parallelism) - StreamApplication("wordCount", Graph(source ~ partitioner ~> parser ~> sum), benchConfig) - } -} diff --git a/src/streambench/gearpumpbench/src/main/scala/com/intel/hibench/streambench/gearpump/source/InMemorySourceProvider.scala b/src/streambench/gearpumpbench/src/main/scala/com/intel/hibench/streambench/gearpump/source/InMemorySourceProvider.scala deleted file mode 100644 index d4de5a14f..000000000 --- a/src/streambench/gearpumpbench/src/main/scala/com/intel/hibench/streambench/gearpump/source/InMemorySourceProvider.scala +++ /dev/null @@ -1,72 +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.intel.hibench.streambench.gearpump.source - -import com.intel.hibench.streambench.gearpump.source.InMemorySourceProvider.InMemorySourceTask -import com.intel.hibench.streambench.gearpump.util.GearpumpConfig -import org.apache.gearpump.streaming.Processor -import org.apache.gearpump.cluster.UserConfig -import org.apache.gearpump.streaming.task.Task -import org.apache.gearpump.streaming.task.TaskContext -import org.apache.gearpump.Message -import org.apache.gearpump.streaming.task.StartTime - -class InMemorySourceProvider extends SourceProvider{ - override def getSourceProcessor(conf: GearpumpConfig): Processor[_ <: Task] = { - Processor[InMemorySourceTask](conf.parallelism) - } -} - -object InMemorySourceProvider { - class InMemorySourceTask(taskContext: TaskContext, conf: UserConfig) extends Task(taskContext, conf) { - val TEXT_TO_SPLIT = - """ - | Licensed to the Apache Software Foundation (ASF) under one - | or more contributor license agreements. See the NOTICE file - | distributed with this work for additional information - | regarding copyright ownership. The ASF licenses this file - | to you under the Apache License, Version 2.0 (the - | "License"); you may not use this file except in compliance - | with the License. You may obtain a copy of the License at - | - | http://www.apache.org/licenses/LICENSE-2.0 - | - | Unless required by applicable law or agreed to in writing, software - | distributed under the License is distributed on an "AS IS" BASIS, - | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - | See the License for the specific language governing permissions and - | limitations under the License. - """.stripMargin - - override def onStart(startTime: StartTime): Unit = { - self ! Message("start") - } - - var times = 0 - val MAX = 1000 * 1000 - override def onNext(msg: Message): Unit = { - if (times < MAX) { - TEXT_TO_SPLIT.lines.foreach { line => - taskContext.output(Message(line, System.currentTimeMillis())) - } - times += 1 - self ! Message("continue") - } - - } - } -} diff --git a/src/streambench/gearpumpbench/src/main/scala/com/intel/hibench/streambench/gearpump/source/KafkaSourceProvider.scala b/src/streambench/gearpumpbench/src/main/scala/com/intel/hibench/streambench/gearpump/source/KafkaSourceProvider.scala deleted file mode 100644 index 2ec7d8af6..000000000 --- a/src/streambench/gearpumpbench/src/main/scala/com/intel/hibench/streambench/gearpump/source/KafkaSourceProvider.scala +++ /dev/null @@ -1,55 +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.intel.hibench.streambench.gearpump.source - -import java.util.Properties - -import akka.actor.ActorSystem -import com.intel.hibench.streambench.gearpump.util.GearpumpConfig -import com.twitter.bijection.Injection -import org.apache.gearpump.Message -import org.apache.gearpump.streaming.Processor -import org.apache.gearpump.streaming.kafka.KafkaSource -import org.apache.gearpump.streaming.kafka.util.KafkaConfig -import org.apache.gearpump.streaming.source.DataSourceProcessor -import org.apache.gearpump.streaming.task.Task -import org.apache.gearpump.streaming.transaction.api.MessageDecoder - -class KafkaSourceProvider(implicit actorSystem: ActorSystem) extends SourceProvider{ - override def getSourceProcessor(conf: GearpumpConfig): Processor[_ <: Task] = { - getKafkaSource(conf.zkHost, conf.brokerList, conf.topic, conf.parallelism) - } - - private def getKafkaSource(zkConnect: String, bootstrapServers: String, topic: String, parallelism: Int): Processor[_ <: Task] = { - val props = new Properties - props.put(KafkaConfig.ZOOKEEPER_CONNECT_CONFIG, zkConnect) - props.put(KafkaConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers) - props.put(KafkaConfig.FETCH_SLEEP_MS_CONFIG, new Integer(1)) - props.put(KafkaConfig.MESSAGE_DECODER_CLASS_CONFIG, classOf[KeyValueDecoder]) - props.put(KafkaConfig.CONSUMER_START_OFFSET_CONFIG, new java.lang.Long(-1)) - - val kafkaSource = new KafkaSource(topic, props) - DataSourceProcessor(kafkaSource, parallelism) - } -} - -class KeyValueDecoder extends MessageDecoder { - override def fromBytes(key: Array[Byte], value: Array[Byte]): Message = { - Message(Injection.invert[String, Array[Byte]](value).get, - Injection.invert[String, Array[Byte]](key).get.toLong) - } -} diff --git a/src/streambench/gearpumpbench/src/main/scala/com/intel/hibench/streambench/gearpump/source/SourceProvider.scala b/src/streambench/gearpumpbench/src/main/scala/com/intel/hibench/streambench/gearpump/source/SourceProvider.scala deleted file mode 100644 index 70f4c465e..000000000 --- a/src/streambench/gearpumpbench/src/main/scala/com/intel/hibench/streambench/gearpump/source/SourceProvider.scala +++ /dev/null @@ -1,25 +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.intel.hibench.streambench.gearpump.source - -import com.intel.hibench.streambench.gearpump.util.GearpumpConfig -import org.apache.gearpump.streaming.Processor -import org.apache.gearpump.streaming.task.Task - -trait SourceProvider { - def getSourceProcessor(conf: GearpumpConfig): Processor[_ <: Task] -} diff --git a/src/streambench/gearpumpbench/src/main/scala/com/intel/hibench/streambench/gearpump/task/Identity.scala b/src/streambench/gearpumpbench/src/main/scala/com/intel/hibench/streambench/gearpump/task/Identity.scala deleted file mode 100644 index 8a3de7033..000000000 --- a/src/streambench/gearpumpbench/src/main/scala/com/intel/hibench/streambench/gearpump/task/Identity.scala +++ /dev/null @@ -1,33 +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.intel.hibench.streambench.gearpump.task - -import com.intel.hibench.streambench.common.metrics.KafkaReporter -import com.intel.hibench.streambench.gearpump.util.GearpumpConfig -import org.apache.gearpump.Message -import org.apache.gearpump.cluster.UserConfig -import org.apache.gearpump.streaming.task.{Task, TaskContext} - -class Identity(taskContext: TaskContext, conf: UserConfig) extends Task(taskContext, conf) { - private val benchConfig = conf.getValue[GearpumpConfig](GearpumpConfig.BENCH_CONFIG).get - val reporter = new KafkaReporter(benchConfig.reporterTopic, benchConfig.brokerList) - - override def onNext(msg: Message): Unit = { - taskContext.output(msg) - reporter.report(msg.timestamp, System.currentTimeMillis()) - } -} diff --git a/src/streambench/gearpumpbench/src/main/scala/com/intel/hibench/streambench/gearpump/task/Parser.scala b/src/streambench/gearpumpbench/src/main/scala/com/intel/hibench/streambench/gearpump/task/Parser.scala deleted file mode 100644 index 613c19582..000000000 --- a/src/streambench/gearpumpbench/src/main/scala/com/intel/hibench/streambench/gearpump/task/Parser.scala +++ /dev/null @@ -1,31 +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.intel.hibench.streambench.gearpump.task - -import com.intel.hibench.streambench.common.UserVisitParser -import org.apache.gearpump.Message -import org.apache.gearpump.cluster.UserConfig -import org.apache.gearpump.streaming.task.{Task, TaskContext} - -class Parser(taskContext: TaskContext, conf: UserConfig) extends Task(taskContext, conf) { - import taskContext.output - - override def onNext(msg: Message): Unit = { - val userVisit = UserVisitParser.parse(msg.msg.asInstanceOf[String]) - output(new Message(userVisit.getIp, msg.timestamp)) - } -} diff --git a/src/streambench/gearpumpbench/src/main/scala/com/intel/hibench/streambench/gearpump/task/SlidingWindow.scala b/src/streambench/gearpumpbench/src/main/scala/com/intel/hibench/streambench/gearpump/task/SlidingWindow.scala deleted file mode 100644 index bfa1405a8..000000000 --- a/src/streambench/gearpumpbench/src/main/scala/com/intel/hibench/streambench/gearpump/task/SlidingWindow.scala +++ /dev/null @@ -1,83 +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.intel.hibench.streambench.gearpump.task - -import com.intel.hibench.streambench.common.metrics.KafkaReporter -import com.intel.hibench.streambench.gearpump.util.GearpumpConfig -import org.apache.gearpump.{Message, TimeStamp} -import org.apache.gearpump.cluster.UserConfig -import org.apache.gearpump.streaming.task.{Task, TaskContext} -import org.eclipse.collections.api.block.procedure.Procedure -import org.eclipse.collections.impl.map.mutable.UnifiedMap -import org.eclipse.collections.impl.map.sorted.mutable.TreeSortedMap - -import scala.collection.mutable.ArrayBuffer - -class SlidingWindow(taskContext: TaskContext, conf: UserConfig) extends Task(taskContext, conf) { - private val benchConfig = conf.getValue[GearpumpConfig](GearpumpConfig.BENCH_CONFIG).get - private val windowDuration = benchConfig.windowDuration - private val windowStep = benchConfig.windowSlideStep - val reporter = new KafkaReporter(benchConfig.reporterTopic, benchConfig.brokerList) - - // windowStartTime -> (ip -> (minMessageTime, count)) - private val windowCounts = new TreeSortedMap[Long, UnifiedMap[String, (TimeStamp, Long)]] - - override def onNext(message: Message): Unit = { - val ip = message.msg.asInstanceOf[String] - val msgTime = System.currentTimeMillis() - getWindows(msgTime).foreach { window => - val countsByIp = windowCounts.getOrDefault(window, new UnifiedMap[String, (TimeStamp, Long)]) - val (minTime, count) = countsByIp.getOrDefault(ip, (msgTime, 0L)) - countsByIp.put(ip, (Math.min(msgTime, minTime), count + 1L)) - windowCounts.put(window, countsByIp) - } - - var hasNext = true - while (hasNext && !windowCounts.isEmpty) { - val windowStart = windowCounts.firstKey() - if (msgTime >= (windowStart + windowDuration)) { - val countsByIp = windowCounts.remove(windowStart) - countsByIp.forEachValue(new Procedure[(TimeStamp, Long)]() { - override def value(tuple: (TimeStamp, Long)): Unit = { - (1 to tuple._2.toInt).foreach(i =>reporter.report(tuple._1, msgTime)) - } - }) - } else { - hasNext = false - } - } - } - - - private def getWindows(timestamp: TimeStamp): List[TimeStamp] = { - val windows = ArrayBuffer.empty[TimeStamp] - var start = lastStartFor(timestamp) - windows += start - start -= windowStep - while (start >= timestamp) { - windows += start - start -= windowStep - } - windows.toList - } - - private def lastStartFor(timestamp: TimeStamp): TimeStamp = { - timestamp - (timestamp + windowStep) % windowStep - } -} - diff --git a/src/streambench/gearpumpbench/src/main/scala/com/intel/hibench/streambench/gearpump/task/Sum.scala b/src/streambench/gearpumpbench/src/main/scala/com/intel/hibench/streambench/gearpump/task/Sum.scala deleted file mode 100644 index 2a4cdc521..000000000 --- a/src/streambench/gearpumpbench/src/main/scala/com/intel/hibench/streambench/gearpump/task/Sum.scala +++ /dev/null @@ -1,23 +0,0 @@ -package com.intel.hibench.streambench.gearpump.task - -import com.intel.hibench.streambench.common.metrics.KafkaReporter -import com.intel.hibench.streambench.gearpump.util.GearpumpConfig -import org.apache.gearpump.Message -import org.apache.gearpump.cluster.UserConfig -import org.apache.gearpump.streaming.task.{Task, TaskContext} - -import scala.collection.mutable - -class Sum(taskContext: TaskContext, conf: UserConfig) extends Task(taskContext, conf) { - private val benchConfig = conf.getValue[GearpumpConfig](GearpumpConfig.BENCH_CONFIG).get - val reporter = new KafkaReporter(benchConfig.reporterTopic, benchConfig.brokerList) - private val map: mutable.HashMap[String, Long] = new mutable.HashMap[String, Long]() - - override def onNext(msg: Message): Unit = { - if (null != msg) { - val current = map.getOrElse(msg.msg.asInstanceOf[String], 0L) - map.put(msg.msg.asInstanceOf[String], current + 1) - reporter.report(msg.timestamp, System.currentTimeMillis()) - } - } -} \ No newline at end of file diff --git a/src/streambench/gearpumpbench/src/main/scala/com/intel/hibench/streambench/gearpump/util/GearpumpConfig.scala b/src/streambench/gearpumpbench/src/main/scala/com/intel/hibench/streambench/gearpump/util/GearpumpConfig.scala deleted file mode 100644 index 157e88eda..000000000 --- a/src/streambench/gearpumpbench/src/main/scala/com/intel/hibench/streambench/gearpump/util/GearpumpConfig.scala +++ /dev/null @@ -1,23 +0,0 @@ -package com.intel.hibench.streambench.gearpump.util - -case class GearpumpConfig( - benchName: String, - zkHost: String, - brokerList: String, - consumerGroup: String, - topic: String, - parallelism: Int, - prob: Double, - reporterTopic: String, - windowDuration: Long, - windowSlideStep: Long, - pattern: String = " ", - fieldIndex: Int = 0, - separator: String = "\\s+" -) - -object GearpumpConfig { - - val BENCH_CONFIG = "gearpump.bench.config" - val BENCH_LATENCY_REPORTER = "gearpump.bench.latency.reporter" -} diff --git a/src/streambench/pom.xml b/src/streambench/pom.xml index 8fd3b3557..0225018cf 100644 --- a/src/streambench/pom.xml +++ b/src/streambench/pom.xml @@ -1,39 +1,35 @@ - 4.0.0 + 4.0.0 + + + com.intel.hibench + hibench + 5.0-SNAPSHOT + - com.intel.hibench - hibench + streambench 5.0-SNAPSHOT - - - com.intel.hibench - streambench - 5.0-SNAPSHOT - pom - Streamingbench - - - - github-releases - http://oss.sonatype.org/content/repositories/github-releases/ - - - clojars.org - http://clojars.org/repo - - + pom + Streamingbench - - common - datagen - sparkbench - stormbench - gearpumpbench - flinkbench - - + diff --git a/src/streambench/samzabench/pom.xml b/src/streambench/samzabench/pom.xml index 0093eae27..fba04fd81 100644 --- a/src/streambench/samzabench/pom.xml +++ b/src/streambench/samzabench/pom.xml @@ -1,24 +1,7 @@ - - - + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> 4.0.0 + com.intel.hibench streambench @@ -29,8 +12,9 @@ 3.0.0 - com.intel.hibench.streambench + com.intel.RPCcloud streaming-bench-samza + 0.1-SNAPSHOT jar Streaming Benchmark Samza @@ -118,13 +102,24 @@ commons-lang 2.6 - - com.intel.hibench.streambench - streaming-bench-common - ${project.version} - + + + my-local-repo + file://${user.home}/.m2/repository + + + apache-releases + https://repository.apache.org/content/groups/public + + + scala-tools.org + Scala-tools Maven2 Repository + https://oss.sonatype.org/content/groups/scala-tools + + + scala-tools.org @@ -134,10 +129,20 @@ + src/main + + org.apache.maven.plugins + maven-compiler-plugin + 3.1 + + 1.6 + 1.6 + + maven-assembly-plugin - ${maven-assembly-plugin.version} + 2.3 src/assembly/src.xml diff --git a/src/streambench/sparkbench/pom.xml b/src/streambench/sparkbench/pom.xml index 97c2d02fd..f0ef42ce1 100644 --- a/src/streambench/sparkbench/pom.xml +++ b/src/streambench/sparkbench/pom.xml @@ -16,8 +16,7 @@ ~ limitations under the License. --> - + 4.0.0 com.intel.hibench @@ -26,7 +25,7 @@ com.intel.hibench.streambench - streaming-bench-spark + streaming-bench-spark_0.1 jar Streaming Benchmark Spark @@ -171,11 +170,6 @@ - - com.intel.hibench.streambench - streaming-bench-common - ${project.version} - @@ -183,68 +177,68 @@ target/scala-${scala.binary.version}/test-classes - net.alchim31.maven - scala-maven-plugin - - ${scala.binary.version} - ${scala.version} - - - - scala-compile-first - process-resources - - add-source - compile - - - - scala-test-compile - process-test-resources - - testCompile - - - - - - org.codehaus.mojo - build-helper-maven-plugin - ${build-helper-maven-plugin.version} - - - add-source - generate-sources - - add-source - - - - src/multiapi/scala/spark${spark.bin.version}/ - - - - + net.alchim31.maven + scala-maven-plugin + + ${scala.binary.version} + ${scala.version} + + + + scala-compile-first + process-resources + + add-source + compile + + + + scala-test-compile + process-test-resources + + testCompile + + + + + org.codehaus.mojo + build-helper-maven-plugin + ${build-helper-maven-plugin.version} + + + add-source + generate-sources + + add-source + + + + src/multiapi/scala/spark${spark.bin.version}/ + + + + + maven-assembly-plugin - ${maven-assembly-plugin.version} - - ${project.build.finalName}-spark${spark.bin.version} - - jar-with-dependencies - - - - - make-assembly - package - - single - - - + ${maven-assembly-plugin.version} + + ${project.build.finalName}-spark${spark.bin.version} + + jar-with-dependencies + + + + + make-assembly + package + + single + + + diff --git a/src/streambench/sparkbench/src/main/java/com/intel/hibench/streambench/spark/ConfigLoader.java b/src/streambench/sparkbench/src/main/java/com/intel/hibench/streambench/spark/ConfigLoader.java new file mode 100644 index 000000000..3844164dd --- /dev/null +++ b/src/streambench/sparkbench/src/main/java/com/intel/hibench/streambench/spark/ConfigLoader.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.intel.hibench.streambench.spark; + +import java.io.BufferedReader; +import java.io.FileNotFoundException; +import java.io.FileReader; +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +public class ConfigLoader { + private String ConfigFileName = null; + private Map store; + + public ConfigLoader(String filename){ + ConfigFileName = filename; + store = new HashMap(); + // Load and parse config + try { + BufferedReader br = new BufferedReader(new FileReader(filename)); + String line = br.readLine(); + while(line != null){ + if ((line.length()>0) && (line.charAt(0)!='#')) { + String[] words = line.split("\\s+"); + if (words.length == 2) { + String key = words[0]; + String value = words[1]; + store.put(key, value); + } else if (words.length == 1) { + String key = words[0]; + store.put(key, ""); + } else { + if (!line.startsWith("hibench")) + System.out.println("Warning: unknown config parsed, skip:" + line); + } + } + line = br.readLine(); + } + } catch (FileNotFoundException e) { + System.out.println("ERROR: Config file not found! Should not happen. Caused by:"); + } catch (IOException e) { + System.out.println("ERROR: IO exception during read file. Should not happen. Caused by:"); + e.printStackTrace(); + } + } + + public String getProperty(String key){ + if (store.containsKey(key)) + return (String) store.get(key); + else { + System.out.println("ERROR: Unknown config key:" + key); + return null; + } + } +} diff --git a/src/streambench/sparkbench/src/main/scala/com/intel/hibench/streambench/spark/RunBench.scala b/src/streambench/sparkbench/src/main/scala/com/intel/hibench/streambench/spark/RunBench.scala index 00b46d6e8..158b19e7c 100644 --- a/src/streambench/sparkbench/src/main/scala/com/intel/hibench/streambench/spark/RunBench.scala +++ b/src/streambench/sparkbench/src/main/scala/com/intel/hibench/streambench/spark/RunBench.scala @@ -17,103 +17,75 @@ package com.intel.hibench.streambench.spark -import com.intel.hibench.common.HiBenchConfig -import com.intel.hibench.streambench.common.metrics.MetricsUtil -import com.intel.hibench.streambench.common._ -import com.intel.hibench.streambench.spark.util.SparkBenchConfig -import com.intel.hibench.streambench.spark.application._ -import kafka.serializer.StringDecoder -import org.apache.spark.SparkConf -import org.apache.spark.streaming.dstream.DStream -import org.apache.spark.streaming.kafka.KafkaUtils -import org.apache.spark.streaming.{Milliseconds, StreamingContext} +import com.intel.hibench.streambench.spark.entity.ParamEntity +import com.intel.hibench.streambench.spark.util._ +import com.intel.hibench.streambench.spark.microbench._ -/** - * The entry point of Spark Streaming benchmark - */ object RunBench { + var reportDir = "" def main(args: Array[String]) { - val conf = new ConfigLoader(args(0)) - - // Load configuration - val master = conf.getProperty(HiBenchConfig.SPARK_MASTER) - - val batchInterval = conf.getProperty(StreamBenchConfig.SPARK_BATCH_INTERVAL).toInt - val receiverNumber = conf.getProperty(StreamBenchConfig.SPARK_RECEIVER_NUMBER).toInt - val copies = conf.getProperty(StreamBenchConfig.SPARK_STORAGE_LEVEL).toInt - val enableWAL = conf.getProperty(StreamBenchConfig.SPARK_ENABLE_WAL).toBoolean - val checkPointPath = conf.getProperty(StreamBenchConfig.SPARK_CHECKPOINT_PATH) - val directMode = conf.getProperty(StreamBenchConfig.SPARK_USE_DIRECT_MODE).toBoolean - val benchName = conf.getProperty(StreamBenchConfig.TESTCASE) - val topic = conf.getProperty(StreamBenchConfig.KAFKA_TOPIC) - val zkHost = conf.getProperty(StreamBenchConfig.ZK_HOST) - val consumerGroup = conf.getProperty(StreamBenchConfig.CONSUMER_GROUP) - val brokerList = if (directMode) conf.getProperty(StreamBenchConfig.KAFKA_BROKER_LIST) else "" - val debugMode = conf.getProperty(StreamBenchConfig.DEBUG_MODE).toBoolean - val recordPerInterval = conf.getProperty(StreamBenchConfig.DATAGEN_RECORDS_PRE_INTERVAL).toLong - val intervalSpan: Int = conf.getProperty(StreamBenchConfig.DATAGEN_INTERVAL_SPAN).toInt - - val windowDuration: Long = conf.getProperty(StreamBenchConfig.FixWINDOW_DURATION).toLong - val windowSlideStep: Long = conf.getProperty(StreamBenchConfig.FixWINDOW_SLIDESTEP).toLong - - val coreNumber = conf.getProperty(HiBenchConfig.YARN_EXECUTOR_NUMBER).toInt * conf.getProperty(HiBenchConfig.YARN_EXECUTOR_CORES).toInt - - val producerNum = conf.getProperty(StreamBenchConfig.DATAGEN_PRODUCER_NUMBER).toInt - val reporterTopic = MetricsUtil.getTopic(Platform.SPARK, topic, producerNum, recordPerInterval, intervalSpan) - println("Reporter Topic" + reporterTopic) - val reporterTopicPartitions = conf.getProperty(StreamBenchConfig.KAFKA_TOPIC_PARTITIONS).toInt - MetricsUtil.createTopic(zkHost, reporterTopic, reporterTopicPartitions) - - val probability = conf.getProperty(StreamBenchConfig.SAMPLE_PROBABILITY).toDouble - // init SparkBenchConfig, it will be passed into every test case - val config = SparkBenchConfig(master, benchName, batchInterval, receiverNumber, copies, - enableWAL, checkPointPath, directMode, zkHost, consumerGroup, topic, reporterTopic, - brokerList, debugMode, coreNumber, probability, windowDuration, windowSlideStep) - - run(config) + this.run(args) } - private def run(config: SparkBenchConfig) { - - // select test case based on given benchName - val testCase : BenchBase = TestCase.withValue(config.benchName) match { - case TestCase.IDENTITY => new Identity() - case TestCase.REPARTITION => new Repartition() - case TestCase.WORDCOUNT => new WordCount() - case TestCase.FIXWINDOW => new FixWindow(config.windowDuration, config.windowSlideStep) - case other => - throw new Exception(s"test case ${other} is not supported") - } - - // defind streaming context - val conf = new SparkConf().setMaster(config.master).setAppName(config.benchName) - val ssc = new StreamingContext(conf, Milliseconds(config.batchInterval)) - ssc.checkpoint(config.checkpointPath) - - if(!config.debugMode) { - ssc.sparkContext.setLogLevel("ERROR") + def run(args: Array[String]) { + if (args.length < 1) { + BenchLogUtil.handleError("Usage: RunBench ") } - val lines: DStream[(String, String)] = if (config.directMode) { - // direct mode with low level Kafka API - KafkaUtils.createDirectStream[String, String, StringDecoder, StringDecoder]( - ssc, config.kafkaParams, Set(config.sourceTopic)) + val conf = new ConfigLoader(args(0)) - } else { - // receiver mode with high level Kafka API - val kafkaInputs = (1 to config.receiverNumber).map{ _ => - KafkaUtils.createStream[String, String, StringDecoder, StringDecoder]( - ssc, config.kafkaParams, Map(config.sourceTopic -> config.threadsPerReceiver), config.storageLevel) - } - ssc.union(kafkaInputs) + val benchName = conf.getProperty("hibench.streamingbench.benchname") + val topic = conf.getProperty("hibench.streamingbench.topic_name") + val master = conf.getProperty("hibench.spark.master") + val batchInterval = conf.getProperty("hibench.streamingbench.batch_interval").toInt + val zkHost = conf.getProperty("hibench.streamingbench.zookeeper.host") + val consumerGroup = conf.getProperty("hibench.streamingbench.consumer_group") + val kafkaThreads = conf.getProperty("hibench.streamingbench.receiver_nodes").toInt + val recordCount = conf.getProperty("hibench.streamingbench.record_count").toLong + val copies = conf.getProperty("hibench.streamingbench.copies").toInt + val testWAL = conf.getProperty("hibench.streamingbench.testWAL").toBoolean + val path = if (testWAL) conf.getProperty("hibench.streamingbench.checkpoint_path") else "" + val debug = conf.getProperty("hibench.streamingbench.debug").toBoolean + val directMode = conf.getProperty("hibench.streamingbench.direct_mode").toBoolean + val brokerList = if (directMode) conf.getProperty("hibench.streamingbench.brokerList") else "" + val totalParallel = conf.getProperty("hibench.yarn.executor.num").toInt * conf.getProperty("hibench.yarn.executor.cores").toInt + + this.reportDir = conf.getProperty("hibench.report.dir") + + val param = ParamEntity(master, benchName, batchInterval, zkHost, consumerGroup, topic, kafkaThreads, recordCount, copies, testWAL, path, debug, directMode, brokerList, totalParallel) + println(s"params:$param") + benchName match { + case "project" => + val fieldIndex = conf.getProperty("hibench.streamingbench.field_index").toInt + val separator = conf.getProperty("hibench.streamingbench.separator") + val ProjectTest = new StreamProjectionJob(param, fieldIndex, separator) + ProjectTest.run() + case "sample" => + val prob = conf.getProperty("hibench.streamingbench.prob").toDouble + val SampleTest = new SampleStreamJob(param, prob) + SampleTest.run() + case "statistics" => + val fieldIndex = conf.getProperty("hibench.streamingbench.field_index").toInt + val separator = conf.getProperty("hibench.streamingbench.separator") + val numericCalc = new NumericCalcJob(param, fieldIndex, separator) + numericCalc.run() + case "wordcount" => + val separator = conf.getProperty("hibench.streamingbench.separator") + val wordCount = new Wordcount(param, separator) + wordCount.run() + case "grep" => + val pattern = conf.getProperty("hibench.streamingbench.pattern") + val GrepStream = new GrepStreamJob(param, pattern) + GrepStream.run() + case "distinctcount" => + val fieldIndex = conf.getProperty("hibench.streamingbench.field_index").toInt + val separator = conf.getProperty("hibench.streamingbench.separator") + val distinct = new DistinctCountJob(param, fieldIndex, separator) + distinct.run() + case _ => + val emptyTest = new IdentityJob(param) + emptyTest.run() } - - // convent key from String to Long, it stands for event creation time. - val parsedLines = lines.map{ case (k, v) => (k.toLong, v) } - testCase.process(parsedLines, config) - - ssc.start() - ssc.awaitTermination() } } diff --git a/src/streambench/sparkbench/src/main/scala/com/intel/hibench/streambench/spark/application/FixWindow.scala b/src/streambench/sparkbench/src/main/scala/com/intel/hibench/streambench/spark/application/FixWindow.scala deleted file mode 100644 index a5b4ed28e..000000000 --- a/src/streambench/sparkbench/src/main/scala/com/intel/hibench/streambench/spark/application/FixWindow.scala +++ /dev/null @@ -1,56 +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.intel.hibench.streambench.spark.application - -import com.intel.hibench.streambench.common.UserVisitParser -import com.intel.hibench.streambench.common.metrics.KafkaReporter -import com.intel.hibench.streambench.spark.util.SparkBenchConfig -import org.apache.spark.streaming.Duration -import org.apache.spark.streaming.dstream.DStream - -class FixWindow(duration: Long, slideStep: Long) extends BenchBase { - - override def process(lines: DStream[(Long, String)], config: SparkBenchConfig): Unit = { - val reportTopic = config.reporterTopic - val brokerList = config.brokerList - - lines.window(Duration(duration), Duration(slideStep)).map{ - case (inTime, line) => { - val uv = UserVisitParser.parse(line) - (uv.getIp, (inTime, 1)) - } - }.reduceByKey((value, result) => { - // maintain the min time of this window and count record number - (Math.min(value._1, result._1), value._2 + result._2) - }).foreachRDD( rdd => rdd.foreachPartition( results => { - - // report back to kafka - val reporter = new KafkaReporter(reportTopic, brokerList) - val outTime = System.currentTimeMillis() - - results.foreach(res => { - (1 to (res._2._2)).foreach { _ => - reporter.report(res._2._1, outTime) - if(config.debugMode) { - println("Event: " + res._2._1 + ", " + outTime) - } - } - }) - })) - } -} \ No newline at end of file diff --git a/src/streambench/sparkbench/src/main/scala/com/intel/hibench/streambench/spark/application/Repartition.scala b/src/streambench/sparkbench/src/main/scala/com/intel/hibench/streambench/spark/application/Repartition.scala deleted file mode 100644 index 0549794aa..000000000 --- a/src/streambench/sparkbench/src/main/scala/com/intel/hibench/streambench/spark/application/Repartition.scala +++ /dev/null @@ -1,43 +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.intel.hibench.streambench.spark.application - -import com.intel.hibench.streambench.common.metrics.KafkaReporter -import com.intel.hibench.streambench.spark.util.SparkBenchConfig - -import org.apache.spark.streaming.dstream.DStream - -class Repartition() extends BenchBase { - - override def process(lines: DStream[(Long, String)], config: SparkBenchConfig): Unit = { - val reportTopic = config.reporterTopic - val brokerList = config.brokerList - - lines.repartition(config.coreNumber).foreachRDD(rdd => rdd.foreachPartition( partLines => { - val reporter = new KafkaReporter(reportTopic, brokerList) - partLines.foreach{ case (inTime , content) => - val outTime = System.currentTimeMillis() - reporter.report(inTime,outTime) - if(config.debugMode) { - println("Event: " + inTime + ", " + outTime) - } - } - })) - } -} - diff --git a/src/streambench/sparkbench/src/main/scala/com/intel/hibench/streambench/spark/application/WordCount.scala b/src/streambench/sparkbench/src/main/scala/com/intel/hibench/streambench/spark/application/WordCount.scala deleted file mode 100644 index 59b338f0d..000000000 --- a/src/streambench/sparkbench/src/main/scala/com/intel/hibench/streambench/spark/application/WordCount.scala +++ /dev/null @@ -1,60 +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.intel.hibench.streambench.spark.application - -import com.intel.hibench.streambench.common.UserVisitParser -import com.intel.hibench.streambench.common.metrics.KafkaReporter -import com.intel.hibench.streambench.spark.util.SparkBenchConfig -import org.apache.spark.streaming.dstream.DStream -import org.apache.spark.streaming.{StateSpec, State} - -class WordCount() extends BenchBase { - - override def process(lines: DStream[(Long, String)], config: SparkBenchConfig) = { - val reportTopic = config.reporterTopic - val brokerList = config.brokerList - - // Project Line to UserVisit, the output means "[IP, [Strat Time, Count]]" - val parsedLine: DStream[(String, (Long, Int))] = lines.map(line => { - val userVisit = UserVisitParser.parse(line._2) - (userVisit.getIp, (line._1, 1)) - }) - - // Define state mapping function - val mappingFunc = (ip: String, one: Option[(Long, Int)], state: State[Int]) => { - if (!one.isDefined) { - throw new Exception("input value is not defined. It should not happen as we don't use timeout function.") - } - val sum = one.get._2 + state.getOption.getOrElse(0) - state.update(sum) - (ip, one.get._1) - } - - - val wordCount = parsedLine.mapWithState(StateSpec.function(mappingFunc)) - - wordCount.foreachRDD(rdd => rdd.foreachPartition(partLines => { - val reporter = new KafkaReporter(reportTopic, brokerList) - partLines.foreach { case (word, inTime) => - val outTime = System.currentTimeMillis() - reporter.report(inTime, outTime) - if (config.debugMode) println(word + ": " + inTime + ", " + outTime ) - } - })) - } -} diff --git a/src/streambench/sparkbench/src/main/scala/com/intel/hibench/streambench/spark/entity/ParamEntity.scala b/src/streambench/sparkbench/src/main/scala/com/intel/hibench/streambench/spark/entity/ParamEntity.scala new file mode 100644 index 000000000..7fed782c0 --- /dev/null +++ b/src/streambench/sparkbench/src/main/scala/com/intel/hibench/streambench/spark/entity/ParamEntity.scala @@ -0,0 +1,19 @@ +package com.intel.hibench.streambench.spark.entity + +case class ParamEntity( + master: String, + appName: String, + batchInterval: Int, + zkHost: String, + consumerGroup: String, + topic: String, + threads: Int, + recordCount: Long, + copies: Int, + testWAL: Boolean, + path: String, + debug: Boolean, + directMode: Boolean, + brokerList: String, + totalParallel: Int +) diff --git a/src/streambench/sparkbench/src/main/scala/com/intel/hibench/streambench/spark/microbench/DistinctCountJob.scala b/src/streambench/sparkbench/src/main/scala/com/intel/hibench/streambench/spark/microbench/DistinctCountJob.scala new file mode 100644 index 000000000..ec4699ae4 --- /dev/null +++ b/src/streambench/sparkbench/src/main/scala/com/intel/hibench/streambench/spark/microbench/DistinctCountJob.scala @@ -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.intel.hibench.streambench.spark.microbench + +import com.intel.hibench.streambench.spark.entity.ParamEntity +import org.apache.spark.streaming.dstream.DStream +import org.apache.spark.streaming.StreamingContext +import com.intel.hibench.streambench.spark.util.BenchLogUtil +import org.apache.spark.streaming.StreamingContext._ + +import scala.collection.mutable.ArrayBuffer + +object SetPool { + private var iset: Set[String] = _ + def getSet(): Set[String] = synchronized { + if (iset == null) { + iset = Set() + } + iset + } + def setSet(iset: Set[String]): Unit = synchronized { + this.iset = iset + } +} + +class DistinctCountJob (subClassParams:ParamEntity, fieldIndex:Int, separator:String) extends RunBenchJobWithInit(subClassParams) { + + override def processStreamData(lines:DStream[String], ssc:StreamingContext) { + val index = fieldIndex + val sep = separator + + val distinctcount = lines + .flatMap(line => { + val splits = line.split(sep) + if (index < splits.length) + Traversable(splits(index)) + else + Traversable.empty + }) + .map(word => (word, 1)) + .reduceByKey((x, y) => x) + + distinctcount.foreachRDD(rdd => { + rdd.foreachPartition(partitionOfRecords => { + var iset = SetPool.getSet + partitionOfRecords.foreach{case(word, count) => + iset += word + } + SetPool.setSet(iset) + }) + }) + } +} diff --git a/src/streambench/sparkbench/src/main/scala/com/intel/hibench/streambench/spark/application/Identity.scala b/src/streambench/sparkbench/src/main/scala/com/intel/hibench/streambench/spark/microbench/GrepStreamJob.scala similarity index 52% rename from src/streambench/sparkbench/src/main/scala/com/intel/hibench/streambench/spark/application/Identity.scala rename to src/streambench/sparkbench/src/main/scala/com/intel/hibench/streambench/spark/microbench/GrepStreamJob.scala index dcaceb148..3e6037810 100644 --- a/src/streambench/sparkbench/src/main/scala/com/intel/hibench/streambench/spark/application/Identity.scala +++ b/src/streambench/sparkbench/src/main/scala/com/intel/hibench/streambench/spark/microbench/GrepStreamJob.scala @@ -15,28 +15,24 @@ * limitations under the License. */ -package com.intel.hibench.streambench.spark.application - -import com.intel.hibench.streambench.common.metrics.KafkaReporter -import com.intel.hibench.streambench.spark.util.SparkBenchConfig +package com.intel.hibench.streambench.spark.microbench +import com.intel.hibench.streambench.spark.entity.ParamEntity import org.apache.spark.streaming.dstream.DStream +import org.apache.spark.streaming.StreamingContext +import com.intel.hibench.streambench.spark.util.BenchLogUtil -class Identity() extends BenchBase { - - override def process(lines: DStream[(Long, String)], config: SparkBenchConfig): Unit = { - val reportTopic = config.reporterTopic - val brokerList = config.brokerList +class GrepStreamJob(subClassParams:ParamEntity,patternStr:String) extends RunBenchJobWithInit(subClassParams){ + override def processStreamData(lines:DStream[String],ssc:StreamingContext){ + BenchLogUtil.logMsg("In GrepStreamJob") + val pattern=patternStr + val debug=subClassParams.debug + val matches=lines.filter(_.contains(pattern)) - lines.foreachRDD(rdd => rdd.foreachPartition( partLines => { - val reporter = new KafkaReporter(reportTopic, brokerList) - partLines.foreach{ case (inTime , content) => - val outTime = System.currentTimeMillis() - reporter.report(inTime, outTime) - if(config.debugMode) { - println("Event: " + inTime + ", " + outTime) - } - } - })) + if(debug){ + matches.print() + }else{ + matches.foreachRDD( rdd => rdd.foreach( _ => Unit )) + } } } diff --git a/src/streambench/sparkbench/src/main/scala/com/intel/hibench/streambench/spark/application/BenchBase.scala b/src/streambench/sparkbench/src/main/scala/com/intel/hibench/streambench/spark/microbench/IdentityJob.scala similarity index 68% rename from src/streambench/sparkbench/src/main/scala/com/intel/hibench/streambench/spark/application/BenchBase.scala rename to src/streambench/sparkbench/src/main/scala/com/intel/hibench/streambench/spark/microbench/IdentityJob.scala index 84b409ce2..b08aaad5b 100644 --- a/src/streambench/sparkbench/src/main/scala/com/intel/hibench/streambench/spark/application/BenchBase.scala +++ b/src/streambench/sparkbench/src/main/scala/com/intel/hibench/streambench/spark/microbench/IdentityJob.scala @@ -15,16 +15,15 @@ * limitations under the License. */ -package com.intel.hibench.streambench.spark.application - -import com.intel.hibench.streambench.common.metrics.KafkaReporter -import com.intel.hibench.streambench.spark.util.SparkBenchConfig +package com.intel.hibench.streambench.spark.microbench +import com.intel.hibench.streambench.spark.entity.ParamEntity import org.apache.spark.streaming.dstream.DStream +import org.apache.spark.streaming.StreamingContext + +class IdentityJob(subClassParams:ParamEntity) extends RunBenchJobWithInit(subClassParams) { -/** - * The base class of all test cases in spark. The sub class need to implement "process" method - */ -trait BenchBase { - def process(lines: DStream[(Long, String)], config: SparkBenchConfig): Unit + override def processStreamData(lines:DStream[String],ssc:StreamingContext){ + lines.foreachRDD(rdd => rdd.foreach( _ => Unit )) + } } diff --git a/src/streambench/sparkbench/src/main/scala/com/intel/hibench/streambench/spark/microbench/NumericCalcJob.scala b/src/streambench/sparkbench/src/main/scala/com/intel/hibench/streambench/spark/microbench/NumericCalcJob.scala new file mode 100644 index 000000000..5a174ad0f --- /dev/null +++ b/src/streambench/sparkbench/src/main/scala/com/intel/hibench/streambench/spark/microbench/NumericCalcJob.scala @@ -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 com.intel.hibench.streambench.spark.microbench + +import com.intel.hibench.streambench.spark.entity.ParamEntity +import org.apache.spark.streaming.dstream.DStream +import com.intel.hibench.streambench.spark.metrics.LatencyListener +import org.apache.spark.streaming.StreamingContext +import com.intel.hibench.streambench.spark.util.BenchLogUtil + +case class MultiReducer(var max: Long, var min: Long, var sum: Long, var count: Long) extends Serializable { + def this() = this(0, Int.MaxValue, 0, 0) + + def reduceValue(value: Long): MultiReducer = { + this.max = Math.max(this.max, value) + this.min = Math.min(this.min, value) + this.sum += value + this.count += 1 + this + } + + def reduce(that: MultiReducer): MultiReducer = { + this.max = Math.max(this.max, that.max) + this.min = Math.min(this.min, that.min) + this.sum += that.sum + this.count += that.count + this + } +} + +class NumericCalcJob(subClassParams: ParamEntity, fieldIndex: Int, separator: String) + extends RunBenchJobWithInit(subClassParams) { + + var history_statistics = new MultiReducer() + + override def processStreamData(lines: DStream[String], ssc: StreamingContext) { + val index = fieldIndex + val sep = separator + + lines.foreachRDD( rdd => { + val numbers = rdd.flatMap( line => { + val splits = line.trim.split(sep) + if (index < splits.length) + Iterator(splits(index).toLong) + else + Iterator.empty + }) + + var zero = new MultiReducer() + val cur = numbers.map(x => new MultiReducer(x, x, x, 1)) + .fold(zero)((v1, v2) => v1.reduce(v2)) + //var cur = numbers.aggregate(zero)((v, x) => v.reduceValue(x), (v1, v2) => v1.reduce(v2)) + history_statistics.reduce(cur) + + BenchLogUtil.logMsg("Current max: " + history_statistics.max) + BenchLogUtil.logMsg("Current min: " + history_statistics.min) + BenchLogUtil.logMsg("Current sum: " + history_statistics.sum) + BenchLogUtil.logMsg("Current total: " + history_statistics.count) + + }) + } +} diff --git a/src/streambench/sparkbench/src/main/scala/com/intel/hibench/streambench/spark/microbench/RunBenchWithInit.scala b/src/streambench/sparkbench/src/main/scala/com/intel/hibench/streambench/spark/microbench/RunBenchWithInit.scala new file mode 100644 index 000000000..262e16b29 --- /dev/null +++ b/src/streambench/sparkbench/src/main/scala/com/intel/hibench/streambench/spark/microbench/RunBenchWithInit.scala @@ -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 com.intel.hibench.streambench.spark.microbench + +import com.intel.hibench.streambench.spark.entity.ParamEntity +import org.apache.spark.SparkConf +import org.apache.spark.streaming.{Seconds,StreamingContext} + +import org.apache.spark.streaming.kafka._ +import org.apache.spark.streaming.dstream._ +import com.intel.hibench.streambench.spark.metrics._ +import org.apache.spark.storage.StorageLevel +import kafka.serializer.StringDecoder + +class RunBenchJobWithInit(params:ParamEntity) extends SpoutTops { + + def run(){ + val conf = new SparkConf().setMaster(params.master) + .setAppName(params.appName) + .set("spark.cleaner.ttl", "7200") + + var ssc:StreamingContext=null + + if (!params.testWAL) { + ssc = new StreamingContext(conf, Seconds(params.batchInterval)) + } else { + val create = ()=> new StreamingContext(conf, Seconds(params.batchInterval)) + ssc = StreamingContext.getOrCreate(params.path, create) + ssc.checkpoint(params.path) + } + + val listener = new LatencyListener(ssc, params) + ssc.addStreamingListener(listener) + + var lines:DStream[String] = null + if (params.directMode) + lines = createDirectStream(ssc).map(_._2) + else + lines = createStream(ssc).map(_._2) + + processStreamData(lines, ssc) + + ssc.start() + ssc.awaitTermination() + } + + def createStream(ssc:StreamingContext):DStream[(String, String)] = { + val kafkaParams=Map( + "zookeeper.connect" -> params.zkHost, + "group.id" -> params.consumerGroup, + "rebalance.backoff.ms" -> "20000", + "zookeeper.session.timeout.ms" -> "20000" + ) + + var storageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 + if (params.copies == 1) + storageLevel = StorageLevel.MEMORY_ONLY + + val kafkaInputs = (1 to params.threads).map{_ => + println(s"Create kafka input, args:$kafkaParams") + KafkaUtils.createStream[String, String, StringDecoder, StringDecoder](ssc, kafkaParams, + Map(params.topic -> params.totalParallel / params.threads), storageLevel) + } + + ssc.union(kafkaInputs) + } + + def createDirectStream(ssc:StreamingContext):DStream[(String, String)]={ + val kafkaParams = Map( + "metadata.broker.list" -> params.brokerList, + "auto.offset.reset" -> "smallest", + "socket.receive.buffer.size" -> "1024*1024*1024" + ) + println(s"Create direct kafka stream, args:$kafkaParams") + KafkaUtils.createDirectStream[String, String, StringDecoder, StringDecoder](ssc, kafkaParams, Set(params.topic)) + } + +} diff --git a/src/streambench/sparkbench/src/main/scala/com/intel/hibench/streambench/spark/microbench/SampleStreamJob.scala b/src/streambench/sparkbench/src/main/scala/com/intel/hibench/streambench/spark/microbench/SampleStreamJob.scala new file mode 100644 index 000000000..30ead6ffb --- /dev/null +++ b/src/streambench/sparkbench/src/main/scala/com/intel/hibench/streambench/spark/microbench/SampleStreamJob.scala @@ -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.intel.hibench.streambench.spark.microbench + +import com.intel.hibench.streambench.spark.entity.ParamEntity +import org.apache.spark.streaming.dstream.DStream +import com.intel.hibench.streambench.spark.util.BenchLogUtil +import org.apache.spark.streaming.StreamingContext + +object ThreadLocalRandom extends Serializable{ + private val localRandom = new ThreadLocal[util.Random] { + override protected def initialValue() = new util.Random + } + + def current = localRandom.get +} + +class SampleStreamJob(subClassParams:ParamEntity, probability:Double) + extends RunBenchJobWithInit(subClassParams) { + + override def processStreamData(lines:DStream[String], ssc:StreamingContext){ + val prob = probability + val samples = lines.filter( _=> { + ThreadLocalRandom.current.nextDouble() < prob + }) + val debug = subClassParams.debug + if(debug){ + var totalCount = 0L + samples.foreachRDD(rdd => { + totalCount += rdd.count() + BenchLogUtil.logMsg("Current sample count:"+totalCount) + }) + }else{ + samples.foreachRDD(rdd => rdd.foreach( _ => Unit )) + } + + } +} diff --git a/src/streambench/common/src/main/scala/com/intel/hibench/streambench/common/metrics/LatencyReporter.scala b/src/streambench/sparkbench/src/main/scala/com/intel/hibench/streambench/spark/microbench/SpoutTops.scala similarity index 76% rename from src/streambench/common/src/main/scala/com/intel/hibench/streambench/common/metrics/LatencyReporter.scala rename to src/streambench/sparkbench/src/main/scala/com/intel/hibench/streambench/spark/microbench/SpoutTops.scala index 33a2e253d..00f5816fe 100644 --- a/src/streambench/common/src/main/scala/com/intel/hibench/streambench/common/metrics/LatencyReporter.scala +++ b/src/streambench/sparkbench/src/main/scala/com/intel/hibench/streambench/spark/microbench/SpoutTops.scala @@ -14,10 +14,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.intel.hibench.streambench.common.metrics -trait LatencyReporter extends java.io.Serializable { +package com.intel.hibench.streambench.spark.microbench - def report(startTime: Long, endTime: Long): Unit +import org.apache.spark.streaming.StreamingContext +import org.apache.spark.streaming.dstream.DStream +trait SpoutTops { + def processStreamData(lines:DStream[String],ssc:StreamingContext){ + + } } diff --git a/src/streambench/sparkbench/src/main/scala/com/intel/hibench/streambench/spark/microbench/StreamProjectionJob.scala b/src/streambench/sparkbench/src/main/scala/com/intel/hibench/streambench/spark/microbench/StreamProjectionJob.scala new file mode 100644 index 000000000..3321e4da0 --- /dev/null +++ b/src/streambench/sparkbench/src/main/scala/com/intel/hibench/streambench/spark/microbench/StreamProjectionJob.scala @@ -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.intel.hibench.streambench.spark.microbench + +import com.intel.hibench.streambench.spark.entity.ParamEntity +import org.apache.spark.streaming.dstream.DStream +import org.apache.spark.streaming.StreamingContext +import com.intel.hibench.streambench.spark.util.BenchLogUtil + +class StreamProjectionJob(subClassParams:ParamEntity,fieldIndex:Int,separator:String) + extends RunBenchJobWithInit(subClassParams) { + + override def processStreamData(lines:DStream[String],ssc:StreamingContext){ + val sep = separator + val index = fieldIndex + val debug = subClassParams.debug + lines.foreachRDD(rdd => { + val fields = rdd.flatMap(line => { + val splits = line.trim.split(sep) + if(index < splits.length) + Iterator(splits(index)) + else + Iterator.empty + }) + fields.foreach(rdd => rdd.foreach( _ => Unit )) + if(debug) + BenchLogUtil.logMsg(fields.collect().mkString("\n")) + }) + } + +} diff --git a/src/streambench/sparkbench/src/main/scala/com/intel/hibench/streambench/spark/microbench/Wordcount.scala b/src/streambench/sparkbench/src/main/scala/com/intel/hibench/streambench/spark/microbench/Wordcount.scala new file mode 100644 index 000000000..f99efc0ee --- /dev/null +++ b/src/streambench/sparkbench/src/main/scala/com/intel/hibench/streambench/spark/microbench/Wordcount.scala @@ -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.intel.hibench.streambench.spark.microbench + +import com.intel.hibench.streambench.spark.entity.ParamEntity +import org.apache.spark.streaming.dstream.DStream +import com.intel.hibench.streambench.spark.metrics.LatencyListener +import org.apache.spark.streaming.StreamingContext +import com.intel.hibench.streambench.spark.util.BenchLogUtil +import org.apache.spark.SparkContext +import org.apache.spark.SparkContext._ +import scala.collection.mutable.Map + +object MapPool { + private var imap: Map[String, Long] = _ + def getMap(): Map[String, Long] = synchronized { + if (imap == null) imap = Map() + imap + } + def setMap(imap: Map[String, Long]) = synchronized { + this.imap = imap + } +} + +class Wordcount(subClassParams:ParamEntity,separator:String) + extends RunBenchJobWithInit(subClassParams){ + + override def processStreamData(lines:DStream[String],ssc:StreamingContext){ + val sep = separator + val wordcount = lines + .flatMap(x => x.split(sep)) + .map(word => (word, 1)) + .reduceByKey(_ + _) + + wordcount.foreachRDD(rdd=> { + rdd.foreachPartition(partitionOfRecords => { + val imap = MapPool.getMap + partitionOfRecords.foreach{case (word, count) => + imap(word) = if (imap.contains(word)) imap(word) + count else count + } + MapPool.setMap(imap) + }) + }) + } +} diff --git a/src/streambench/common/src/main/scala/com/intel/hibench/streambench/common/metrics/MetricsReader.scala b/src/streambench/sparkbench/src/main/scala/com/intel/hibench/streambench/spark/util/LogUtil.scala similarity index 50% rename from src/streambench/common/src/main/scala/com/intel/hibench/streambench/common/metrics/MetricsReader.scala rename to src/streambench/sparkbench/src/main/scala/com/intel/hibench/streambench/spark/util/LogUtil.scala index 9d9637985..5de219936 100644 --- a/src/streambench/common/src/main/scala/com/intel/hibench/streambench/common/metrics/MetricsReader.scala +++ b/src/streambench/sparkbench/src/main/scala/com/intel/hibench/streambench/spark/util/LogUtil.scala @@ -14,20 +14,34 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.intel.hibench.streambench.common.metrics -object MetricsReader extends App { +package com.intel.hibench.streambench.spark.util - if (args.length < 5) { - System.err.println("args: need to be specified!") +import org.apache.spark.Logging +import org.apache.log4j.{Level, Logger} +import com.intel.hibench.streambench.spark.RunBench + +object BenchLogUtil extends Logging{ + def setLogLevel(){ + val log4jInitialized = Logger.getRootLogger.getAllAppenders.hasMoreElements + if (!log4jInitialized) { + logInfo("Setting log level to [WARN] for streaming example." + + " To override add a custom log4j.properties to the classpath.") + Logger.getRootLogger.setLevel(Level.WARN) + } + } + + val file=new java.io.File(RunBench.reportDir + "/streamingbench/spark/streambenchlog.txt") + val out=new java.io.PrintWriter(file) + + def logMsg(msg:String) { + out.println(msg) + out.flush() + System.out.println(msg) + } + + def handleError(msg:String){ + System.err.println(msg) System.exit(1) } - - val zookeeperConnect = args(0) - val topic = args(1) - val outputDir = args(2) - val sampleNum = args(3).toInt - val threadNum = args(4).toInt - val latencyCollector = new KafkaCollector(zookeeperConnect, topic, outputDir, sampleNum, threadNum) - latencyCollector.start() } diff --git a/src/streambench/sparkbench/src/main/scala/com/intel/hibench/streambench/spark/util/SparkBenchConfig.scala b/src/streambench/sparkbench/src/main/scala/com/intel/hibench/streambench/spark/util/SparkBenchConfig.scala deleted file mode 100644 index cc82a4d95..000000000 --- a/src/streambench/sparkbench/src/main/scala/com/intel/hibench/streambench/spark/util/SparkBenchConfig.scala +++ /dev/null @@ -1,44 +0,0 @@ -package com.intel.hibench.streambench.spark.util - -import org.apache.spark.storage.StorageLevel - -case class SparkBenchConfig ( - // Spark - master: String, - benchName: String, - batchInterval: Int, - receiverNumber: Int, - copies: Int, - enableWAL: Boolean, - checkpointPath: String, - directMode: Boolean, - - // Kafka - zkHost: String, - consumerGroup: String, - sourceTopic: String, - reporterTopic: String, - brokerList: String, - - - // Hibench - debugMode: Boolean, - coreNumber: Int, - sampleProbability: Double, - windowDuration: Long, - windowSlideStep: Long) { - - def storageLevel = copies match { - case 0 => StorageLevel.MEMORY_ONLY - case 1 => StorageLevel.MEMORY_AND_DISK_SER - case _ => StorageLevel.MEMORY_AND_DISK_SER_2 - } - - def kafkaParams = Map ( - "group.id" -> consumerGroup, - "zookeeper.connect" -> zkHost, - "metadata.broker.list" -> brokerList - ) - - def threadsPerReceiver = coreNumber / receiverNumber -} diff --git a/src/streambench/sparkbench/src/multiapi/scala/spark1.3/LatencyCollector.scala b/src/streambench/sparkbench/src/multiapi/scala/spark1.3/LatencyCollector.scala new file mode 100644 index 000000000..eb238df65 --- /dev/null +++ b/src/streambench/sparkbench/src/multiapi/scala/spark1.3/LatencyCollector.scala @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.intel.hibench.streambench.spark.metrics + +import com.intel.hibench.streambench.spark.entity.ParamEntity +import org.apache.spark.streaming.StreamingContext +import org.apache.spark.streaming.scheduler._ +import com.intel.hibench.streambench.spark.util._ + +class StopContextThread(ssc: StreamingContext) extends Runnable { + def run { + ssc.stop(true, true) + } +} + +class LatencyListener(ssc: StreamingContext, params: ParamEntity) extends StreamingListener { + + var startTime=0L + var endTime=0L + //This delay is processDelay of every batch * record count in this batch + var totalDelay=0L + var hasStarted=false + var batchCount=0 + var totalRecords=0L + + val thread: Thread = new Thread(new StopContextThread(ssc)) + + override def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted): Unit = { + val clazz:Class[_] = Class.forName("org.apache.spark.streaming.scheduler.ReceivedBlockInfo") + val numRecordMethod = clazz.getMethod("numRecords") + val batchInfo = batchCompleted.batchInfo + val map = batchInfo.receivedBlockInfo + val prevCount = totalRecords + var recordThisBatch = 0L + + map.foreach{ + case (id,array) => { + for(receivedInfo <- array){ + val thisRecord = numRecordMethod.invoke(receivedInfo) +// BenchLogUtil.logMsg("This time id:"+id+" thisRecord:"+thisRecord) + recordThisBatch += thisRecord.toString.toLong + } + } + } + + if (!thread.isAlive) { + totalRecords += recordThisBatch + BenchLogUtil.logMsg("LatencyController: this batch: " + recordThisBatch) + BenchLogUtil.logMsg("LatencyController: total records: " + totalRecords + " Receivers: " + map.size) + } + + if (totalRecords >= params.recordCount) { + if(hasStarted && !thread.isAlive){ + //not receiving any data more, finish + endTime = System.currentTimeMillis() + val totalTime = (endTime-startTime).toDouble / 1000 + //This is weighted avg of every batch process time. The weight is records processed int the batch + val avgLatency = totalDelay.toDouble / totalRecords + if(avgLatency > params.batchInterval.toDouble * 1000) + BenchLogUtil.logMsg("WARNING:SPARK CLUSTER IN UNSTABLE STATE. TRY REDUCE INPUT SPEED") + + val avgLatencyAdjust = avgLatency + params.batchInterval.toDouble * 500 + val recordThroughput = params.recordCount / totalTime + BenchLogUtil.logMsg("Batch count = " + batchCount) + BenchLogUtil.logMsg("Total processing delay = " + totalDelay + " ms") + BenchLogUtil.logMsg("Consumed time = " + totalTime + " s") + BenchLogUtil.logMsg("Avg latency/batchInterval = " + avgLatencyAdjust + " ms") + BenchLogUtil.logMsg("Avg records/sec = " + recordThroughput + " records/s") + thread.start + } + }else if(!hasStarted){ + startTime = batchCompleted.batchInfo.submissionTime + hasStarted = true + } + + if(hasStarted){ +// BenchLogUtil.logMsg("This delay:"+batchCompleted.batchInfo.processingDelay+"ms") + batchCompleted.batchInfo.processingDelay match{ + case Some(value) => totalDelay += value * recordThisBatch + case None => //Nothing + } + batchCount += 1 + } + } + +} diff --git a/src/streambench/sparkbench/src/multiapi/scala/spark1.4/LatencyCollector.scala b/src/streambench/sparkbench/src/multiapi/scala/spark1.4/LatencyCollector.scala new file mode 100644 index 000000000..58af3b952 --- /dev/null +++ b/src/streambench/sparkbench/src/multiapi/scala/spark1.4/LatencyCollector.scala @@ -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.intel.hibench.streambench.spark.metrics + +import com.intel.hibench.streambench.spark.entity.ParamEntity +import org.apache.spark.streaming.StreamingContext +import org.apache.spark.streaming.scheduler._ +import com.intel.hibench.streambench.spark.util._ + +class StopContextThread(ssc: StreamingContext) extends Runnable { + def run { + ssc.stop(true, true) + } +} + +class LatencyListener(ssc: StreamingContext, params: ParamEntity) extends StreamingListener { + + var startTime=0L + var endTime=0L + //This delay is processDelay of every batch * record count in this batch + var totalDelay=0L + var hasStarted=false + var batchCount=0 + var totalRecords=0L + + val thread: Thread = new Thread(new StopContextThread(ssc)) + + override def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted): Unit={ + val batchInfo = batchCompleted.batchInfo + val prevCount=totalRecords + var recordThisBatch = batchInfo.numRecords + + if (!thread.isAlive) { + totalRecords += recordThisBatch + BenchLogUtil.logMsg("LatencyController: this batch: " + recordThisBatch) + BenchLogUtil.logMsg("LatencyController: total records: " + totalRecords) + } + + if (totalRecords >= params.recordCount) { + if (hasStarted && !thread.isAlive) { + //not receiving any data more, finish + endTime = System.currentTimeMillis() + val totalTime = (endTime-startTime).toDouble/1000 + //This is weighted avg of every batch process time. The weight is records processed int the batch + val avgLatency = totalDelay.toDouble/totalRecords + if (avgLatency > params.batchInterval.toDouble*1000) + BenchLogUtil.logMsg("WARNING:SPARK CLUSTER IN UNSTABLE STATE. TRY REDUCE INPUT SPEED") + + val avgLatencyAdjust = avgLatency + params.batchInterval.toDouble*500 + val recordThroughput = params.recordCount / totalTime + BenchLogUtil.logMsg("Batch count = " + batchCount) + BenchLogUtil.logMsg("Total processing delay = " + totalDelay + " ms") + BenchLogUtil.logMsg("Consumed time = " + totalTime + " s") + BenchLogUtil.logMsg("Avg latency/batchInterval = " + avgLatencyAdjust + " ms") + BenchLogUtil.logMsg("Avg records/sec = " + recordThroughput + " records/s") + thread.start + } + } else if (!hasStarted) { + startTime = batchCompleted.batchInfo.submissionTime + hasStarted = true + } + + if (hasStarted) { +// BenchLogUtil.logMsg("This delay:"+batchCompleted.batchInfo.processingDelay+"ms") + batchCompleted.batchInfo.processingDelay match { + case Some(value) => totalDelay += value*recordThisBatch + case None => //Nothing + } + batchCount = batchCount+1 + } + } + +} diff --git a/src/streambench/sparkbench/src/multiapi/scala/spark1.5/LatencyCollector.scala b/src/streambench/sparkbench/src/multiapi/scala/spark1.5/LatencyCollector.scala new file mode 100644 index 000000000..58af3b952 --- /dev/null +++ b/src/streambench/sparkbench/src/multiapi/scala/spark1.5/LatencyCollector.scala @@ -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.intel.hibench.streambench.spark.metrics + +import com.intel.hibench.streambench.spark.entity.ParamEntity +import org.apache.spark.streaming.StreamingContext +import org.apache.spark.streaming.scheduler._ +import com.intel.hibench.streambench.spark.util._ + +class StopContextThread(ssc: StreamingContext) extends Runnable { + def run { + ssc.stop(true, true) + } +} + +class LatencyListener(ssc: StreamingContext, params: ParamEntity) extends StreamingListener { + + var startTime=0L + var endTime=0L + //This delay is processDelay of every batch * record count in this batch + var totalDelay=0L + var hasStarted=false + var batchCount=0 + var totalRecords=0L + + val thread: Thread = new Thread(new StopContextThread(ssc)) + + override def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted): Unit={ + val batchInfo = batchCompleted.batchInfo + val prevCount=totalRecords + var recordThisBatch = batchInfo.numRecords + + if (!thread.isAlive) { + totalRecords += recordThisBatch + BenchLogUtil.logMsg("LatencyController: this batch: " + recordThisBatch) + BenchLogUtil.logMsg("LatencyController: total records: " + totalRecords) + } + + if (totalRecords >= params.recordCount) { + if (hasStarted && !thread.isAlive) { + //not receiving any data more, finish + endTime = System.currentTimeMillis() + val totalTime = (endTime-startTime).toDouble/1000 + //This is weighted avg of every batch process time. The weight is records processed int the batch + val avgLatency = totalDelay.toDouble/totalRecords + if (avgLatency > params.batchInterval.toDouble*1000) + BenchLogUtil.logMsg("WARNING:SPARK CLUSTER IN UNSTABLE STATE. TRY REDUCE INPUT SPEED") + + val avgLatencyAdjust = avgLatency + params.batchInterval.toDouble*500 + val recordThroughput = params.recordCount / totalTime + BenchLogUtil.logMsg("Batch count = " + batchCount) + BenchLogUtil.logMsg("Total processing delay = " + totalDelay + " ms") + BenchLogUtil.logMsg("Consumed time = " + totalTime + " s") + BenchLogUtil.logMsg("Avg latency/batchInterval = " + avgLatencyAdjust + " ms") + BenchLogUtil.logMsg("Avg records/sec = " + recordThroughput + " records/s") + thread.start + } + } else if (!hasStarted) { + startTime = batchCompleted.batchInfo.submissionTime + hasStarted = true + } + + if (hasStarted) { +// BenchLogUtil.logMsg("This delay:"+batchCompleted.batchInfo.processingDelay+"ms") + batchCompleted.batchInfo.processingDelay match { + case Some(value) => totalDelay += value*recordThisBatch + case None => //Nothing + } + batchCount = batchCount+1 + } + } + +} diff --git a/src/streambench/sparkbench/src/multiapi/scala/spark1.6/LatencyCollector.scala b/src/streambench/sparkbench/src/multiapi/scala/spark1.6/LatencyCollector.scala new file mode 100644 index 000000000..58af3b952 --- /dev/null +++ b/src/streambench/sparkbench/src/multiapi/scala/spark1.6/LatencyCollector.scala @@ -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.intel.hibench.streambench.spark.metrics + +import com.intel.hibench.streambench.spark.entity.ParamEntity +import org.apache.spark.streaming.StreamingContext +import org.apache.spark.streaming.scheduler._ +import com.intel.hibench.streambench.spark.util._ + +class StopContextThread(ssc: StreamingContext) extends Runnable { + def run { + ssc.stop(true, true) + } +} + +class LatencyListener(ssc: StreamingContext, params: ParamEntity) extends StreamingListener { + + var startTime=0L + var endTime=0L + //This delay is processDelay of every batch * record count in this batch + var totalDelay=0L + var hasStarted=false + var batchCount=0 + var totalRecords=0L + + val thread: Thread = new Thread(new StopContextThread(ssc)) + + override def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted): Unit={ + val batchInfo = batchCompleted.batchInfo + val prevCount=totalRecords + var recordThisBatch = batchInfo.numRecords + + if (!thread.isAlive) { + totalRecords += recordThisBatch + BenchLogUtil.logMsg("LatencyController: this batch: " + recordThisBatch) + BenchLogUtil.logMsg("LatencyController: total records: " + totalRecords) + } + + if (totalRecords >= params.recordCount) { + if (hasStarted && !thread.isAlive) { + //not receiving any data more, finish + endTime = System.currentTimeMillis() + val totalTime = (endTime-startTime).toDouble/1000 + //This is weighted avg of every batch process time. The weight is records processed int the batch + val avgLatency = totalDelay.toDouble/totalRecords + if (avgLatency > params.batchInterval.toDouble*1000) + BenchLogUtil.logMsg("WARNING:SPARK CLUSTER IN UNSTABLE STATE. TRY REDUCE INPUT SPEED") + + val avgLatencyAdjust = avgLatency + params.batchInterval.toDouble*500 + val recordThroughput = params.recordCount / totalTime + BenchLogUtil.logMsg("Batch count = " + batchCount) + BenchLogUtil.logMsg("Total processing delay = " + totalDelay + " ms") + BenchLogUtil.logMsg("Consumed time = " + totalTime + " s") + BenchLogUtil.logMsg("Avg latency/batchInterval = " + avgLatencyAdjust + " ms") + BenchLogUtil.logMsg("Avg records/sec = " + recordThroughput + " records/s") + thread.start + } + } else if (!hasStarted) { + startTime = batchCompleted.batchInfo.submissionTime + hasStarted = true + } + + if (hasStarted) { +// BenchLogUtil.logMsg("This delay:"+batchCompleted.batchInfo.processingDelay+"ms") + batchCompleted.batchInfo.processingDelay match { + case Some(value) => totalDelay += value*recordThisBatch + case None => //Nothing + } + batchCount = batchCount+1 + } + } + +} diff --git a/src/streambench/stormbench/pom.xml b/src/streambench/stormbench/pom.xml index 840f2928c..dbcbc8af0 100644 --- a/src/streambench/stormbench/pom.xml +++ b/src/streambench/stormbench/pom.xml @@ -1,130 +1,189 @@ - - - + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> 4.0.0 + com.intel.hibench streambench 5.0-SNAPSHOT - ../pom.xml com.intel.hibench.streambench streaming-bench-storm + 0.1-SNAPSHOT jar Streaming Benchmark Storm + + UTF-8 + + + + junit + junit + 3.8.1 + test + + + org.testng + testng + 6.8.5 + test + + + org.mockito + mockito-all + 1.9.0 + test + + + org.easytesting + fest-assert-core + 2.0M8 + test + + + org.jmock + jmock + 2.6.0 + test + org.apache.storm storm-core - 1.0.1 + 0.9.3 + provided org.apache.storm storm-kafka - 1.0.1 - - - org.slf4j - slf4j-api - - - org.apache.kafka - kafka-clients - - + 0.9.3 + compile org.apache.kafka - kafka_2.11 - 0.8.2.1 + kafka_2.10 + 0.8.1.1 + compile - - org.apache.zookeeper - zookeeper - - - org.slf4j - slf4j-api - - - org.slf4j - slf4j-log4j12 - - - org.slf4j - log4j-over-slf4j - - - log4j - log4j - + + org.apache.zookeeper + zookeeper + + + log4j + log4j + + + commons-collections + commons-collections + 3.2.2 + com.google.guava guava 15.0 - com.intel.hibench.streambench - streaming-bench-common - ${project.version} - - - org.slf4j - slf4j-log4j12 - - - log4j - log4j - - + org.apache.thrift + libthrift + 0.9.1 + src/main + + - org.apache.maven.plugins - maven-shade-plugin + maven-assembly-plugin - true + + jar-with-dependencies + + + + + + + make-assembly package - shade + single + + + + + + + com.theoryinpractise + clojure-maven-plugin + 1.3.12 + true + + + src/clj + + + + + compile + compile + + compile + + + + + + + + org.codehaus.mojo + exec-maven-plugin + 1.2.1 + + + + exec - - - - - - - + + java + true + false + compile + ${storm.topology} + + + + + org.apache.maven.plugins + maven-compiler-plugin + + 1.6 + 1.6 + diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/RunBench.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/RunBench.java index de4f7319e..2806d61de 100644 --- a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/RunBench.java +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/RunBench.java @@ -17,89 +17,125 @@ package com.intel.hibench.streambench.storm; -import com.intel.hibench.streambench.common.ConfigLoader; -import com.intel.hibench.streambench.common.Platform; -import com.intel.hibench.streambench.common.StreamBenchConfig; -import com.intel.hibench.streambench.common.TestCase; -import com.intel.hibench.streambench.common.metrics.MetricsUtil; +import com.intel.hibench.streambench.storm.util.ConfigLoader; +import com.intel.hibench.streambench.storm.util.*; import com.intel.hibench.streambench.storm.micro.*; import com.intel.hibench.streambench.storm.trident.*; -import com.intel.hibench.streambench.storm.util.BenchLogUtil; -import com.intel.hibench.streambench.storm.util.StormBenchConfig; +import com.intel.hibench.streambench.storm.metrics.Reporter; +import com.intel.hibench.streambench.storm.spout.*; public class RunBench { - public static void main(String[] args) throws Exception { - runAll(args); - } + public static void main(String[] args) throws Exception { + runAll(args); + } - private static void runAll(String[] args) throws Exception { + public static void runAll(String[] args) throws Exception { - if (args.length < 2) - BenchLogUtil.handleError("Usage: RunBench "); + if (args.length < 2) + BenchLogUtil.handleError("Usage: RunBench "); - StormBenchConfig conf = new StormBenchConfig(); + StormBenchConfig conf = new StormBenchConfig(); - ConfigLoader cl = new ConfigLoader(args[0]); - boolean TridentFramework = false; - if (args[1].equals("trident")) TridentFramework = true; + ConfigLoader cl = new ConfigLoader(args[0]); + boolean TridentFramework = false; + if (args[1].equals("trident")) TridentFramework = true; - conf.zkHost = cl.getProperty(StreamBenchConfig.ZK_HOST); - conf.workerCount = Integer.parseInt(cl.getProperty(StreamBenchConfig.STORM_WORKERCOUNT)); - conf.spoutThreads = Integer.parseInt(cl.getProperty(StreamBenchConfig.STORM_SPOUT_THREADS)); - conf.boltThreads = Integer.parseInt(cl.getProperty(StreamBenchConfig.STORM_BOLT_THREADS)); - conf.benchName = cl.getProperty(StreamBenchConfig.TESTCASE); - conf.topic = cl.getProperty(StreamBenchConfig.KAFKA_TOPIC); - conf.consumerGroup = cl.getProperty(StreamBenchConfig.CONSUMER_GROUP); - conf.ackon = Boolean.parseBoolean(cl.getProperty(StreamBenchConfig.STORM_ACKON)); - conf.localShuffle = Boolean.parseBoolean(cl.getProperty(StreamBenchConfig.STORM_LOCAL_SHUFFLE)); + conf.nimbus = cl.getProperty("hibench.streamingbench.storm.nimbus"); + conf.nimbusAPIPort = Integer.parseInt(cl.getProperty("hibench.streamingbench.storm.nimbusAPIPort")); + conf.zkHost = cl.getProperty("hibench.streamingbench.zookeeper.host"); + conf.workerCount = Integer.parseInt(cl.getProperty("hibench.streamingbench.storm.worker_count")); + conf.spoutThreads = Integer.parseInt(cl.getProperty("hibench.streamingbench.storm.spout_threads")); + conf.boltThreads = Integer.parseInt(cl.getProperty("hibench.streamingbench.storm.bolt_threads")); + conf.benchName = cl.getProperty("hibench.streamingbench.benchname"); + conf.recordCount = Long.parseLong(cl.getProperty("hibench.streamingbench.record_count")); + conf.topic = cl.getProperty("hibench.streamingbench.topic_name"); + conf.consumerGroup = cl.getProperty("hibench.streamingbench.consumer_group"); + conf.readFromStart = Boolean.parseBoolean(cl.getProperty("hibench.streamingbench.storm.read_from_start")); + conf.ackon = Boolean.parseBoolean(cl.getProperty("hibench.streamingbench.storm.ackon")); + conf.nimbusContactInterval = Integer.parseInt(cl.getProperty("hibench.streamingbench.storm.nimbusContactInterval")); - conf.windowDuration = Long.parseLong(cl.getProperty(StreamBenchConfig.FixWINDOW_DURATION)); - conf.windowSlideStep = Long.parseLong(cl.getProperty(StreamBenchConfig.FixWINDOW_SLIDESTEP)); + boolean isLocal = false; - conf.brokerList = cl.getProperty(StreamBenchConfig.KAFKA_BROKER_LIST); - int producerNum = Integer.parseInt(cl.getProperty(StreamBenchConfig.DATAGEN_PRODUCER_NUMBER)); - long recordPerInterval = Long.parseLong(cl.getProperty(StreamBenchConfig.DATAGEN_RECORDS_PRE_INTERVAL)); - int intervalSpan = Integer.parseInt(cl.getProperty(StreamBenchConfig.DATAGEN_INTERVAL_SPAN)); - if (TridentFramework) { - conf.reporterTopic = MetricsUtil.getTopic(Platform.TRIDENT, - conf.topic, producerNum, recordPerInterval, intervalSpan); - } else { - conf.reporterTopic = MetricsUtil.getTopic(Platform.STORM, - conf.topic, producerNum, recordPerInterval, intervalSpan); - } - int reportTopicPartitions = Integer.parseInt(cl.getProperty(StreamBenchConfig.KAFKA_TOPIC_PARTITIONS)); - MetricsUtil.createTopic(conf.zkHost, conf.reporterTopic, reportTopicPartitions); - TestCase benchName = TestCase.withValue(conf.benchName); + ConstructSpoutUtil.setConfig(conf); + + String benchName = conf.benchName; - BenchLogUtil.logMsg("Benchmark starts... " + " " + benchName + - " Frameworks:" + (TridentFramework ? "Trident" : "Storm")); + BenchLogUtil.logMsg("Benchmark starts... local:" + isLocal + " " + benchName + + " Frameworks:" + (TridentFramework?"Trident":"Storm") ); - if (TridentFramework) { // For trident workloads - if (benchName.equals(TestCase.WORDCOUNT)) { - TridentWordcount wordcount = new TridentWordcount(conf); - wordcount.run(); - } else if (benchName.equals(TestCase.IDENTITY)) { - TridentIdentity identity = new TridentIdentity(conf); - identity.run(); - } else if (benchName.equals(TestCase.REPARTITION)) { - TridentRepartition repartition = new TridentRepartition(conf); - repartition.run(); - } else if (benchName.equals(TestCase.FIXWINDOW)) { - TridentWindow window = new TridentWindow(conf); - window.run(); - } - } else { // For storm workloads - if (benchName.equals(TestCase.IDENTITY)) { - Identity identity = new Identity(conf); - identity.run(); - } else if (benchName.equals(TestCase.WORDCOUNT)) { - WordCount wordCount = new WordCount(conf); - wordCount.run(); - } else if (benchName.equals(TestCase.FIXWINDOW)) { - WindowedCount window = new WindowedCount(conf); - window.run(); - } + if (TridentFramework) { // For trident workloads + if (benchName.equals("wordcount")) { + conf.separator = cl.getProperty("hibench.streamingbench.separator"); + TridentWordcount wordcount = new TridentWordcount(conf); + wordcount.run(); + } else if (benchName.equals("identity")) { + TridentIdentity identity = new TridentIdentity(conf); + identity.run(); + } else if (benchName.equals("sample")) { + conf.prob = Double.parseDouble(cl.getProperty("hibench.streamingbench.prob")); + TridentSample sample = new TridentSample(conf); + sample.run(); + } else if (benchName.equals("project")) { + conf.separator = cl.getProperty("hibench.streamingbench.separator"); + conf.fieldIndex = Integer.parseInt(cl.getProperty("hibench.streamingbench.field_index")); + TridentProject project = new TridentProject(conf); + project.run(); + } else if (benchName.equals("grep")) { + conf.pattern = cl.getProperty("hibench.streamingbench.pattern"); + TridentGrep grep = new TridentGrep(conf); + grep.run(); + } else if (benchName.equals("distinctcount")) { + conf.separator = cl.getProperty("hibench.streamingbench.separator"); + conf.fieldIndex = Integer.parseInt(cl.getProperty("hibench.streamingbench.field_index")); + TridentDistinctCount distinct = new TridentDistinctCount(conf); + distinct.run(); + } else if (benchName.equals("statistics")) { + conf.separator = cl.getProperty("hibench.streamingbench.separator"); + conf.fieldIndex = Integer.parseInt(cl.getProperty("hibench.streamingbench.field_index")); + TridentNumericCalc numeric = new TridentNumericCalc(conf); + numeric.run(); + } + } else { // For storm workloads + if (benchName.equals("identity")) { + Identity identity = new Identity(conf); + identity.run(); + } else if (benchName.equals("project")) { + conf.separator = cl.getProperty("hibench.streamingbench.separator"); + conf.fieldIndex = Integer.parseInt(cl.getProperty("hibench.streamingbench.field_index")); + ProjectStream project = new ProjectStream(conf); + project.run(); + } else if (benchName.equals("sample")) { + conf.prob = Double.parseDouble(cl.getProperty("hibench.streamingbench.prob")); + SampleStream sample = new SampleStream(conf); + sample.run(); + } else if (benchName.equals("wordcount")) { + conf.separator = cl.getProperty("hibench.streamingbench.separator"); + Wordcount wordcount = new Wordcount(conf); + wordcount.run(); + } else if (benchName.equals("grep")) { + conf.pattern = cl.getProperty("hibench.streamingbench.pattern"); + GrepStream grep = new GrepStream(conf); + grep.run(); + } else if (benchName.equals("statistics")) { + conf.separator = cl.getProperty("hibench.streamingbench.separator"); + conf.fieldIndex = Integer.parseInt(cl.getProperty("hibench.streamingbench.field_index")); + NumericCalc numeric = new NumericCalc(conf); + numeric.run(); + } else if (benchName.equals("distinctcount")) { + conf.separator = cl.getProperty("hibench.streamingbench.separator"); + conf.fieldIndex = Integer.parseInt(cl.getProperty("hibench.streamingbench.field_index")); + DistinctCount distinct = new DistinctCount(conf); + distinct.run(); + } else if (benchName.equals("statistics")) { + conf.separator = cl.getProperty("hibench.streamingbench.separator"); + conf.fieldIndex = Integer.parseInt(cl.getProperty("hibench.streamingbench.field_index")); + NumericCalcSep numeric = new NumericCalcSep(conf); + numeric.run(); + } + } + //Collect metrics data + Thread metricCollector = new Thread(new Reporter(conf.nimbus, conf.nimbusAPIPort, conf.benchName, conf.recordCount, conf.nimbusContactInterval)); + metricCollector.start(); } - } } diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/metrics/Reporter.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/metrics/Reporter.java new file mode 100644 index 000000000..491c5cc30 --- /dev/null +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/metrics/Reporter.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.intel.hibench.streambench.storm.metrics; + +import com.intel.hibench.streambench.storm.util.*; + +public class Reporter implements Runnable{ + private String nimbus; + private int port; + private String benchName; + private long recordCount; + private int interval; + + public Reporter(String nimbus,int port,String benchName,long recordCount,int interval){ + this.nimbus=nimbus; + this.port=port; + this.benchName=benchName; + this.recordCount=recordCount; + this.interval=interval; + } + + public void run(){ + StatFacade statsUtil=new StatFacade(nimbus,port,benchName,interval); + + //Get throughput + double runtime=statsUtil.getRunTimeInSec(); + if(runtime==0){ + BenchLogUtil.handleError("Runtime is less than collect time!"); + } + double throughput=(double)recordCount/runtime; + System.out.println("Runtime is: "+runtime+" throughput is: "+throughput); + //Get latency. Currently the metric is from Storm UI + } + + //public static void main(String[] args){ + //run(new String[]{"sr119","6627","microbench_sketch","1000000"}); + //} + + +} diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/metrics/StatFacade.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/metrics/StatFacade.java new file mode 100644 index 000000000..ff87895df --- /dev/null +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/metrics/StatFacade.java @@ -0,0 +1,202 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.intel.hibench.streambench.storm.metrics; + +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.Timer; +import java.util.TimerTask; + +import org.apache.thrift.TException; +import org.apache.thrift.protocol.TBinaryProtocol; +import org.apache.thrift.transport.TFramedTransport; +import org.apache.thrift.transport.TSocket; +import org.apache.thrift.transport.TTransportException; + +import com.intel.hibench.streambench.storm.thrift.generated.ClusterSummary; +import com.intel.hibench.streambench.storm.thrift.generated.ExecutorSummary; +import com.intel.hibench.streambench.storm.thrift.generated.Nimbus; +import com.intel.hibench.streambench.storm.thrift.generated.NotAliveException; +import com.intel.hibench.streambench.storm.thrift.generated.TopologyInfo; +import com.intel.hibench.streambench.storm.thrift.generated.GlobalStreamId; +import com.intel.hibench.streambench.storm.thrift.generated.TopologySummary; + +public class StatFacade { + Nimbus.Client client; + boolean finished=false; + String topologyId; + long finishedTime=0; + Object mutex=new Object(); + long startTime=0; + + int COLLECT_INTERVAL=3000; //3s + + public StatFacade(String host,int port,String topologyName,int interval){ + COLLECT_INTERVAL=interval*1000; + TSocket tsocket = new TSocket(host, port); + TFramedTransport tTransport = new TFramedTransport(tsocket); + TBinaryProtocol tBinaryProtocol = new TBinaryProtocol(tTransport); + client = new Nimbus.Client(tBinaryProtocol); + try { + tTransport.open(); + topologyId=getTopologyId(topologyName); + } catch (TTransportException e) { + e.printStackTrace(); + } + } + + private String getTopologyId(String topoName){ + ClusterSummary clusterSummary; + try { + clusterSummary = client.getClusterInfo(); + List summary=clusterSummary.getTopologies(); + for(TopologySummary topo:summary){ + if(topo.getName().equals(topoName)) + return topo.getId(); + } + } catch (TException e) { + e.printStackTrace(); + } + return null; + } + + //The logic: if the processed tuples count remains same in 5 seconds, then the bench is finished + private void waitToFinish(){ + Timer timer=new Timer(); + timer.schedule(new CheckTask(timer), 0 ,COLLECT_INTERVAL); + try { + synchronized(mutex){ + mutex.wait(); + } + } catch (InterruptedException e) { + e.printStackTrace(); + } + finished=true; + } + + private long prevTransfer=0; + class CheckTask extends TimerTask{ + Timer timer; + public CheckTask(Timer timer){ + this.timer=timer; + } + + public void run(){ + long curTransfer=getAckedSize(); + + System.out.println("Prev transfer: "+prevTransfer+" curTransfer: "+curTransfer); + //If no data transfered during the period, then the benchmark finishes + if(curTransfer==prevTransfer && curTransfer!=0){ + synchronized(mutex){ + mutex.notify(); + } + + finishedTime=System.currentTimeMillis()-COLLECT_INTERVAL-COLLECT_INTERVAL/2; //Last interval has ended + timer.cancel(); + timer.purge(); + }else if(prevTransfer==0&&curTransfer!=0){ //This marks the beginning + prevTransfer=curTransfer; + startTime=System.currentTimeMillis()-COLLECT_INTERVAL/2; //An average modify of the start + }else{ + prevTransfer=curTransfer; + } + } + } + + private long getTransferSize(){ + try { + TopologyInfo topologyInfo = client.getTopologyInfo(topologyId); + long transfferedAll=0; + for(ExecutorSummary executorInfo:topologyInfo.executors){ + if(executorInfo==null || executorInfo.stats==null) return 0; + Map> sent=executorInfo.stats.transferred; + if(sent==null) return 0; + Set> transferSet10min=sent.get(":all-time").entrySet(); + for(Map.Entry subEntry:transferSet10min){ + transfferedAll+=subEntry.getValue(); + } + } + return transfferedAll; + } catch (NotAliveException e) { + e.printStackTrace(); + } catch (TException e) { + e.printStackTrace(); + } + return 0; + } + + private long getAckedSize(){ + try { + TopologyInfo topologyInfo = client.getTopologyInfo(topologyId); + long ackTotal=0; + for(ExecutorSummary executorInfo:topologyInfo.executors){ + if(executorInfo==null || executorInfo.stats==null) return 0; + String id = executorInfo.component_id; + if(id.equals("spout") || id.equals("$mastercoord-bg0") || id.equals("$spoutcoord-spout0") || id.equals("spout0") || id.equals("__acker")) + continue; + if(executorInfo.stats.specific == null) return 0; + if(executorInfo.stats.specific.getBolt() == null) return 0; + Map> acked = executorInfo.stats.specific.getBolt().acked; + Map map = acked.get(":all-time"); + if(map == null) return 0; + Set> allTimeAck = map.entrySet(); + for(Map.Entry subEntry:allTimeAck){ + if (id.contains("b-")) { + String ikey = subEntry.getKey().toString(); + if (!ikey.contains("streamId:s")) { + continue; + } + } + ackTotal += subEntry.getValue(); + } + } + return ackTotal; + } catch (NotAliveException e) { + e.printStackTrace(); + } catch (TException e) { + e.printStackTrace(); + } + return 0; + } + + private int getTopologyUpTime(){ + try { + TopologyInfo topologyInfo = client.getTopologyInfo(topologyId); + return topologyInfo.getUptime_secs(); + } catch (NotAliveException e) { + e.printStackTrace(); + } catch (TException e) { + e.printStackTrace(); + } + return 0; + } + + public void checkFinished(){ + if(!finished) + waitToFinish(); + } + + //This shall be called first to ensure the benchmark finishes + public double getRunTimeInSec(){ + checkFinished(); + double time=(double)(finishedTime-startTime)/(double)1000; + //If data is consumed in one interval, in average we estimate the consumption time to be half of the interval + return time>0?time:((double)COLLECT_INTERVAL)/((double)2000); + } +} diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/micro/DistinctCount.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/micro/DistinctCount.java new file mode 100644 index 000000000..c963ee483 --- /dev/null +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/micro/DistinctCount.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.intel.hibench.streambench.storm.micro; + +import backtype.storm.topology.base.*; +import backtype.storm.topology.*; +import backtype.storm.tuple.*; + +import com.intel.hibench.streambench.storm.util.*; +import com.intel.hibench.streambench.storm.topologies.*; + +import java.util.HashSet; +import java.util.Set; + +public class DistinctCount extends SingleSpoutTops{ + + public DistinctCount(StormBenchConfig config){ + super(config); + } + + public void setBolt(TopologyBuilder builder){ + builder.setBolt("sketch",new ProjectStreamBolt(config.fieldIndex,config.separator),config.boltThreads).shuffleGrouping("spout"); + builder.setBolt("distinct",new TotalDistinctCountBolt(),config.boltThreads).fieldsGrouping("sketch", new Fields("field")); + } + + public static class TotalDistinctCountBolt extends BaseBasicBolt { + Set set = new HashSet(); + + @Override + public void execute(Tuple tuple, BasicOutputCollector collector){ + String word = tuple.getString(0); //FIXME: always pick up index 0? should be configurable according to sparkstream's version + set.add(word); + BenchLogUtil.logMsg("Distinct count:"+set.size()); + collector.emit(new Values(set.size())); + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields("size")); + } + } + + + +} diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/micro/GrepStream.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/micro/GrepStream.java new file mode 100644 index 000000000..5054d6163 --- /dev/null +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/micro/GrepStream.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.intel.hibench.streambench.storm.micro; + +import backtype.storm.topology.base.*; +import backtype.storm.topology.*; +import backtype.storm.tuple.*; +import storm.kafka.*; + +import com.intel.hibench.streambench.storm.*; +import com.intel.hibench.streambench.storm.util.*; +import com.intel.hibench.streambench.storm.topologies.*; + +import java.util.HashMap; +import java.util.Map; + +public class GrepStream extends SingleSpoutTops{ + + public GrepStream(StormBenchConfig config) { + super(config); + } + + public void setBolt(TopologyBuilder builder){ + builder.setBolt("grepAndPrint",new GrepBolt(config.pattern),config.boltThreads).shuffleGrouping("spout"); + } + + public static class GrepBolt extends BaseBasicBolt{ + private String pattern; + + public GrepBolt(String p){ + pattern=p; + } + + public void execute(Tuple tuple, BasicOutputCollector collector) { + String val=tuple.getString(0); + if(val.contains(pattern)){ + collector.emit(new Values(val)); + //BenchLogUtil.logMsg("Matched:"+val); + } + } + + public void declareOutputFields(OutputFieldsDeclarer ofd) { + } + } + +} diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/micro/Identity.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/micro/Identity.java index a4ba145c6..ec83cc653 100644 --- a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/micro/Identity.java +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/micro/Identity.java @@ -17,52 +17,29 @@ package com.intel.hibench.streambench.storm.micro; -import com.google.common.collect.ImmutableMap; -import com.intel.hibench.streambench.common.metrics.KafkaReporter; -import com.intel.hibench.streambench.common.metrics.LatencyReporter; -import com.intel.hibench.streambench.storm.topologies.SingleSpoutTops; -import com.intel.hibench.streambench.storm.util.StormBenchConfig; -import org.apache.storm.topology.BasicOutputCollector; -import org.apache.storm.topology.BoltDeclarer; -import org.apache.storm.topology.OutputFieldsDeclarer; -import org.apache.storm.topology.TopologyBuilder; -import org.apache.storm.topology.base.BaseBasicBolt; -import org.apache.storm.tuple.Fields; -import org.apache.storm.tuple.Tuple; -import org.apache.storm.tuple.Values; +import backtype.storm.topology.base.*; +import backtype.storm.topology.*; +import backtype.storm.tuple.*; +import com.intel.hibench.streambench.storm.util.*; +import com.intel.hibench.streambench.storm.topologies.*; -public class Identity extends SingleSpoutTops { - public Identity(StormBenchConfig config) { +public class Identity extends SingleSpoutTops{ + + public Identity(StormBenchConfig config){ super(config); } - - @Override - public void setBolts(TopologyBuilder builder) { - BoltDeclarer boltDeclarer = builder.setBolt("identity", new IdentityBolt(config), - config.boltThreads); - if (config.localShuffle) { - boltDeclarer.localOrShuffleGrouping("spout"); - } else { - boltDeclarer.shuffleGrouping("spout"); - } + + public void setBolt(TopologyBuilder builder){ + builder.setBolt("identity",new IdentityBolt(), config.boltThreads).shuffleGrouping("spout"); } - - private static class IdentityBolt extends BaseBasicBolt { - - private final StormBenchConfig config; - - IdentityBolt(StormBenchConfig config) { - this.config = config; - } + + public static class IdentityBolt extends BaseBasicBolt { @Override - public void execute(Tuple tuple, BasicOutputCollector collector) { - final LatencyReporter latencyReporter = new KafkaReporter(config.reporterTopic, config.brokerList); - ImmutableMap kv = (ImmutableMap) tuple.getValue(0); - collector.emit(new Values(kv)); - latencyReporter.report(Long.parseLong(kv.keySet().iterator().next()), System.currentTimeMillis()); + public void execute(Tuple tuple, BasicOutputCollector collector){ + collector.emit(new Values(tuple.getValues())); } @Override diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/micro/NumericCalc.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/micro/NumericCalc.java new file mode 100644 index 000000000..5ba2fc7e8 --- /dev/null +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/micro/NumericCalc.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 com.intel.hibench.streambench.storm.micro; + +import backtype.storm.topology.base.*; +import backtype.storm.topology.*; +import backtype.storm.tuple.*; + +import com.intel.hibench.streambench.storm.util.*; +import com.intel.hibench.streambench.storm.topologies.*; + +public class NumericCalc extends SingleSpoutTops{ + + public NumericCalc(StormBenchConfig config){ + super(config); + } + + public void setBolt(TopologyBuilder builder){ + builder.setBolt("precalc",new NumericBolt(config.separator,config.fieldIndex),Math.max(1,config.boltThreads-1)).shuffleGrouping("spout"); + builder.setBolt("calc",new NumericTogetherBolt(),1).globalGrouping("precalc"); + } + + public static class NumericBolt extends BaseBasicBolt{ + private int fieldIndexInner; + private String separatorInner; + private long max=0; + private long min=Long.MAX_VALUE; + + public NumericBolt(String separator,int fieldIndex){ + fieldIndexInner=fieldIndex; + separatorInner=separator; + } + + public void execute(Tuple tuple, BasicOutputCollector collector) { + String record=tuple.getString(0); + String[] fields=record.trim().split(separatorInner); + if(fields.length>fieldIndexInner){ + long val=Long.parseLong(fields[fieldIndexInner]); + if(val>max) max=val; + if(valmax) max=curMax; + if(curMinfieldIndexInner){ + long val=Long.parseLong(fields[fieldIndexInner]); + if(val>max) max=val; + if(val readAndParseConfig(FlinkBenchConfig config) throws IllegalAccessException{ - Map map = new HashMap(); - Field[] fields = config.getClass().getFields(); - for (Field f : fields) { - map.put(f.getName(), f.get(config).toString()); - } - return map; +import com.intel.hibench.streambench.storm.util.*; +import com.intel.hibench.streambench.storm.topologies.*; + + +public class ProjectStream extends SingleSpoutTops{ + + public ProjectStream(StormBenchConfig config){ + super(config); + } + + public void setBolt(TopologyBuilder builder){ + builder.setBolt("sketchAndPrint",new ProjectStreamBolt(config.fieldIndex,config.separator),config.boltThreads).shuffleGrouping("spout"); } + + } diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/micro/ProjectStreamBolt.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/micro/ProjectStreamBolt.java new file mode 100644 index 000000000..fe2a2a779 --- /dev/null +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/micro/ProjectStreamBolt.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.intel.hibench.streambench.storm.micro; + +import backtype.storm.topology.BasicOutputCollector; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.topology.base.BaseBasicBolt; +import backtype.storm.tuple.Tuple; +import backtype.storm.tuple.Values; + +import backtype.storm.tuple.Fields; + +public class ProjectStreamBolt extends BaseBasicBolt{ + private int fieldIndex; + private String separator; + + public ProjectStreamBolt(int fieldIndex, String separator){ + this.fieldIndex=fieldIndex; + this.separator=separator; + } + + public void execute(Tuple tuple, BasicOutputCollector collector) { + String record=tuple.getString(0); + String[] fields=record.split(separator); + if(fields.length>fieldIndex){ + //BenchLogUtil.logMsg(fields[fieldIndex]); + collector.emit(new Values(fields[fieldIndex])); + } + + } + + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields("field")); + } +} diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/micro/SampleStream.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/micro/SampleStream.java new file mode 100644 index 000000000..63827a528 --- /dev/null +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/micro/SampleStream.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.intel.hibench.streambench.storm.micro; + +import backtype.storm.topology.base.*; +import backtype.storm.topology.*; +import backtype.storm.tuple.*; + +import com.intel.hibench.streambench.storm.util.*; +import com.intel.hibench.streambench.storm.topologies.*; + +import java.util.Random; + +public class SampleStream extends SingleSpoutTops{ + + public SampleStream(StormBenchConfig config){ + super(config); + } + + public void setBolt(TopologyBuilder builder){ + builder.setBolt("sampleAndPrint",new SampleBolt(config.prob),config.boltThreads).shuffleGrouping("spout"); + } + + public static class SampleBolt extends BaseBasicBolt{ + + private double probability; + private int count=0; + private ThreadLocal rand = null; + + public SampleBolt(double prob){ + probability=prob; + rand = threadRandom(1); + } + + public void execute(Tuple tuple, BasicOutputCollector collector) { + double randVal = rand.get().nextDouble(); + if(randVal<=probability){ + count+=1; + collector.emit(new Values(tuple.getString(0))); + BenchLogUtil.logMsg(" count:"+count); + } + } + + public void declareOutputFields(OutputFieldsDeclarer ofd){ + } + } + + public static ThreadLocal threadRandom(final long seed) { + return new ThreadLocal(){ + @Override + protected Random initialValue() { + return new Random(seed); + } + }; + } + +} diff --git a/src/streambench/common/src/main/java/com/intel/hibench/common/HiBenchConfig.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/micro/SplitStreamBolt.java similarity index 50% rename from src/streambench/common/src/main/java/com/intel/hibench/common/HiBenchConfig.java rename to src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/micro/SplitStreamBolt.java index 2c1521c79..8e9dac626 100644 --- a/src/streambench/common/src/main/java/com/intel/hibench/common/HiBenchConfig.java +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/micro/SplitStreamBolt.java @@ -15,29 +15,32 @@ * limitations under the License. */ +package com.intel.hibench.streambench.storm.micro; -package com.intel.hibench.common; +import backtype.storm.topology.BasicOutputCollector; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.topology.base.BaseBasicBolt; +import backtype.storm.tuple.Tuple; -/** - * common configurations used in HiBench project are defined here. Later we plan to move this file - * to higher project. - */ -public class HiBenchConfig { - - // ===================================== - // Spark Related Conf - // ===================================== - public static String SPARK_MASTER = "hibench.spark.master"; - - // ===================================== - // DFS Related Conf - // ===================================== - public static String DFS_MASTER = "hibench.hdfs.master"; - - // ===================================== - // YARN Related Conf - // ===================================== - public static String YARN_EXECUTOR_NUMBER = "hibench.yarn.executor.num"; +import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Values; - public static String YARN_EXECUTOR_CORES = "hibench.yarn.executor.cores"; +public class SplitStreamBolt extends BaseBasicBolt{ + private String separator; + + public SplitStreamBolt(String separator){ + this.separator=separator; + } + + public void execute(Tuple tuple, BasicOutputCollector collector) { + String record=tuple.getString(0); + String[] fields=record.split(separator); + for(String s:fields){ + collector.emit(new Values(s)); + } + } + + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields("word")); + } } diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/micro/WindowedCount.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/micro/WindowedCount.java deleted file mode 100644 index 7dbe457b9..000000000 --- a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/micro/WindowedCount.java +++ /dev/null @@ -1,112 +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.intel.hibench.streambench.storm.micro; - -import com.google.common.collect.ImmutableMap; -import com.intel.hibench.streambench.common.UserVisit; -import com.intel.hibench.streambench.common.UserVisitParser; -import com.intel.hibench.streambench.common.metrics.KafkaReporter; -import com.intel.hibench.streambench.common.metrics.LatencyReporter; -import com.intel.hibench.streambench.storm.topologies.SingleSpoutTops; -import com.intel.hibench.streambench.storm.util.StormBenchConfig; -import org.apache.storm.topology.BasicOutputCollector; -import org.apache.storm.topology.BoltDeclarer; -import org.apache.storm.topology.OutputFieldsDeclarer; -import org.apache.storm.topology.base.BaseBasicBolt; -import org.apache.storm.tuple.Fields; -import org.apache.storm.tuple.Tuple; -import org.apache.storm.topology.TopologyBuilder; -import org.apache.storm.topology.base.BaseWindowedBolt; -import org.apache.storm.topology.base.BaseWindowedBolt.Duration; -import org.apache.storm.tuple.Values; -import org.apache.storm.windowing.TupleWindow; - -import java.util.HashMap; -import java.util.Map; -import java.util.concurrent.TimeUnit; - -public class WindowedCount extends SingleSpoutTops{ - - public WindowedCount(StormBenchConfig config) { - super(config); - } - - @Override - public void setBolts(TopologyBuilder builder) { - Duration windowDuration = new Duration((int)config.windowDuration, TimeUnit.MILLISECONDS); - Duration windowSlide = new Duration((int)config.windowSlideStep, TimeUnit.MILLISECONDS); - BoltDeclarer boltDeclarer = builder.setBolt("parser", new ParserBolt(), config.boltThreads); - if (config.localShuffle) { - boltDeclarer.localOrShuffleGrouping("spout"); - } else { - boltDeclarer.shuffleGrouping("spout"); - } - builder.setBolt("window", new SlidingWindowBolt(config) - .withWindow(windowDuration, windowSlide), - config.boltThreads).fieldsGrouping("parser", new Fields("ip")); - } - - private static class ParserBolt extends BaseBasicBolt { - - @Override - public void execute(Tuple tuple, BasicOutputCollector basicOutputCollector) { - ImmutableMap kv = (ImmutableMap) tuple.getValue(0); - String time = kv.keySet().iterator().next(); - UserVisit uv = UserVisitParser.parse(kv.get(time)); - basicOutputCollector.emit(new Values(uv.getIp(), Long.parseLong(time))); - } - - @Override - public void declareOutputFields(OutputFieldsDeclarer outputFieldsDeclarer) { - outputFieldsDeclarer.declare(new Fields("ip", "time")); - } - } - - private static class SlidingWindowBolt extends BaseWindowedBolt { - private final StormBenchConfig config; - - SlidingWindowBolt(StormBenchConfig config) { - this.config = config; - } - - @Override - public void execute(TupleWindow inputWindow) { - Map counts = new HashMap(); - for(Tuple tuple: inputWindow.get()) { - Long time = tuple.getLong(1); - String ip = tuple.getString(0); - Long[] timeAndCount = counts.get(ip); - if (null == timeAndCount) { - timeAndCount = new Long[2]; - timeAndCount[0] = time; - timeAndCount[1] = 0L; - } - timeAndCount[0] = Math.min(timeAndCount[0], time); - timeAndCount[1]++; - counts.put(ip, timeAndCount); - } - LatencyReporter latencyReporter = new KafkaReporter(config.reporterTopic, config.brokerList); - for (Long[] timeAndCount: counts.values()) { - for (int i = 0; i < timeAndCount[1]; i++) { - latencyReporter.report(timeAndCount[0], System.currentTimeMillis()); - } - } - } - - } -} diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/micro/WordCount.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/micro/WordCount.java deleted file mode 100644 index 5c17ed5e7..000000000 --- a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/micro/WordCount.java +++ /dev/null @@ -1,100 +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.intel.hibench.streambench.storm.micro; - -import com.google.common.collect.ImmutableMap; -import com.intel.hibench.streambench.common.UserVisit; -import com.intel.hibench.streambench.common.UserVisitParser; -import com.intel.hibench.streambench.common.metrics.KafkaReporter; -import com.intel.hibench.streambench.common.metrics.LatencyReporter; -import com.intel.hibench.streambench.storm.topologies.SingleSpoutTops; -import com.intel.hibench.streambench.storm.util.StormBenchConfig; -import org.apache.storm.topology.BasicOutputCollector; -import org.apache.storm.topology.BoltDeclarer; -import org.apache.storm.topology.OutputFieldsDeclarer; -import org.apache.storm.topology.TopologyBuilder; -import org.apache.storm.topology.base.BaseBasicBolt; -import org.apache.storm.tuple.Fields; -import org.apache.storm.tuple.Tuple; -import org.apache.storm.tuple.Values; - -import java.util.HashMap; -import java.util.Map; - -public class WordCount extends SingleSpoutTops { - - public WordCount(StormBenchConfig config) { - super(config); - } - - @Override - public void setBolts(TopologyBuilder builder) { - BoltDeclarer boltDeclarer = builder.setBolt("split", new SplitStreamBolt(), - config.boltThreads); - if (config.localShuffle) { - boltDeclarer.localOrShuffleGrouping("spout"); - } else { - boltDeclarer.shuffleGrouping("spout"); - } - builder.setBolt("count", new WordCountBolt(config), - config.boltThreads).fieldsGrouping("split", new Fields("ip")); - } - - private static class WordCountBolt extends BaseBasicBolt { - Map counts = new HashMap(); - private final StormBenchConfig config; - - WordCountBolt(StormBenchConfig config) { - this.config = config; - } - - @Override - public void execute(Tuple tuple, BasicOutputCollector collector) { - String word = tuple.getString(0); - Integer count = counts.get(word); - if (count == null) - count = 0; - count++; - counts.put(word, count); - - LatencyReporter latencyReporter = new KafkaReporter(config.reporterTopic, config.brokerList); - latencyReporter.report(tuple.getLong(1), System.currentTimeMillis()); - } - - @Override - public void declareOutputFields(OutputFieldsDeclarer declarer) { - declarer.declare(new Fields("ip", "count")); - } - } - - private static class SplitStreamBolt extends BaseBasicBolt { - - public void execute(Tuple tuple, BasicOutputCollector collector) { - ImmutableMap kv = (ImmutableMap) tuple.getValue(0); - String key = kv.keySet().iterator().next(); - Long startTime = Long.parseLong(key); - UserVisit uv = UserVisitParser.parse(kv.get(key)); - collector.emit(new Values(uv.getIp(), startTime)); - } - - public void declareOutputFields(OutputFieldsDeclarer declarer) { - declarer.declare(new Fields("ip", "time")); - } - } - -} diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/micro/Wordcount.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/micro/Wordcount.java new file mode 100644 index 000000000..8954af3b2 --- /dev/null +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/micro/Wordcount.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.intel.hibench.streambench.storm.micro; + +import backtype.storm.topology.base.*; +import backtype.storm.topology.*; +import backtype.storm.tuple.*; + +import com.intel.hibench.streambench.storm.util.*; +import com.intel.hibench.streambench.storm.topologies.*; + +import java.util.HashMap; +import java.util.Map; + +public class Wordcount extends SingleSpoutTops{ + + public Wordcount(StormBenchConfig config){ + super(config); + } + + public void setBolt(TopologyBuilder builder){ + builder.setBolt("split",new SplitStreamBolt(config.separator),config.boltThreads).shuffleGrouping("spout"); + builder.setBolt("count",new WordCountBolt(), config.boltThreads).fieldsGrouping("split", new Fields("word")); + } + + public static class WordCountBolt extends BaseBasicBolt { + Map counts = new HashMap(); + + @Override + public void execute(Tuple tuple, BasicOutputCollector collector){ + String word = tuple.getString(0); + Integer count = counts.get(word); + if (count == null) + count = 0; + count++; + counts.put(word, count); + //BenchLogUtil.logMsg("Word:"+word+" count:"+count); + collector.emit(new Values(word, count)); + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields("word", "count")); + } + } + +} diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/spout/ConstructSpoutUtil.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/spout/ConstructSpoutUtil.java new file mode 100644 index 000000000..52d8316f3 --- /dev/null +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/spout/ConstructSpoutUtil.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 com.intel.hibench.streambench.storm.spout; + +import backtype.storm.topology.base.BaseRichSpout; +import com.intel.hibench.streambench.storm.util.*; +import java.io.*; +import java.util.*; +import storm.kafka.trident.*; + +public class ConstructSpoutUtil{ + + private static StormBenchConfig conf; + private static boolean isLocal=false; + + public static BaseRichSpout constructSpout(){ + BaseRichSpout spout=null; + if(isLocal){ + //if(args.length<2) + //BenchLogUtil.handleError("RunBench Local Usage: "); + //String dataFile=prop.getProperty("datafile"); + //spout=LocalSpoutFactory.getSpout(dataFile,recordCount); + spout=null; + }else{ + String topic=conf.topic; + String consumerGroup=conf.consumerGroup; + boolean readFromStart=conf.readFromStart; + String zkHost=conf.zkHost; + BenchLogUtil.logMsg("Topic:"+topic+" consumerGroup:"+consumerGroup+" zkHost:"+zkHost); + spout=KafkaSpoutFactory.getSpout(zkHost,topic,consumerGroup,readFromStart); + } + return spout; + } + + + public static OpaqueTridentKafkaSpout constructTridentSpout(){ + String topic = conf.topic; + String consumerGroup = conf.consumerGroup; + boolean readFromStart=conf.readFromStart; + String zkHost = conf.zkHost; + BenchLogUtil.logMsg("Topic:"+topic+" consumerGroup:"+consumerGroup+" zkHost:"+zkHost); + OpaqueTridentKafkaSpout spout=KafkaSpoutFactory.getTridentSpout(zkHost,topic,consumerGroup,readFromStart); + return spout; + } + + + public static void setConfig(StormBenchConfig c){ + conf=c; + } + + + + +} diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/spout/KafkaSpoutFactory.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/spout/KafkaSpoutFactory.java index 0548be0e9..70961e67e 100644 --- a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/spout/KafkaSpoutFactory.java +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/spout/KafkaSpoutFactory.java @@ -17,44 +17,34 @@ package com.intel.hibench.streambench.storm.spout; -import com.intel.hibench.streambench.storm.util.StormBenchConfig; -import kafka.api.OffsetRequest; -import org.apache.storm.kafka.*; -import org.apache.storm.kafka.trident.OpaqueTridentKafkaSpout; -import org.apache.storm.kafka.trident.TransactionalTridentKafkaSpout; -import org.apache.storm.kafka.trident.TridentKafkaConfig; -import org.apache.storm.topology.base.BaseRichSpout; -import org.apache.storm.trident.spout.ITridentDataSource; +import backtype.storm.topology.base.BaseRichSpout; +import backtype.storm.Config; +import storm.kafka.*; +import backtype.storm.spout.SchemeAsMultiScheme; +import storm.kafka.trident.*; -public class KafkaSpoutFactory { - - public static BaseRichSpout getSpout(StormBenchConfig conf) { - String topic = conf.topic; - String consumerGroup = conf.consumerGroup; - String zkHost = conf.zkHost; - BrokerHosts brokerHosts = new ZkHosts(zkHost); - SpoutConfig spoutConfig = new SpoutConfig(brokerHosts, topic, "", consumerGroup); - spoutConfig.scheme = new KeyValueSchemeAsMultiScheme(new StringKeyValueScheme()); - spoutConfig.ignoreZkOffsets = true; - spoutConfig.startOffsetTime = OffsetRequest.LatestTime(); - return new KafkaSpout(spoutConfig); +public class KafkaSpoutFactory{ + + public static BaseRichSpout getSpout(String zkHost,String topic,String consumerGroup,boolean readFromStart){ + BrokerHosts brokerHosts=new ZkHosts(zkHost); + SpoutConfig spoutConfig = new SpoutConfig(brokerHosts,topic,"/"+consumerGroup,consumerGroup); + spoutConfig.scheme=new SchemeAsMultiScheme(new StringScheme()); + //spoutConfig.stateUpdateIntervalMs = 1000; + spoutConfig.forceFromStart=readFromStart; + KafkaSpout kafkaSpout = new KafkaSpout(spoutConfig); + return kafkaSpout; } + - - public static ITridentDataSource getTridentSpout(StormBenchConfig conf, boolean opaque) { - String topic = conf.topic; - String consumerGroup = conf.consumerGroup; - String zkHost = conf.zkHost; - BrokerHosts brokerHosts = new ZkHosts(zkHost); - TridentKafkaConfig tridentKafkaConfig = new TridentKafkaConfig(brokerHosts, topic, consumerGroup); - tridentKafkaConfig.scheme = new KeyValueSchemeAsMultiScheme(new StringKeyValueScheme()); - tridentKafkaConfig.ignoreZkOffsets = true; - tridentKafkaConfig.startOffsetTime = OffsetRequest.LatestTime(); - if (opaque) { - return new OpaqueTridentKafkaSpout(tridentKafkaConfig); - } else { - return new TransactionalTridentKafkaSpout(tridentKafkaConfig); - } + public static OpaqueTridentKafkaSpout getTridentSpout(String zkHost,String topic,String consumerGroup,boolean readFromStart){ + BrokerHosts brokerHosts=new ZkHosts(zkHost); + TridentKafkaConfig tridentKafkaConfig = new TridentKafkaConfig(brokerHosts,topic,consumerGroup); + tridentKafkaConfig.fetchSizeBytes = 10*1024; + tridentKafkaConfig.scheme = new SchemeAsMultiScheme(new StringScheme()); + tridentKafkaConfig.forceFromStart = readFromStart; + OpaqueTridentKafkaSpout opaqueTridentKafkaSpout = new OpaqueTridentKafkaSpout(tridentKafkaConfig); + return opaqueTridentKafkaSpout; } - + + } diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/AlreadyAliveException.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/AlreadyAliveException.java new file mode 100644 index 000000000..09088de1c --- /dev/null +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/AlreadyAliveException.java @@ -0,0 +1,387 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package com.intel.hibench.streambench.storm.thrift.generated; + +import org.apache.commons.lang.builder.HashCodeBuilder; +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 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 TException implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + 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()); + } + + public 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.isSetMsg()) { + this.msg = other.msg; + } + } + + public AlreadyAliveException deepCopy() { + return new AlreadyAliveException(this); + } + + @Override + public void clear() { + this.msg = null; + } + + public String getMsg() { + return this.msg; + } + + public AlreadyAliveException setMsg(String msg) { + this.msg = msg; + return this; + } + + public void unsetMsg() { + this.msg = null; + } + + /** Returns true if field msg is set (has been assigned a value) and false otherwise */ + public boolean isSetMsg() { + return this.msg != null; + } + + public void setMsgIsSet(boolean value) { + if (!value) { + this.msg = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case MSG: + if (value == null) { + unsetMsg(); + } else { + setMsg((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case MSG: + return getMsg(); + + } + 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 isSetMsg(); + } + 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.isSetMsg(); + boolean that_present_msg = true && that.isSetMsg(); + 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 && (isSetMsg()); + 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(isSetMsg()).compareTo(typedOther.isSetMsg()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetMsg()) { + lastComparison = org.apache.thrift.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.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 (msg == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'msg' was not present! 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.setMsgIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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.setMsgIsSet(true); + } + } + +} + diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/Bolt.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/Bolt.java new file mode 100644 index 000000000..1285ea08c --- /dev/null +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/Bolt.java @@ -0,0 +1,495 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package com.intel.hibench.streambench.storm.thrift.generated; + +import org.apache.commons.lang.builder.HashCodeBuilder; +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 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.thrift.TBase, java.io.Serializable, Cloneable { + 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()); + } + + public ComponentObject bolt_object; // required + public 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.isSetBolt_object()) { + this.bolt_object = new ComponentObject(other.bolt_object); + } + if (other.isSetCommon()) { + 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 getBolt_object() { + return this.bolt_object; + } + + public Bolt setBolt_object(ComponentObject bolt_object) { + this.bolt_object = bolt_object; + return this; + } + + public void unsetBolt_object() { + this.bolt_object = null; + } + + /** Returns true if field bolt_object is set (has been assigned a value) and false otherwise */ + public boolean isSetBolt_object() { + return this.bolt_object != null; + } + + public void setBolt_objectIsSet(boolean value) { + if (!value) { + this.bolt_object = null; + } + } + + public ComponentCommon getCommon() { + return this.common; + } + + public Bolt setCommon(ComponentCommon common) { + this.common = common; + return this; + } + + public void unsetCommon() { + this.common = null; + } + + /** Returns true if field common is set (has been assigned a value) and false otherwise */ + public boolean isSetCommon() { + return this.common != null; + } + + public void setCommonIsSet(boolean value) { + if (!value) { + this.common = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case BOLT_OBJECT: + if (value == null) { + unsetBolt_object(); + } else { + setBolt_object((ComponentObject)value); + } + break; + + case COMMON: + if (value == null) { + unsetCommon(); + } else { + setCommon((ComponentCommon)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case BOLT_OBJECT: + return getBolt_object(); + + case COMMON: + return getCommon(); + + } + 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 isSetBolt_object(); + case COMMON: + return isSetCommon(); + } + 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.isSetBolt_object(); + boolean that_present_bolt_object = true && that.isSetBolt_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.isSetCommon(); + boolean that_present_common = true && that.isSetCommon(); + 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 && (isSetBolt_object()); + builder.append(present_bolt_object); + if (present_bolt_object) + builder.append(bolt_object); + + boolean present_common = true && (isSetCommon()); + 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(isSetBolt_object()).compareTo(typedOther.isSetBolt_object()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetBolt_object()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.bolt_object, typedOther.bolt_object); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetCommon()).compareTo(typedOther.isSetCommon()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetCommon()) { + lastComparison = org.apache.thrift.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.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 (bolt_object == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'bolt_object' was not present! Struct: " + toString()); + } + if (common == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'common' was not present! 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.setBolt_objectIsSet(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.setCommonIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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.setBolt_objectIsSet(true); + struct.common = new ComponentCommon(); + struct.common.read(iprot); + struct.setCommonIsSet(true); + } + } + +} + diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/BoltStats.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/BoltStats.java new file mode 100644 index 000000000..146d0734d --- /dev/null +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/BoltStats.java @@ -0,0 +1,1371 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package com.intel.hibench.streambench.storm.thrift.generated; + +import org.apache.commons.lang.builder.HashCodeBuilder; +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 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.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("BoltStats"); + + private static final org.apache.thrift.protocol.TField ACKED_FIELD_DESC = new org.apache.thrift.protocol.TField("acked", org.apache.thrift.protocol.TType.MAP, (short)1); + private static final org.apache.thrift.protocol.TField FAILED_FIELD_DESC = new org.apache.thrift.protocol.TField("failed", org.apache.thrift.protocol.TType.MAP, (short)2); + private static final org.apache.thrift.protocol.TField PROCESS_MS_AVG_FIELD_DESC = new org.apache.thrift.protocol.TField("process_ms_avg", org.apache.thrift.protocol.TType.MAP, (short)3); + private static final org.apache.thrift.protocol.TField EXECUTED_FIELD_DESC = new org.apache.thrift.protocol.TField("executed", org.apache.thrift.protocol.TType.MAP, (short)4); + private static final org.apache.thrift.protocol.TField EXECUTE_MS_AVG_FIELD_DESC = new org.apache.thrift.protocol.TField("execute_ms_avg", org.apache.thrift.protocol.TType.MAP, (short)5); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new BoltStatsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new BoltStatsTupleSchemeFactory()); + } + + public Map> acked; // required + public Map> failed; // required + public Map> process_ms_avg; // required + public Map> executed; // required + public 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.thrift.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.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.ACKED, new org.apache.thrift.meta_data.FieldMetaData("acked", 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.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.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))))); + tmpMap.put(_Fields.FAILED, new org.apache.thrift.meta_data.FieldMetaData("failed", 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.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.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))))); + tmpMap.put(_Fields.PROCESS_MS_AVG, new org.apache.thrift.meta_data.FieldMetaData("process_ms_avg", 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.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.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE))))); + tmpMap.put(_Fields.EXECUTED, new org.apache.thrift.meta_data.FieldMetaData("executed", 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.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.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))))); + tmpMap.put(_Fields.EXECUTE_MS_AVG, new org.apache.thrift.meta_data.FieldMetaData("execute_ms_avg", 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.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.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE))))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.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.isSetAcked()) { + 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.isSetFailed()) { + 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.isSetProcess_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.isSetExecuted()) { + 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.isSetExecute_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 getAckedSize() { + return (this.acked == null) ? 0 : this.acked.size(); + } + + public void putToAcked(String key, Map val) { + if (this.acked == null) { + this.acked = new HashMap>(); + } + this.acked.put(key, val); + } + + public Map> getAcked() { + return this.acked; + } + + public BoltStats setAcked(Map> acked) { + this.acked = acked; + return this; + } + + public void unsetAcked() { + this.acked = null; + } + + /** Returns true if field acked is set (has been assigned a value) and false otherwise */ + public boolean isSetAcked() { + return this.acked != null; + } + + public void setAckedIsSet(boolean value) { + if (!value) { + this.acked = null; + } + } + + public int getFailedSize() { + return (this.failed == null) ? 0 : this.failed.size(); + } + + public void putToFailed(String key, Map val) { + if (this.failed == null) { + this.failed = new HashMap>(); + } + this.failed.put(key, val); + } + + public Map> getFailed() { + return this.failed; + } + + public BoltStats setFailed(Map> failed) { + this.failed = failed; + return this; + } + + public void unsetFailed() { + this.failed = null; + } + + /** Returns true if field failed is set (has been assigned a value) and false otherwise */ + public boolean isSetFailed() { + return this.failed != null; + } + + public void setFailedIsSet(boolean value) { + if (!value) { + this.failed = null; + } + } + + public int getProcess_ms_avgSize() { + return (this.process_ms_avg == null) ? 0 : this.process_ms_avg.size(); + } + + public void putToProcess_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> getProcess_ms_avg() { + return this.process_ms_avg; + } + + public BoltStats setProcess_ms_avg(Map> process_ms_avg) { + this.process_ms_avg = process_ms_avg; + return this; + } + + public void unsetProcess_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 isSetProcess_ms_avg() { + return this.process_ms_avg != null; + } + + public void setProcess_ms_avgIsSet(boolean value) { + if (!value) { + this.process_ms_avg = null; + } + } + + public int getExecutedSize() { + return (this.executed == null) ? 0 : this.executed.size(); + } + + public void putToExecuted(String key, Map val) { + if (this.executed == null) { + this.executed = new HashMap>(); + } + this.executed.put(key, val); + } + + public Map> getExecuted() { + return this.executed; + } + + public BoltStats setExecuted(Map> executed) { + this.executed = executed; + return this; + } + + public void unsetExecuted() { + this.executed = null; + } + + /** Returns true if field executed is set (has been assigned a value) and false otherwise */ + public boolean isSetExecuted() { + return this.executed != null; + } + + public void setExecutedIsSet(boolean value) { + if (!value) { + this.executed = null; + } + } + + public int getExecute_ms_avgSize() { + return (this.execute_ms_avg == null) ? 0 : this.execute_ms_avg.size(); + } + + public void putToExecute_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> getExecute_ms_avg() { + return this.execute_ms_avg; + } + + public BoltStats setExecute_ms_avg(Map> execute_ms_avg) { + this.execute_ms_avg = execute_ms_avg; + return this; + } + + public void unsetExecute_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 isSetExecute_ms_avg() { + return this.execute_ms_avg != null; + } + + public void setExecute_ms_avgIsSet(boolean value) { + if (!value) { + this.execute_ms_avg = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case ACKED: + if (value == null) { + unsetAcked(); + } else { + setAcked((Map>)value); + } + break; + + case FAILED: + if (value == null) { + unsetFailed(); + } else { + setFailed((Map>)value); + } + break; + + case PROCESS_MS_AVG: + if (value == null) { + unsetProcess_ms_avg(); + } else { + setProcess_ms_avg((Map>)value); + } + break; + + case EXECUTED: + if (value == null) { + unsetExecuted(); + } else { + setExecuted((Map>)value); + } + break; + + case EXECUTE_MS_AVG: + if (value == null) { + unsetExecute_ms_avg(); + } else { + setExecute_ms_avg((Map>)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case ACKED: + return getAcked(); + + case FAILED: + return getFailed(); + + case PROCESS_MS_AVG: + return getProcess_ms_avg(); + + case EXECUTED: + return getExecuted(); + + case EXECUTE_MS_AVG: + return getExecute_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 isSetAcked(); + case FAILED: + return isSetFailed(); + case PROCESS_MS_AVG: + return isSetProcess_ms_avg(); + case EXECUTED: + return isSetExecuted(); + case EXECUTE_MS_AVG: + return isSetExecute_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.isSetAcked(); + boolean that_present_acked = true && that.isSetAcked(); + 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.isSetFailed(); + boolean that_present_failed = true && that.isSetFailed(); + 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.isSetProcess_ms_avg(); + boolean that_present_process_ms_avg = true && that.isSetProcess_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.isSetExecuted(); + boolean that_present_executed = true && that.isSetExecuted(); + 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.isSetExecute_ms_avg(); + boolean that_present_execute_ms_avg = true && that.isSetExecute_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 && (isSetAcked()); + builder.append(present_acked); + if (present_acked) + builder.append(acked); + + boolean present_failed = true && (isSetFailed()); + builder.append(present_failed); + if (present_failed) + builder.append(failed); + + boolean present_process_ms_avg = true && (isSetProcess_ms_avg()); + builder.append(present_process_ms_avg); + if (present_process_ms_avg) + builder.append(process_ms_avg); + + boolean present_executed = true && (isSetExecuted()); + builder.append(present_executed); + if (present_executed) + builder.append(executed); + + boolean present_execute_ms_avg = true && (isSetExecute_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(isSetAcked()).compareTo(typedOther.isSetAcked()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetAcked()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.acked, typedOther.acked); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetFailed()).compareTo(typedOther.isSetFailed()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetFailed()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.failed, typedOther.failed); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetProcess_ms_avg()).compareTo(typedOther.isSetProcess_ms_avg()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetProcess_ms_avg()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.process_ms_avg, typedOther.process_ms_avg); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetExecuted()).compareTo(typedOther.isSetExecuted()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetExecuted()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.executed, typedOther.executed); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetExecute_ms_avg()).compareTo(typedOther.isSetExecute_ms_avg()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetExecute_ms_avg()) { + lastComparison = org.apache.thrift.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.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("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.thrift.TException { + // check for required fields + if (acked == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'acked' was not present! Struct: " + toString()); + } + if (failed == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'failed' was not present! Struct: " + toString()); + } + if (process_ms_avg == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'process_ms_avg' was not present! Struct: " + toString()); + } + if (executed == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'executed' was not present! Struct: " + toString()); + } + if (execute_ms_avg == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'execute_ms_avg' was not present! 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 BoltStatsStandardSchemeFactory implements SchemeFactory { + public BoltStatsStandardScheme getScheme() { + return new BoltStatsStandardScheme(); + } + } + + private static class BoltStatsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, BoltStats 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: // ACKED + if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { + { + org.apache.thrift.protocol.TMap _map90 = iprot.readMapBegin(); + struct.acked = new HashMap>(2*_map90.size); + for (int _i91 = 0; _i91 < _map90.size; ++_i91) + { + String _key92; // required + Map _val93; // required + _key92 = iprot.readString(); + { + org.apache.thrift.protocol.TMap _map94 = iprot.readMapBegin(); + _val93 = new HashMap(2*_map94.size); + for (int _i95 = 0; _i95 < _map94.size; ++_i95) + { + GlobalStreamId _key96; // required + long _val97; // required + _key96 = new GlobalStreamId(); + _key96.read(iprot); + _val97 = iprot.readI64(); + _val93.put(_key96, _val97); + } + iprot.readMapEnd(); + } + struct.acked.put(_key92, _val93); + } + iprot.readMapEnd(); + } + struct.setAckedIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // FAILED + if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { + { + org.apache.thrift.protocol.TMap _map98 = iprot.readMapBegin(); + struct.failed = new HashMap>(2*_map98.size); + for (int _i99 = 0; _i99 < _map98.size; ++_i99) + { + String _key100; // required + Map _val101; // required + _key100 = iprot.readString(); + { + org.apache.thrift.protocol.TMap _map102 = iprot.readMapBegin(); + _val101 = new HashMap(2*_map102.size); + for (int _i103 = 0; _i103 < _map102.size; ++_i103) + { + GlobalStreamId _key104; // required + long _val105; // required + _key104 = new GlobalStreamId(); + _key104.read(iprot); + _val105 = iprot.readI64(); + _val101.put(_key104, _val105); + } + iprot.readMapEnd(); + } + struct.failed.put(_key100, _val101); + } + iprot.readMapEnd(); + } + struct.setFailedIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // PROCESS_MS_AVG + if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { + { + org.apache.thrift.protocol.TMap _map106 = iprot.readMapBegin(); + struct.process_ms_avg = new HashMap>(2*_map106.size); + for (int _i107 = 0; _i107 < _map106.size; ++_i107) + { + String _key108; // required + Map _val109; // required + _key108 = iprot.readString(); + { + org.apache.thrift.protocol.TMap _map110 = iprot.readMapBegin(); + _val109 = new HashMap(2*_map110.size); + for (int _i111 = 0; _i111 < _map110.size; ++_i111) + { + GlobalStreamId _key112; // required + double _val113; // required + _key112 = new GlobalStreamId(); + _key112.read(iprot); + _val113 = iprot.readDouble(); + _val109.put(_key112, _val113); + } + iprot.readMapEnd(); + } + struct.process_ms_avg.put(_key108, _val109); + } + iprot.readMapEnd(); + } + struct.setProcess_ms_avgIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // EXECUTED + if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { + { + org.apache.thrift.protocol.TMap _map114 = iprot.readMapBegin(); + struct.executed = new HashMap>(2*_map114.size); + for (int _i115 = 0; _i115 < _map114.size; ++_i115) + { + String _key116; // required + Map _val117; // required + _key116 = iprot.readString(); + { + org.apache.thrift.protocol.TMap _map118 = iprot.readMapBegin(); + _val117 = new HashMap(2*_map118.size); + for (int _i119 = 0; _i119 < _map118.size; ++_i119) + { + GlobalStreamId _key120; // required + long _val121; // required + _key120 = new GlobalStreamId(); + _key120.read(iprot); + _val121 = iprot.readI64(); + _val117.put(_key120, _val121); + } + iprot.readMapEnd(); + } + struct.executed.put(_key116, _val117); + } + iprot.readMapEnd(); + } + struct.setExecutedIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 5: // EXECUTE_MS_AVG + if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { + { + org.apache.thrift.protocol.TMap _map122 = iprot.readMapBegin(); + struct.execute_ms_avg = new HashMap>(2*_map122.size); + for (int _i123 = 0; _i123 < _map122.size; ++_i123) + { + String _key124; // required + Map _val125; // required + _key124 = iprot.readString(); + { + org.apache.thrift.protocol.TMap _map126 = iprot.readMapBegin(); + _val125 = new HashMap(2*_map126.size); + for (int _i127 = 0; _i127 < _map126.size; ++_i127) + { + GlobalStreamId _key128; // required + double _val129; // required + _key128 = new GlobalStreamId(); + _key128.read(iprot); + _val129 = iprot.readDouble(); + _val125.put(_key128, _val129); + } + iprot.readMapEnd(); + } + struct.execute_ms_avg.put(_key124, _val125); + } + iprot.readMapEnd(); + } + struct.setExecute_ms_avgIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, BoltStats struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.acked != null) { + oprot.writeFieldBegin(ACKED_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.acked.size())); + for (Map.Entry> _iter130 : struct.acked.entrySet()) + { + oprot.writeString(_iter130.getKey()); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.I64, _iter130.getValue().size())); + for (Map.Entry _iter131 : _iter130.getValue().entrySet()) + { + _iter131.getKey().write(oprot); + oprot.writeI64(_iter131.getValue()); + } + oprot.writeMapEnd(); + } + } + oprot.writeMapEnd(); + } + oprot.writeFieldEnd(); + } + if (struct.failed != null) { + oprot.writeFieldBegin(FAILED_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.failed.size())); + for (Map.Entry> _iter132 : struct.failed.entrySet()) + { + oprot.writeString(_iter132.getKey()); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.I64, _iter132.getValue().size())); + for (Map.Entry _iter133 : _iter132.getValue().entrySet()) + { + _iter133.getKey().write(oprot); + oprot.writeI64(_iter133.getValue()); + } + oprot.writeMapEnd(); + } + } + oprot.writeMapEnd(); + } + oprot.writeFieldEnd(); + } + if (struct.process_ms_avg != null) { + oprot.writeFieldBegin(PROCESS_MS_AVG_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.process_ms_avg.size())); + for (Map.Entry> _iter134 : struct.process_ms_avg.entrySet()) + { + oprot.writeString(_iter134.getKey()); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.DOUBLE, _iter134.getValue().size())); + for (Map.Entry _iter135 : _iter134.getValue().entrySet()) + { + _iter135.getKey().write(oprot); + oprot.writeDouble(_iter135.getValue()); + } + oprot.writeMapEnd(); + } + } + oprot.writeMapEnd(); + } + oprot.writeFieldEnd(); + } + if (struct.executed != null) { + oprot.writeFieldBegin(EXECUTED_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.executed.size())); + for (Map.Entry> _iter136 : struct.executed.entrySet()) + { + oprot.writeString(_iter136.getKey()); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.I64, _iter136.getValue().size())); + for (Map.Entry _iter137 : _iter136.getValue().entrySet()) + { + _iter137.getKey().write(oprot); + oprot.writeI64(_iter137.getValue()); + } + oprot.writeMapEnd(); + } + } + oprot.writeMapEnd(); + } + oprot.writeFieldEnd(); + } + if (struct.execute_ms_avg != null) { + oprot.writeFieldBegin(EXECUTE_MS_AVG_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.execute_ms_avg.size())); + for (Map.Entry> _iter138 : struct.execute_ms_avg.entrySet()) + { + oprot.writeString(_iter138.getKey()); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.DOUBLE, _iter138.getValue().size())); + for (Map.Entry _iter139 : _iter138.getValue().entrySet()) + { + _iter139.getKey().write(oprot); + oprot.writeDouble(_iter139.getValue()); + } + oprot.writeMapEnd(); + } + } + oprot.writeMapEnd(); + } + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class BoltStatsTupleSchemeFactory implements SchemeFactory { + public BoltStatsTupleScheme getScheme() { + return new BoltStatsTupleScheme(); + } + } + + private static class BoltStatsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, BoltStats struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + { + oprot.writeI32(struct.acked.size()); + for (Map.Entry> _iter140 : struct.acked.entrySet()) + { + oprot.writeString(_iter140.getKey()); + { + oprot.writeI32(_iter140.getValue().size()); + for (Map.Entry _iter141 : _iter140.getValue().entrySet()) + { + _iter141.getKey().write(oprot); + oprot.writeI64(_iter141.getValue()); + } + } + } + } + { + oprot.writeI32(struct.failed.size()); + for (Map.Entry> _iter142 : struct.failed.entrySet()) + { + oprot.writeString(_iter142.getKey()); + { + oprot.writeI32(_iter142.getValue().size()); + for (Map.Entry _iter143 : _iter142.getValue().entrySet()) + { + _iter143.getKey().write(oprot); + oprot.writeI64(_iter143.getValue()); + } + } + } + } + { + oprot.writeI32(struct.process_ms_avg.size()); + for (Map.Entry> _iter144 : struct.process_ms_avg.entrySet()) + { + oprot.writeString(_iter144.getKey()); + { + oprot.writeI32(_iter144.getValue().size()); + for (Map.Entry _iter145 : _iter144.getValue().entrySet()) + { + _iter145.getKey().write(oprot); + oprot.writeDouble(_iter145.getValue()); + } + } + } + } + { + oprot.writeI32(struct.executed.size()); + for (Map.Entry> _iter146 : struct.executed.entrySet()) + { + oprot.writeString(_iter146.getKey()); + { + oprot.writeI32(_iter146.getValue().size()); + for (Map.Entry _iter147 : _iter146.getValue().entrySet()) + { + _iter147.getKey().write(oprot); + oprot.writeI64(_iter147.getValue()); + } + } + } + } + { + oprot.writeI32(struct.execute_ms_avg.size()); + for (Map.Entry> _iter148 : struct.execute_ms_avg.entrySet()) + { + oprot.writeString(_iter148.getKey()); + { + oprot.writeI32(_iter148.getValue().size()); + for (Map.Entry _iter149 : _iter148.getValue().entrySet()) + { + _iter149.getKey().write(oprot); + oprot.writeDouble(_iter149.getValue()); + } + } + } + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, BoltStats struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + { + 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.acked = new HashMap>(2*_map150.size); + for (int _i151 = 0; _i151 < _map150.size; ++_i151) + { + String _key152; // required + Map _val153; // required + _key152 = iprot.readString(); + { + org.apache.thrift.protocol.TMap _map154 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.I64, iprot.readI32()); + _val153 = new HashMap(2*_map154.size); + for (int _i155 = 0; _i155 < _map154.size; ++_i155) + { + GlobalStreamId _key156; // required + long _val157; // required + _key156 = new GlobalStreamId(); + _key156.read(iprot); + _val157 = iprot.readI64(); + _val153.put(_key156, _val157); + } + } + struct.acked.put(_key152, _val153); + } + } + struct.setAckedIsSet(true); + { + 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.failed = new HashMap>(2*_map158.size); + for (int _i159 = 0; _i159 < _map158.size; ++_i159) + { + String _key160; // required + Map _val161; // required + _key160 = iprot.readString(); + { + org.apache.thrift.protocol.TMap _map162 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.I64, iprot.readI32()); + _val161 = new HashMap(2*_map162.size); + for (int _i163 = 0; _i163 < _map162.size; ++_i163) + { + GlobalStreamId _key164; // required + long _val165; // required + _key164 = new GlobalStreamId(); + _key164.read(iprot); + _val165 = iprot.readI64(); + _val161.put(_key164, _val165); + } + } + struct.failed.put(_key160, _val161); + } + } + struct.setFailedIsSet(true); + { + org.apache.thrift.protocol.TMap _map166 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32()); + struct.process_ms_avg = new HashMap>(2*_map166.size); + for (int _i167 = 0; _i167 < _map166.size; ++_i167) + { + String _key168; // required + Map _val169; // required + _key168 = iprot.readString(); + { + org.apache.thrift.protocol.TMap _map170 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.DOUBLE, iprot.readI32()); + _val169 = new HashMap(2*_map170.size); + for (int _i171 = 0; _i171 < _map170.size; ++_i171) + { + GlobalStreamId _key172; // required + double _val173; // required + _key172 = new GlobalStreamId(); + _key172.read(iprot); + _val173 = iprot.readDouble(); + _val169.put(_key172, _val173); + } + } + struct.process_ms_avg.put(_key168, _val169); + } + } + struct.setProcess_ms_avgIsSet(true); + { + org.apache.thrift.protocol.TMap _map174 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32()); + struct.executed = new HashMap>(2*_map174.size); + for (int _i175 = 0; _i175 < _map174.size; ++_i175) + { + String _key176; // required + Map _val177; // required + _key176 = iprot.readString(); + { + org.apache.thrift.protocol.TMap _map178 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.I64, iprot.readI32()); + _val177 = new HashMap(2*_map178.size); + for (int _i179 = 0; _i179 < _map178.size; ++_i179) + { + GlobalStreamId _key180; // required + long _val181; // required + _key180 = new GlobalStreamId(); + _key180.read(iprot); + _val181 = iprot.readI64(); + _val177.put(_key180, _val181); + } + } + struct.executed.put(_key176, _val177); + } + } + struct.setExecutedIsSet(true); + { + org.apache.thrift.protocol.TMap _map182 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32()); + struct.execute_ms_avg = new HashMap>(2*_map182.size); + for (int _i183 = 0; _i183 < _map182.size; ++_i183) + { + String _key184; // required + Map _val185; // required + _key184 = iprot.readString(); + { + org.apache.thrift.protocol.TMap _map186 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.DOUBLE, iprot.readI32()); + _val185 = new HashMap(2*_map186.size); + for (int _i187 = 0; _i187 < _map186.size; ++_i187) + { + GlobalStreamId _key188; // required + double _val189; // required + _key188 = new GlobalStreamId(); + _key188.read(iprot); + _val189 = iprot.readDouble(); + _val185.put(_key188, _val189); + } + } + struct.execute_ms_avg.put(_key184, _val185); + } + } + struct.setExecute_ms_avgIsSet(true); + } + } + +} + diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/ClusterSummary.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/ClusterSummary.java new file mode 100644 index 000000000..a8a35b881 --- /dev/null +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/ClusterSummary.java @@ -0,0 +1,696 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package com.intel.hibench.streambench.storm.thrift.generated; + +import org.apache.commons.lang.builder.HashCodeBuilder; +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 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.thrift.TBase, java.io.Serializable, Cloneable { + 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 SUPERVISORS_FIELD_DESC = new org.apache.thrift.protocol.TField("supervisors", org.apache.thrift.protocol.TType.LIST, (short)1); + private static final org.apache.thrift.protocol.TField NIMBUS_UPTIME_SECS_FIELD_DESC = new org.apache.thrift.protocol.TField("nimbus_uptime_secs", org.apache.thrift.protocol.TType.I32, (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()); + } + + public List supervisors; // required + public int nimbus_uptime_secs; // required + public 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 { + SUPERVISORS((short)1, "supervisors"), + NIMBUS_UPTIME_SECS((short)2, "nimbus_uptime_secs"), + 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: // SUPERVISORS + return SUPERVISORS; + case 2: // NIMBUS_UPTIME_SECS + return NIMBUS_UPTIME_SECS; + 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 + private static final int __NIMBUS_UPTIME_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.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.NIMBUS_UPTIME_SECS, new org.apache.thrift.meta_data.FieldMetaData("nimbus_uptime_secs", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + 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( + List supervisors, + int nimbus_uptime_secs, + List topologies) + { + this(); + this.supervisors = supervisors; + this.nimbus_uptime_secs = nimbus_uptime_secs; + setNimbus_uptime_secsIsSet(true); + this.topologies = topologies; + } + + /** + * Performs a deep copy on other. + */ + public ClusterSummary(ClusterSummary other) { + __isset_bitfield = other.__isset_bitfield; + if (other.isSetSupervisors()) { + 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.isSetTopologies()) { + List __this__topologies = new ArrayList(); + 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.supervisors = null; + setNimbus_uptime_secsIsSet(false); + this.nimbus_uptime_secs = 0; + this.topologies = null; + } + + public int getSupervisorsSize() { + return (this.supervisors == null) ? 0 : this.supervisors.size(); + } + + public java.util.Iterator getSupervisorsIterator() { + return (this.supervisors == null) ? null : this.supervisors.iterator(); + } + + public void addToSupervisors(SupervisorSummary elem) { + if (this.supervisors == null) { + this.supervisors = new ArrayList(); + } + this.supervisors.add(elem); + } + + public List getSupervisors() { + return this.supervisors; + } + + public ClusterSummary setSupervisors(List supervisors) { + this.supervisors = supervisors; + return this; + } + + public void unsetSupervisors() { + this.supervisors = null; + } + + /** Returns true if field supervisors is set (has been assigned a value) and false otherwise */ + public boolean isSetSupervisors() { + return this.supervisors != null; + } + + public void setSupervisorsIsSet(boolean value) { + if (!value) { + this.supervisors = null; + } + } + + public int getNimbus_uptime_secs() { + return this.nimbus_uptime_secs; + } + + public ClusterSummary setNimbus_uptime_secs(int nimbus_uptime_secs) { + this.nimbus_uptime_secs = nimbus_uptime_secs; + setNimbus_uptime_secsIsSet(true); + return this; + } + + public void unsetNimbus_uptime_secs() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __NIMBUS_UPTIME_SECS_ISSET_ID); + } + + /** Returns true if field nimbus_uptime_secs is set (has been assigned a value) and false otherwise */ + public boolean isSetNimbus_uptime_secs() { + return EncodingUtils.testBit(__isset_bitfield, __NIMBUS_UPTIME_SECS_ISSET_ID); + } + + public void setNimbus_uptime_secsIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __NIMBUS_UPTIME_SECS_ISSET_ID, value); + } + + public int getTopologiesSize() { + return (this.topologies == null) ? 0 : this.topologies.size(); + } + + public java.util.Iterator getTopologiesIterator() { + return (this.topologies == null) ? null : this.topologies.iterator(); + } + + public void addToTopologies(TopologySummary elem) { + if (this.topologies == null) { + this.topologies = new ArrayList(); + } + this.topologies.add(elem); + } + + public List getTopologies() { + return this.topologies; + } + + public ClusterSummary setTopologies(List topologies) { + this.topologies = topologies; + return this; + } + + public void unsetTopologies() { + this.topologies = null; + } + + /** Returns true if field topologies is set (has been assigned a value) and false otherwise */ + public boolean isSetTopologies() { + return this.topologies != null; + } + + public void setTopologiesIsSet(boolean value) { + if (!value) { + this.topologies = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUPERVISORS: + if (value == null) { + unsetSupervisors(); + } else { + setSupervisors((List)value); + } + break; + + case NIMBUS_UPTIME_SECS: + if (value == null) { + unsetNimbus_uptime_secs(); + } else { + setNimbus_uptime_secs((Integer)value); + } + break; + + case TOPOLOGIES: + if (value == null) { + unsetTopologies(); + } else { + setTopologies((List)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUPERVISORS: + return getSupervisors(); + + case NIMBUS_UPTIME_SECS: + return Integer.valueOf(getNimbus_uptime_secs()); + + case TOPOLOGIES: + return getTopologies(); + + } + 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 isSetSupervisors(); + case NIMBUS_UPTIME_SECS: + return isSetNimbus_uptime_secs(); + case TOPOLOGIES: + return isSetTopologies(); + } + 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.isSetSupervisors(); + boolean that_present_supervisors = true && that.isSetSupervisors(); + 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.isSetTopologies(); + boolean that_present_topologies = true && that.isSetTopologies(); + 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() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_supervisors = true && (isSetSupervisors()); + 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 && (isSetTopologies()); + builder.append(present_topologies); + if (present_topologies) + builder.append(topologies); + + 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(isSetSupervisors()).compareTo(typedOther.isSetSupervisors()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSupervisors()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.supervisors, typedOther.supervisors); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetNimbus_uptime_secs()).compareTo(typedOther.isSetNimbus_uptime_secs()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetNimbus_uptime_secs()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nimbus_uptime_secs, typedOther.nimbus_uptime_secs); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetTopologies()).compareTo(typedOther.isSetTopologies()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetTopologies()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.topologies, typedOther.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("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; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (supervisors == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'supervisors' was not present! Struct: " + toString()); + } + // alas, we cannot check 'nimbus_uptime_secs' because it's a primitive and you chose the non-beans generator. + if (topologies == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'topologies' was not present! 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 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: // SUPERVISORS + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list74 = iprot.readListBegin(); + struct.supervisors = new ArrayList(_list74.size); + for (int _i75 = 0; _i75 < _list74.size; ++_i75) + { + SupervisorSummary _elem76; // required + _elem76 = new SupervisorSummary(); + _elem76.read(iprot); + struct.supervisors.add(_elem76); + } + iprot.readListEnd(); + } + struct.setSupervisorsIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // NIMBUS_UPTIME_SECS + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.nimbus_uptime_secs = iprot.readI32(); + struct.setNimbus_uptime_secsIsSet(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 _list77 = iprot.readListBegin(); + struct.topologies = new ArrayList(_list77.size); + for (int _i78 = 0; _i78 < _list77.size; ++_i78) + { + TopologySummary _elem79; // required + _elem79 = new TopologySummary(); + _elem79.read(iprot); + struct.topologies.add(_elem79); + } + iprot.readListEnd(); + } + struct.setTopologiesIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + if (!struct.isSetNimbus_uptime_secs()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'nimbus_uptime_secs' was not found in serialized data! Struct: " + toString()); + } + 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.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 _iter80 : struct.supervisors) + { + _iter80.write(oprot); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + oprot.writeFieldBegin(NIMBUS_UPTIME_SECS_FIELD_DESC); + oprot.writeI32(struct.nimbus_uptime_secs); + 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 _iter81 : struct.topologies) + { + _iter81.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; + { + oprot.writeI32(struct.supervisors.size()); + for (SupervisorSummary _iter82 : struct.supervisors) + { + _iter82.write(oprot); + } + } + oprot.writeI32(struct.nimbus_uptime_secs); + { + oprot.writeI32(struct.topologies.size()); + for (TopologySummary _iter83 : struct.topologies) + { + _iter83.write(oprot); + } + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, ClusterSummary struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + { + org.apache.thrift.protocol.TList _list84 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.supervisors = new ArrayList(_list84.size); + for (int _i85 = 0; _i85 < _list84.size; ++_i85) + { + SupervisorSummary _elem86; // required + _elem86 = new SupervisorSummary(); + _elem86.read(iprot); + struct.supervisors.add(_elem86); + } + } + struct.setSupervisorsIsSet(true); + struct.nimbus_uptime_secs = iprot.readI32(); + struct.setNimbus_uptime_secsIsSet(true); + { + org.apache.thrift.protocol.TList _list87 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.topologies = new ArrayList(_list87.size); + for (int _i88 = 0; _i88 < _list87.size; ++_i88) + { + TopologySummary _elem89; // required + _elem89 = new TopologySummary(); + _elem89.read(iprot); + struct.topologies.add(_elem89); + } + } + struct.setTopologiesIsSet(true); + } + } + +} + diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/ComponentCommon.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/ComponentCommon.java new file mode 100644 index 000000000..3b0ab0893 --- /dev/null +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/ComponentCommon.java @@ -0,0 +1,835 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package com.intel.hibench.streambench.storm.thrift.generated; + +import org.apache.commons.lang.builder.HashCodeBuilder; +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 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.thrift.TBase, java.io.Serializable, Cloneable { + 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()); + } + + public Map inputs; // required + public Map streams; // required + public int parallelism_hint; // optional + public 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 _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.isSetInputs()) { + 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.isSetStreams()) { + 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.isSetJson_conf()) { + this.json_conf = other.json_conf; + } + } + + public ComponentCommon deepCopy() { + return new ComponentCommon(this); + } + + @Override + public void clear() { + this.inputs = null; + this.streams = null; + setParallelism_hintIsSet(false); + this.parallelism_hint = 0; + this.json_conf = null; + } + + public int getInputsSize() { + return (this.inputs == null) ? 0 : this.inputs.size(); + } + + public void putToInputs(GlobalStreamId key, Grouping val) { + if (this.inputs == null) { + this.inputs = new HashMap(); + } + this.inputs.put(key, val); + } + + public Map getInputs() { + return this.inputs; + } + + public ComponentCommon setInputs(Map inputs) { + this.inputs = inputs; + return this; + } + + public void unsetInputs() { + this.inputs = null; + } + + /** Returns true if field inputs is set (has been assigned a value) and false otherwise */ + public boolean isSetInputs() { + return this.inputs != null; + } + + public void setInputsIsSet(boolean value) { + if (!value) { + this.inputs = null; + } + } + + public int getStreamsSize() { + return (this.streams == null) ? 0 : this.streams.size(); + } + + public void putToStreams(String key, StreamInfo val) { + if (this.streams == null) { + this.streams = new HashMap(); + } + this.streams.put(key, val); + } + + public Map getStreams() { + return this.streams; + } + + public ComponentCommon setStreams(Map streams) { + this.streams = streams; + return this; + } + + public void unsetStreams() { + this.streams = null; + } + + /** Returns true if field streams is set (has been assigned a value) and false otherwise */ + public boolean isSetStreams() { + return this.streams != null; + } + + public void setStreamsIsSet(boolean value) { + if (!value) { + this.streams = null; + } + } + + public int getParallelism_hint() { + return this.parallelism_hint; + } + + public ComponentCommon setParallelism_hint(int parallelism_hint) { + this.parallelism_hint = parallelism_hint; + setParallelism_hintIsSet(true); + return this; + } + + public void unsetParallelism_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 isSetParallelism_hint() { + return EncodingUtils.testBit(__isset_bitfield, __PARALLELISM_HINT_ISSET_ID); + } + + public void setParallelism_hintIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __PARALLELISM_HINT_ISSET_ID, value); + } + + public String getJson_conf() { + return this.json_conf; + } + + public ComponentCommon setJson_conf(String json_conf) { + this.json_conf = json_conf; + return this; + } + + public void unsetJson_conf() { + this.json_conf = null; + } + + /** Returns true if field json_conf is set (has been assigned a value) and false otherwise */ + public boolean isSetJson_conf() { + return this.json_conf != null; + } + + public void setJson_confIsSet(boolean value) { + if (!value) { + this.json_conf = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case INPUTS: + if (value == null) { + unsetInputs(); + } else { + setInputs((Map)value); + } + break; + + case STREAMS: + if (value == null) { + unsetStreams(); + } else { + setStreams((Map)value); + } + break; + + case PARALLELISM_HINT: + if (value == null) { + unsetParallelism_hint(); + } else { + setParallelism_hint((Integer)value); + } + break; + + case JSON_CONF: + if (value == null) { + unsetJson_conf(); + } else { + setJson_conf((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case INPUTS: + return getInputs(); + + case STREAMS: + return getStreams(); + + case PARALLELISM_HINT: + return Integer.valueOf(getParallelism_hint()); + + case JSON_CONF: + return getJson_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 isSetInputs(); + case STREAMS: + return isSetStreams(); + case PARALLELISM_HINT: + return isSetParallelism_hint(); + case JSON_CONF: + return isSetJson_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.isSetInputs(); + boolean that_present_inputs = true && that.isSetInputs(); + 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.isSetStreams(); + boolean that_present_streams = true && that.isSetStreams(); + 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.isSetParallelism_hint(); + boolean that_present_parallelism_hint = true && that.isSetParallelism_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.isSetJson_conf(); + boolean that_present_json_conf = true && that.isSetJson_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 && (isSetInputs()); + builder.append(present_inputs); + if (present_inputs) + builder.append(inputs); + + boolean present_streams = true && (isSetStreams()); + builder.append(present_streams); + if (present_streams) + builder.append(streams); + + boolean present_parallelism_hint = true && (isSetParallelism_hint()); + builder.append(present_parallelism_hint); + if (present_parallelism_hint) + builder.append(parallelism_hint); + + boolean present_json_conf = true && (isSetJson_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(isSetInputs()).compareTo(typedOther.isSetInputs()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetInputs()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.inputs, typedOther.inputs); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetStreams()).compareTo(typedOther.isSetStreams()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStreams()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.streams, typedOther.streams); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetParallelism_hint()).compareTo(typedOther.isSetParallelism_hint()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetParallelism_hint()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.parallelism_hint, typedOther.parallelism_hint); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetJson_conf()).compareTo(typedOther.isSetJson_conf()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetJson_conf()) { + lastComparison = org.apache.thrift.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.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 (isSetParallelism_hint()) { + if (!first) sb.append(", "); + sb.append("parallelism_hint:"); + sb.append(this.parallelism_hint); + first = false; + } + if (isSetJson_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 (inputs == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'inputs' was not present! Struct: " + toString()); + } + if (streams == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'streams' was not present! 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); + for (int _i25 = 0; _i25 < _map24.size; ++_i25) + { + GlobalStreamId _key26; // required + Grouping _val27; // required + _key26 = new GlobalStreamId(); + _key26.read(iprot); + _val27 = new Grouping(); + _val27.read(iprot); + struct.inputs.put(_key26, _val27); + } + iprot.readMapEnd(); + } + struct.setInputsIsSet(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); + for (int _i29 = 0; _i29 < _map28.size; ++_i29) + { + String _key30; // required + StreamInfo _val31; // required + _key30 = iprot.readString(); + _val31 = new StreamInfo(); + _val31.read(iprot); + struct.streams.put(_key30, _val31); + } + iprot.readMapEnd(); + } + struct.setStreamsIsSet(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.setParallelism_hintIsSet(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.setJson_confIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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.isSetParallelism_hint()) { + oprot.writeFieldBegin(PARALLELISM_HINT_FIELD_DESC); + oprot.writeI32(struct.parallelism_hint); + oprot.writeFieldEnd(); + } + if (struct.json_conf != null) { + if (struct.isSetJson_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.isSetParallelism_hint()) { + optionals.set(0); + } + if (struct.isSetJson_conf()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.isSetParallelism_hint()) { + oprot.writeI32(struct.parallelism_hint); + } + if (struct.isSetJson_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); + for (int _i37 = 0; _i37 < _map36.size; ++_i37) + { + GlobalStreamId _key38; // required + Grouping _val39; // required + _key38 = new GlobalStreamId(); + _key38.read(iprot); + _val39 = new Grouping(); + _val39.read(iprot); + struct.inputs.put(_key38, _val39); + } + } + struct.setInputsIsSet(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); + for (int _i41 = 0; _i41 < _map40.size; ++_i41) + { + String _key42; // required + StreamInfo _val43; // required + _key42 = iprot.readString(); + _val43 = new StreamInfo(); + _val43.read(iprot); + struct.streams.put(_key42, _val43); + } + } + struct.setStreamsIsSet(true); + BitSet incoming = iprot.readBitSet(2); + if (incoming.get(0)) { + struct.parallelism_hint = iprot.readI32(); + struct.setParallelism_hintIsSet(true); + } + if (incoming.get(1)) { + struct.json_conf = iprot.readString(); + struct.setJson_confIsSet(true); + } + } + } + +} + diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/ComponentObject.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/ComponentObject.java new file mode 100644 index 000000000..0f287bc46 --- /dev/null +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/ComponentObject.java @@ -0,0 +1,441 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package com.intel.hibench.streambench.storm.thrift.generated; + +import org.apache.commons.lang.builder.HashCodeBuilder; +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 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.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.setSerialized_java(value); + return x; + } + + public static ComponentObject serialized_java(byte[] value) { + ComponentObject x = new ComponentObject(); + x.setSerialized_java(ByteBuffer.wrap(value)); + return x; + } + + public static ComponentObject shell(ShellComponent value) { + ComponentObject x = new ComponentObject(); + x.setShell(value); + return x; + } + + public static ComponentObject java_object(JavaObject value) { + ComponentObject x = new ComponentObject(); + x.setJava_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 { + 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[] getSerialized_java() { + setSerialized_java(org.apache.thrift.TBaseHelper.rightSize(bufferForSerialized_java())); + ByteBuffer b = bufferForSerialized_java(); + return b == null ? null : b.array(); + } + + public ByteBuffer bufferForSerialized_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 setSerialized_java(byte[] value) { + setSerialized_java(ByteBuffer.wrap(value)); + } + + public void setSerialized_java(ByteBuffer value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.SERIALIZED_JAVA; + value_ = value; + } + + public ShellComponent getShell() { + 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 setShell(ShellComponent value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.SHELL; + value_ = value; + } + + public JavaObject getJava_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 setJava_object(JavaObject value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.JAVA_OBJECT; + value_ = value; + } + + public boolean isSetSerialized_java() { + return setField_ == _Fields.SERIALIZED_JAVA; + } + + + public boolean isSetShell() { + return setField_ == _Fields.SHELL; + } + + + public boolean isSetJava_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() { + HashCodeBuilder hcb = new HashCodeBuilder(); + hcb.append(this.getClass().getName()); + org.apache.thrift.TFieldIdEnum setField = getSetField(); + if (setField != null) { + hcb.append(setField.getThriftFieldId()); + Object value = getFieldValue(); + if (value instanceof org.apache.thrift.TEnum) { + hcb.append(((org.apache.thrift.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.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/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/DRPCExecutionException.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/DRPCExecutionException.java new file mode 100644 index 000000000..d58219e90 --- /dev/null +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/DRPCExecutionException.java @@ -0,0 +1,387 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package com.intel.hibench.streambench.storm.thrift.generated; + +import org.apache.commons.lang.builder.HashCodeBuilder; +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 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 TException implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + 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()); + } + + public 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.isSetMsg()) { + this.msg = other.msg; + } + } + + public DRPCExecutionException deepCopy() { + return new DRPCExecutionException(this); + } + + @Override + public void clear() { + this.msg = null; + } + + public String getMsg() { + return this.msg; + } + + public DRPCExecutionException setMsg(String msg) { + this.msg = msg; + return this; + } + + public void unsetMsg() { + this.msg = null; + } + + /** Returns true if field msg is set (has been assigned a value) and false otherwise */ + public boolean isSetMsg() { + return this.msg != null; + } + + public void setMsgIsSet(boolean value) { + if (!value) { + this.msg = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case MSG: + if (value == null) { + unsetMsg(); + } else { + setMsg((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case MSG: + return getMsg(); + + } + 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 isSetMsg(); + } + 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.isSetMsg(); + boolean that_present_msg = true && that.isSetMsg(); + 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 && (isSetMsg()); + 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(isSetMsg()).compareTo(typedOther.isSetMsg()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetMsg()) { + lastComparison = org.apache.thrift.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.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 (msg == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'msg' was not present! 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.setMsgIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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.setMsgIsSet(true); + } + } + +} + diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/DRPCRequest.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/DRPCRequest.java new file mode 100644 index 000000000..8d89c3956 --- /dev/null +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/DRPCRequest.java @@ -0,0 +1,488 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package com.intel.hibench.streambench.storm.thrift.generated; + +import org.apache.commons.lang.builder.HashCodeBuilder; +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 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.thrift.TBase, java.io.Serializable, Cloneable { + 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()); + } + + public String func_args; // required + public 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.isSetFunc_args()) { + this.func_args = other.func_args; + } + if (other.isSetRequest_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 getFunc_args() { + return this.func_args; + } + + public DRPCRequest setFunc_args(String func_args) { + this.func_args = func_args; + return this; + } + + public void unsetFunc_args() { + this.func_args = null; + } + + /** Returns true if field func_args is set (has been assigned a value) and false otherwise */ + public boolean isSetFunc_args() { + return this.func_args != null; + } + + public void setFunc_argsIsSet(boolean value) { + if (!value) { + this.func_args = null; + } + } + + public String getRequest_id() { + return this.request_id; + } + + public DRPCRequest setRequest_id(String request_id) { + this.request_id = request_id; + return this; + } + + public void unsetRequest_id() { + this.request_id = null; + } + + /** Returns true if field request_id is set (has been assigned a value) and false otherwise */ + public boolean isSetRequest_id() { + return this.request_id != null; + } + + public void setRequest_idIsSet(boolean value) { + if (!value) { + this.request_id = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case FUNC_ARGS: + if (value == null) { + unsetFunc_args(); + } else { + setFunc_args((String)value); + } + break; + + case REQUEST_ID: + if (value == null) { + unsetRequest_id(); + } else { + setRequest_id((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case FUNC_ARGS: + return getFunc_args(); + + case REQUEST_ID: + return getRequest_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 isSetFunc_args(); + case REQUEST_ID: + return isSetRequest_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.isSetFunc_args(); + boolean that_present_func_args = true && that.isSetFunc_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.isSetRequest_id(); + boolean that_present_request_id = true && that.isSetRequest_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 && (isSetFunc_args()); + builder.append(present_func_args); + if (present_func_args) + builder.append(func_args); + + boolean present_request_id = true && (isSetRequest_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(isSetFunc_args()).compareTo(typedOther.isSetFunc_args()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetFunc_args()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.func_args, typedOther.func_args); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetRequest_id()).compareTo(typedOther.isSetRequest_id()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetRequest_id()) { + lastComparison = org.apache.thrift.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.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 (func_args == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'func_args' was not present! Struct: " + toString()); + } + if (request_id == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'request_id' was not present! 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.setFunc_argsIsSet(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.setRequest_idIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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.setFunc_argsIsSet(true); + struct.request_id = iprot.readString(); + struct.setRequest_idIsSet(true); + } + } + +} + diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/DistributedRPC.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/DistributedRPC.java new file mode 100644 index 000000000..e8d3bf3b2 --- /dev/null +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/DistributedRPC.java @@ -0,0 +1,1125 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package com.intel.hibench.streambench.storm.thrift.generated; + +import org.apache.commons.lang.builder.HashCodeBuilder; +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 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.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.setFunctionName(functionName); + args.setFuncArgs(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.isSetSuccess()) { + 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.setFunctionName(functionName); + args.setFuncArgs(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 execute_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + 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()); + } + + public String functionName; // required + public 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.isSetFunctionName()) { + this.functionName = other.functionName; + } + if (other.isSetFuncArgs()) { + 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 getFunctionName() { + return this.functionName; + } + + public execute_args setFunctionName(String functionName) { + this.functionName = functionName; + return this; + } + + public void unsetFunctionName() { + this.functionName = null; + } + + /** Returns true if field functionName is set (has been assigned a value) and false otherwise */ + public boolean isSetFunctionName() { + return this.functionName != null; + } + + public void setFunctionNameIsSet(boolean value) { + if (!value) { + this.functionName = null; + } + } + + public String getFuncArgs() { + return this.funcArgs; + } + + public execute_args setFuncArgs(String funcArgs) { + this.funcArgs = funcArgs; + return this; + } + + public void unsetFuncArgs() { + this.funcArgs = null; + } + + /** Returns true if field funcArgs is set (has been assigned a value) and false otherwise */ + public boolean isSetFuncArgs() { + return this.funcArgs != null; + } + + public void setFuncArgsIsSet(boolean value) { + if (!value) { + this.funcArgs = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case FUNCTION_NAME: + if (value == null) { + unsetFunctionName(); + } else { + setFunctionName((String)value); + } + break; + + case FUNC_ARGS: + if (value == null) { + unsetFuncArgs(); + } else { + setFuncArgs((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case FUNCTION_NAME: + return getFunctionName(); + + case FUNC_ARGS: + return getFuncArgs(); + + } + 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 isSetFunctionName(); + case FUNC_ARGS: + return isSetFuncArgs(); + } + 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.isSetFunctionName(); + boolean that_present_functionName = true && that.isSetFunctionName(); + 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.isSetFuncArgs(); + boolean that_present_funcArgs = true && that.isSetFuncArgs(); + 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 && (isSetFunctionName()); + builder.append(present_functionName); + if (present_functionName) + builder.append(functionName); + + boolean present_funcArgs = true && (isSetFuncArgs()); + 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(isSetFunctionName()).compareTo(typedOther.isSetFunctionName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetFunctionName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.functionName, typedOther.functionName); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetFuncArgs()).compareTo(typedOther.isSetFuncArgs()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetFuncArgs()) { + lastComparison = org.apache.thrift.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.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.setFunctionNameIsSet(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.setFuncArgsIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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.isSetFunctionName()) { + optionals.set(0); + } + if (struct.isSetFuncArgs()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.isSetFunctionName()) { + oprot.writeString(struct.functionName); + } + if (struct.isSetFuncArgs()) { + 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.setFunctionNameIsSet(true); + } + if (incoming.get(1)) { + struct.funcArgs = iprot.readString(); + struct.setFuncArgsIsSet(true); + } + } + } + + } + + public static class execute_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + 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()); + } + + public String success; // required + public 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.isSetSuccess()) { + this.success = other.success; + } + if (other.isSetE()) { + 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 getSuccess() { + return this.success; + } + + public execute_result setSuccess(String success) { + this.success = success; + return this; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public DRPCExecutionException getE() { + return this.e; + } + + public execute_result setE(DRPCExecutionException e) { + this.e = e; + return this; + } + + public void unsetE() { + this.e = null; + } + + /** Returns true if field e is set (has been assigned a value) and false otherwise */ + public boolean isSetE() { + return this.e != null; + } + + public void setEIsSet(boolean value) { + if (!value) { + this.e = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((String)value); + } + break; + + case E: + if (value == null) { + unsetE(); + } else { + setE((DRPCExecutionException)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + case E: + return getE(); + + } + 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 isSetSuccess(); + case E: + return isSetE(); + } + 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.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + 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.isSetE(); + boolean that_present_e = true && that.isSetE(); + 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 && (isSetSuccess()); + builder.append(present_success); + if (present_success) + builder.append(success); + + boolean present_e = true && (isSetE()); + 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(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetE()).compareTo(typedOther.isSetE()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetE()) { + lastComparison = org.apache.thrift.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.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.setSuccessIsSet(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.setEIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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.isSetSuccess()) { + optionals.set(0); + } + if (struct.isSetE()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.isSetSuccess()) { + oprot.writeString(struct.success); + } + if (struct.isSetE()) { + 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.setSuccessIsSet(true); + } + if (incoming.get(1)) { + struct.e = new DRPCExecutionException(); + struct.e.read(iprot); + struct.setEIsSet(true); + } + } + } + + } + +} diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/DistributedRPCInvocations.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/DistributedRPCInvocations.java new file mode 100644 index 000000000..b7f2dac12 --- /dev/null +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/DistributedRPCInvocations.java @@ -0,0 +1,2388 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package com.intel.hibench.streambench.storm.thrift.generated; + +import org.apache.commons.lang.builder.HashCodeBuilder; +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 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.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.setId(id); + args.setResult(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.setFunctionName(functionName); + sendBase("fetchRequest", args); + } + + public DRPCRequest recv_fetchRequest() throws org.apache.thrift.TException + { + fetchRequest_result result = new fetchRequest_result(); + receiveBase(result, "fetchRequest"); + if (result.isSetSuccess()) { + 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.setId(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.setId(id); + args.setResult(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.setFunctionName(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.setId(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 result_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + 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()); + } + + public String id; // required + public 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.isSetId()) { + this.id = other.id; + } + if (other.isSetResult()) { + 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 getId() { + return this.id; + } + + public result_args setId(String id) { + this.id = id; + return this; + } + + public void unsetId() { + this.id = null; + } + + /** Returns true if field id is set (has been assigned a value) and false otherwise */ + public boolean isSetId() { + return this.id != null; + } + + public void setIdIsSet(boolean value) { + if (!value) { + this.id = null; + } + } + + public String getResult() { + return this.result; + } + + public result_args setResult(String result) { + this.result = result; + return this; + } + + public void unsetResult() { + this.result = null; + } + + /** Returns true if field result is set (has been assigned a value) and false otherwise */ + public boolean isSetResult() { + return this.result != null; + } + + public void setResultIsSet(boolean value) { + if (!value) { + this.result = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case ID: + if (value == null) { + unsetId(); + } else { + setId((String)value); + } + break; + + case RESULT: + if (value == null) { + unsetResult(); + } else { + setResult((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case ID: + return getId(); + + case RESULT: + return getResult(); + + } + 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 isSetId(); + case RESULT: + return isSetResult(); + } + 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.isSetId(); + boolean that_present_id = true && that.isSetId(); + 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.isSetResult(); + boolean that_present_result = true && that.isSetResult(); + 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 && (isSetId()); + builder.append(present_id); + if (present_id) + builder.append(id); + + boolean present_result = true && (isSetResult()); + 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(isSetId()).compareTo(typedOther.isSetId()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetId()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.id, typedOther.id); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetResult()).compareTo(typedOther.isSetResult()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetResult()) { + lastComparison = org.apache.thrift.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.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.setIdIsSet(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.setResultIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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.isSetId()) { + optionals.set(0); + } + if (struct.isSetResult()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.isSetId()) { + oprot.writeString(struct.id); + } + if (struct.isSetResult()) { + 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.setIdIsSet(true); + } + if (incoming.get(1)) { + struct.result = iprot.readString(); + struct.setResultIsSet(true); + } + } + } + + } + + public static class result_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + 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() { + 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.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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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 { + 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()); + } + + public 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.isSetFunctionName()) { + this.functionName = other.functionName; + } + } + + public fetchRequest_args deepCopy() { + return new fetchRequest_args(this); + } + + @Override + public void clear() { + this.functionName = null; + } + + public String getFunctionName() { + return this.functionName; + } + + public fetchRequest_args setFunctionName(String functionName) { + this.functionName = functionName; + return this; + } + + public void unsetFunctionName() { + this.functionName = null; + } + + /** Returns true if field functionName is set (has been assigned a value) and false otherwise */ + public boolean isSetFunctionName() { + return this.functionName != null; + } + + public void setFunctionNameIsSet(boolean value) { + if (!value) { + this.functionName = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case FUNCTION_NAME: + if (value == null) { + unsetFunctionName(); + } else { + setFunctionName((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case FUNCTION_NAME: + return getFunctionName(); + + } + 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 isSetFunctionName(); + } + 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.isSetFunctionName(); + boolean that_present_functionName = true && that.isSetFunctionName(); + 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 && (isSetFunctionName()); + 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(isSetFunctionName()).compareTo(typedOther.isSetFunctionName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetFunctionName()) { + lastComparison = org.apache.thrift.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.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.setFunctionNameIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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.isSetFunctionName()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetFunctionName()) { + 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.setFunctionNameIsSet(true); + } + } + } + + } + + public static class fetchRequest_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + 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()); + } + + public 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.isSetSuccess()) { + this.success = new DRPCRequest(other.success); + } + } + + public fetchRequest_result deepCopy() { + return new fetchRequest_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public DRPCRequest getSuccess() { + return this.success; + } + + public fetchRequest_result setSuccess(DRPCRequest success) { + this.success = success; + return this; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((DRPCRequest)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + } + 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 isSetSuccess(); + } + 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.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + 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 && (isSetSuccess()); + 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(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.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.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.setSuccessIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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.isSetSuccess()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetSuccess()) { + 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.setSuccessIsSet(true); + } + } + } + + } + + public static class failRequest_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + 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()); + } + + public 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.isSetId()) { + this.id = other.id; + } + } + + public failRequest_args deepCopy() { + return new failRequest_args(this); + } + + @Override + public void clear() { + this.id = null; + } + + public String getId() { + return this.id; + } + + public failRequest_args setId(String id) { + this.id = id; + return this; + } + + public void unsetId() { + this.id = null; + } + + /** Returns true if field id is set (has been assigned a value) and false otherwise */ + public boolean isSetId() { + return this.id != null; + } + + public void setIdIsSet(boolean value) { + if (!value) { + this.id = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case ID: + if (value == null) { + unsetId(); + } else { + setId((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case ID: + return getId(); + + } + 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 isSetId(); + } + 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.isSetId(); + boolean that_present_id = true && that.isSetId(); + 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 && (isSetId()); + 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(isSetId()).compareTo(typedOther.isSetId()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetId()) { + lastComparison = org.apache.thrift.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.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.setIdIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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.isSetId()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetId()) { + 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.setIdIsSet(true); + } + } + } + + } + + public static class failRequest_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + 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() { + 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.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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/ErrorInfo.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/ErrorInfo.java new file mode 100644 index 000000000..1af82ebb5 --- /dev/null +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/ErrorInfo.java @@ -0,0 +1,487 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package com.intel.hibench.streambench.storm.thrift.generated; + +import org.apache.commons.lang.builder.HashCodeBuilder; +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 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.thrift.TBase, java.io.Serializable, Cloneable { + 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()); + } + + public String error; // required + public 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; + setError_time_secsIsSet(true); + } + + /** + * Performs a deep copy on other. + */ + public ErrorInfo(ErrorInfo other) { + __isset_bitfield = other.__isset_bitfield; + if (other.isSetError()) { + 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; + setError_time_secsIsSet(false); + this.error_time_secs = 0; + } + + public String getError() { + return this.error; + } + + public ErrorInfo setError(String error) { + this.error = error; + return this; + } + + public void unsetError() { + this.error = null; + } + + /** Returns true if field error is set (has been assigned a value) and false otherwise */ + public boolean isSetError() { + return this.error != null; + } + + public void setErrorIsSet(boolean value) { + if (!value) { + this.error = null; + } + } + + public int getError_time_secs() { + return this.error_time_secs; + } + + public ErrorInfo setError_time_secs(int error_time_secs) { + this.error_time_secs = error_time_secs; + setError_time_secsIsSet(true); + return this; + } + + public void unsetError_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 isSetError_time_secs() { + return EncodingUtils.testBit(__isset_bitfield, __ERROR_TIME_SECS_ISSET_ID); + } + + public void setError_time_secsIsSet(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) { + unsetError(); + } else { + setError((String)value); + } + break; + + case ERROR_TIME_SECS: + if (value == null) { + unsetError_time_secs(); + } else { + setError_time_secs((Integer)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case ERROR: + return getError(); + + case ERROR_TIME_SECS: + return Integer.valueOf(getError_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 isSetError(); + case ERROR_TIME_SECS: + return isSetError_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.isSetError(); + boolean that_present_error = true && that.isSetError(); + 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 && (isSetError()); + 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(isSetError()).compareTo(typedOther.isSetError()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetError()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.error, typedOther.error); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetError_time_secs()).compareTo(typedOther.isSetError_time_secs()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetError_time_secs()) { + lastComparison = org.apache.thrift.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.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 (error == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'error' was not present! Struct: " + toString()); + } + // alas, we cannot check 'error_time_secs' because it's a primitive and you chose the non-beans generator. + // 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.setErrorIsSet(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.setError_time_secsIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + if (!struct.isSetError_time_secs()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'error_time_secs' was not found in serialized data! Struct: " + toString()); + } + 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.setErrorIsSet(true); + struct.error_time_secs = iprot.readI32(); + struct.setError_time_secsIsSet(true); + } + } + +} + diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/ExecutorInfo.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/ExecutorInfo.java new file mode 100644 index 000000000..338ab1075 --- /dev/null +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/ExecutorInfo.java @@ -0,0 +1,482 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package com.intel.hibench.streambench.storm.thrift.generated; + +import org.apache.commons.lang.builder.HashCodeBuilder; +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 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.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ExecutorInfo"); + + private static final org.apache.thrift.protocol.TField TASK_START_FIELD_DESC = new org.apache.thrift.protocol.TField("task_start", org.apache.thrift.protocol.TType.I32, (short)1); + private static final org.apache.thrift.protocol.TField TASK_END_FIELD_DESC = new org.apache.thrift.protocol.TField("task_end", org.apache.thrift.protocol.TType.I32, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new ExecutorInfoStandardSchemeFactory()); + schemes.put(TupleScheme.class, new ExecutorInfoTupleSchemeFactory()); + } + + public int task_start; // required + public 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.thrift.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 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_START, new org.apache.thrift.meta_data.FieldMetaData("task_start", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + tmpMap.put(_Fields.TASK_END, new org.apache.thrift.meta_data.FieldMetaData("task_end", 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(ExecutorInfo.class, metaDataMap); + } + + public ExecutorInfo() { + } + + public ExecutorInfo( + int task_start, + int task_end) + { + this(); + this.task_start = task_start; + setTask_startIsSet(true); + this.task_end = task_end; + setTask_endIsSet(true); + } + + /** + * Performs a deep copy on other. + */ + public ExecutorInfo(ExecutorInfo other) { + __isset_bitfield = other.__isset_bitfield; + this.task_start = other.task_start; + this.task_end = other.task_end; + } + + public ExecutorInfo deepCopy() { + return new ExecutorInfo(this); + } + + @Override + public void clear() { + setTask_startIsSet(false); + this.task_start = 0; + setTask_endIsSet(false); + this.task_end = 0; + } + + public int getTask_start() { + return this.task_start; + } + + public ExecutorInfo setTask_start(int task_start) { + this.task_start = task_start; + setTask_startIsSet(true); + return this; + } + + public void unsetTask_start() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TASK_START_ISSET_ID); + } + + /** Returns true if field task_start is set (has been assigned a value) and false otherwise */ + public boolean isSetTask_start() { + return EncodingUtils.testBit(__isset_bitfield, __TASK_START_ISSET_ID); + } + + public void setTask_startIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TASK_START_ISSET_ID, value); + } + + public int getTask_end() { + return this.task_end; + } + + public ExecutorInfo setTask_end(int task_end) { + this.task_end = task_end; + setTask_endIsSet(true); + return this; + } + + public void unsetTask_end() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TASK_END_ISSET_ID); + } + + /** Returns true if field task_end is set (has been assigned a value) and false otherwise */ + public boolean isSetTask_end() { + return EncodingUtils.testBit(__isset_bitfield, __TASK_END_ISSET_ID); + } + + public void setTask_endIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TASK_END_ISSET_ID, value); + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case TASK_START: + if (value == null) { + unsetTask_start(); + } else { + setTask_start((Integer)value); + } + break; + + case TASK_END: + if (value == null) { + unsetTask_end(); + } else { + setTask_end((Integer)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case TASK_START: + return Integer.valueOf(getTask_start()); + + case TASK_END: + return Integer.valueOf(getTask_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 isSetTask_start(); + case TASK_END: + return isSetTask_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(isSetTask_start()).compareTo(typedOther.isSetTask_start()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetTask_start()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.task_start, typedOther.task_start); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetTask_end()).compareTo(typedOther.isSetTask_end()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetTask_end()) { + lastComparison = org.apache.thrift.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.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("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.thrift.TException { + // check for required fields + // alas, we cannot check 'task_start' because it's a primitive and you chose the non-beans generator. + // alas, we cannot check 'task_end' because it's a primitive and you chose the non-beans generator. + // 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 ExecutorInfoStandardSchemeFactory implements SchemeFactory { + public ExecutorInfoStandardScheme getScheme() { + return new ExecutorInfoStandardScheme(); + } + } + + private static class ExecutorInfoStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, ExecutorInfo 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_START + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.task_start = iprot.readI32(); + struct.setTask_startIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // TASK_END + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.task_end = iprot.readI32(); + struct.setTask_endIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + if (!struct.isSetTask_start()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'task_start' was not found in serialized data! Struct: " + toString()); + } + if (!struct.isSetTask_end()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'task_end' was not found in serialized data! Struct: " + toString()); + } + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, ExecutorInfo struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + oprot.writeFieldBegin(TASK_START_FIELD_DESC); + oprot.writeI32(struct.task_start); + oprot.writeFieldEnd(); + oprot.writeFieldBegin(TASK_END_FIELD_DESC); + oprot.writeI32(struct.task_end); + oprot.writeFieldEnd(); + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class ExecutorInfoTupleSchemeFactory implements SchemeFactory { + public ExecutorInfoTupleScheme getScheme() { + return new ExecutorInfoTupleScheme(); + } + } + + private static class ExecutorInfoTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, ExecutorInfo struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeI32(struct.task_start); + oprot.writeI32(struct.task_end); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, ExecutorInfo struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.task_start = iprot.readI32(); + struct.setTask_startIsSet(true); + struct.task_end = iprot.readI32(); + struct.setTask_endIsSet(true); + } + } + +} + diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/ExecutorSpecificStats.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/ExecutorSpecificStats.java new file mode 100644 index 000000000..0721cc085 --- /dev/null +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/ExecutorSpecificStats.java @@ -0,0 +1,366 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package com.intel.hibench.streambench.storm.thrift.generated; + +import org.apache.commons.lang.builder.HashCodeBuilder; +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 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.thrift.TUnion { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ExecutorSpecificStats"); + private static final org.apache.thrift.protocol.TField BOLT_FIELD_DESC = new org.apache.thrift.protocol.TField("bolt", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField SPOUT_FIELD_DESC = new org.apache.thrift.protocol.TField("spout", org.apache.thrift.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.thrift.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.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, new org.apache.thrift.meta_data.FieldMetaData("bolt", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, BoltStats.class))); + tmpMap.put(_Fields.SPOUT, new org.apache.thrift.meta_data.FieldMetaData("spout", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, SpoutStats.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.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.setBolt(value); + return x; + } + + public static ExecutorSpecificStats spout(SpoutStats value) { + ExecutorSpecificStats x = new ExecutorSpecificStats(); + x.setSpout(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 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 BOLT: + if (field.type == BOLT_FIELD_DESC.type) { + BoltStats bolt; + bolt = new BoltStats(); + bolt.read(iprot); + return bolt; + } else { + org.apache.thrift.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.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 { + return null; + } + } + + @Override + protected void standardSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.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 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 BOLT: + BoltStats bolt; + bolt = new BoltStats(); + bolt.read(iprot); + return bolt; + case SPOUT: + SpoutStats spout; + spout = new SpoutStats(); + spout.read(iprot); + return spout; + 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 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.thrift.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.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 BoltStats getBolt() { + 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 setBolt(BoltStats value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.BOLT; + value_ = value; + } + + public SpoutStats getSpout() { + 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 setSpout(SpoutStats value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.SPOUT; + value_ = value; + } + + public boolean isSetBolt() { + return setField_ == _Fields.BOLT; + } + + + public boolean isSetSpout() { + 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.thrift.TBaseHelper.compareTo(getSetField(), other.getSetField()); + if (lastComparison == 0) { + return org.apache.thrift.TBaseHelper.compareTo(getFieldValue(), other.getFieldValue()); + } + return lastComparison; + } + + + @Override + public int hashCode() { + HashCodeBuilder hcb = new HashCodeBuilder(); + hcb.append(this.getClass().getName()); + org.apache.thrift.TFieldIdEnum setField = getSetField(); + if (setField != null) { + hcb.append(setField.getThriftFieldId()); + Object value = getFieldValue(); + if (value instanceof org.apache.thrift.TEnum) { + hcb.append(((org.apache.thrift.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.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/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/ExecutorStats.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/ExecutorStats.java new file mode 100644 index 000000000..a5f29ca64 --- /dev/null +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/ExecutorStats.java @@ -0,0 +1,819 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package com.intel.hibench.streambench.storm.thrift.generated; + +import org.apache.commons.lang.builder.HashCodeBuilder; +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 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 ExecutorStats implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ExecutorStats"); + + private static final org.apache.thrift.protocol.TField EMITTED_FIELD_DESC = new org.apache.thrift.protocol.TField("emitted", org.apache.thrift.protocol.TType.MAP, (short)1); + private static final org.apache.thrift.protocol.TField TRANSFERRED_FIELD_DESC = new org.apache.thrift.protocol.TField("transferred", org.apache.thrift.protocol.TType.MAP, (short)2); + private static final org.apache.thrift.protocol.TField SPECIFIC_FIELD_DESC = new org.apache.thrift.protocol.TField("specific", org.apache.thrift.protocol.TType.STRUCT, (short)3); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new ExecutorStatsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new ExecutorStatsTupleSchemeFactory()); + } + + public Map> emitted; // required + public Map> transferred; // required + public ExecutorSpecificStats specific; // 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 { + EMITTED((short)1, "emitted"), + TRANSFERRED((short)2, "transferred"), + SPECIFIC((short)3, "specific"); + + 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: // TRANSFERRED + return TRANSFERRED; + case 3: // SPECIFIC + return SPECIFIC; + 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.EMITTED, new org.apache.thrift.meta_data.FieldMetaData("emitted", 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.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.I64))))); + tmpMap.put(_Fields.TRANSFERRED, new org.apache.thrift.meta_data.FieldMetaData("transferred", 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.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.I64))))); + tmpMap.put(_Fields.SPECIFIC, new org.apache.thrift.meta_data.FieldMetaData("specific", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ExecutorSpecificStats.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ExecutorStats.class, metaDataMap); + } + + public ExecutorStats() { + } + + public ExecutorStats( + Map> emitted, + Map> transferred, + ExecutorSpecificStats specific) + { + this(); + this.emitted = emitted; + this.transferred = transferred; + this.specific = specific; + } + + /** + * Performs a deep copy on other. + */ + public ExecutorStats(ExecutorStats other) { + if (other.isSetEmitted()) { + 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.isSetTransferred()) { + Map> __this__transferred = new HashMap>(); + for (Map.Entry> other_element : other.transferred.entrySet()) { + + String other_element_key = other_element.getKey(); + Map other_element_value = other_element.getValue(); + + String __this__transferred_copy_key = other_element_key; + + Map __this__transferred_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__transferred_copy_value_copy_key = other_element_value_element_key; + + Long __this__transferred_copy_value_copy_value = other_element_value_element_value; + + __this__transferred_copy_value.put(__this__transferred_copy_value_copy_key, __this__transferred_copy_value_copy_value); + } + + __this__transferred.put(__this__transferred_copy_key, __this__transferred_copy_value); + } + this.transferred = __this__transferred; + } + if (other.isSetSpecific()) { + this.specific = new ExecutorSpecificStats(other.specific); + } + } + + public ExecutorStats deepCopy() { + return new ExecutorStats(this); + } + + @Override + public void clear() { + this.emitted = null; + this.transferred = null; + this.specific = null; + } + + public int getEmittedSize() { + return (this.emitted == null) ? 0 : this.emitted.size(); + } + + public void putToEmitted(String key, Map val) { + if (this.emitted == null) { + this.emitted = new HashMap>(); + } + this.emitted.put(key, val); + } + + public Map> getEmitted() { + return this.emitted; + } + + public ExecutorStats setEmitted(Map> emitted) { + this.emitted = emitted; + return this; + } + + public void unsetEmitted() { + this.emitted = null; + } + + /** Returns true if field emitted is set (has been assigned a value) and false otherwise */ + public boolean isSetEmitted() { + return this.emitted != null; + } + + public void setEmittedIsSet(boolean value) { + if (!value) { + this.emitted = null; + } + } + + public int getTransferredSize() { + return (this.transferred == null) ? 0 : this.transferred.size(); + } + + public void putToTransferred(String key, Map val) { + if (this.transferred == null) { + this.transferred = new HashMap>(); + } + this.transferred.put(key, val); + } + + public Map> getTransferred() { + return this.transferred; + } + + public ExecutorStats setTransferred(Map> transferred) { + this.transferred = transferred; + return this; + } + + public void unsetTransferred() { + this.transferred = null; + } + + /** Returns true if field transferred is set (has been assigned a value) and false otherwise */ + public boolean isSetTransferred() { + return this.transferred != null; + } + + public void setTransferredIsSet(boolean value) { + if (!value) { + this.transferred = null; + } + } + + public ExecutorSpecificStats getSpecific() { + return this.specific; + } + + public ExecutorStats setSpecific(ExecutorSpecificStats specific) { + this.specific = specific; + return this; + } + + public void unsetSpecific() { + this.specific = null; + } + + /** Returns true if field specific is set (has been assigned a value) and false otherwise */ + public boolean isSetSpecific() { + return this.specific != null; + } + + public void setSpecificIsSet(boolean value) { + if (!value) { + this.specific = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case EMITTED: + if (value == null) { + unsetEmitted(); + } else { + setEmitted((Map>)value); + } + break; + + case TRANSFERRED: + if (value == null) { + unsetTransferred(); + } else { + setTransferred((Map>)value); + } + break; + + case SPECIFIC: + if (value == null) { + unsetSpecific(); + } else { + setSpecific((ExecutorSpecificStats)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case EMITTED: + return getEmitted(); + + case TRANSFERRED: + return getTransferred(); + + case SPECIFIC: + return getSpecific(); + + } + 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 isSetEmitted(); + case TRANSFERRED: + return isSetTransferred(); + case SPECIFIC: + return isSetSpecific(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof ExecutorStats) + return this.equals((ExecutorStats)that); + return false; + } + + public boolean equals(ExecutorStats that) { + if (that == null) + return false; + + boolean this_present_emitted = true && this.isSetEmitted(); + boolean that_present_emitted = true && that.isSetEmitted(); + 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_transferred = true && this.isSetTransferred(); + boolean that_present_transferred = true && that.isSetTransferred(); + if (this_present_transferred || that_present_transferred) { + if (!(this_present_transferred && that_present_transferred)) + return false; + if (!this.transferred.equals(that.transferred)) + return false; + } + + boolean this_present_specific = true && this.isSetSpecific(); + boolean that_present_specific = true && that.isSetSpecific(); + if (this_present_specific || that_present_specific) { + if (!(this_present_specific && that_present_specific)) + return false; + if (!this.specific.equals(that.specific)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_emitted = true && (isSetEmitted()); + builder.append(present_emitted); + if (present_emitted) + builder.append(emitted); + + boolean present_transferred = true && (isSetTransferred()); + builder.append(present_transferred); + if (present_transferred) + builder.append(transferred); + + boolean present_specific = true && (isSetSpecific()); + builder.append(present_specific); + if (present_specific) + builder.append(specific); + + return builder.toHashCode(); + } + + public int compareTo(ExecutorStats other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + ExecutorStats typedOther = (ExecutorStats)other; + + lastComparison = Boolean.valueOf(isSetEmitted()).compareTo(typedOther.isSetEmitted()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetEmitted()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.emitted, typedOther.emitted); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetTransferred()).compareTo(typedOther.isSetTransferred()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetTransferred()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.transferred, typedOther.transferred); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetSpecific()).compareTo(typedOther.isSetSpecific()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSpecific()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.specific, typedOther.specific); + 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("ExecutorStats("); + 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("transferred:"); + if (this.transferred == null) { + sb.append("null"); + } else { + sb.append(this.transferred); + } + first = false; + if (!first) sb.append(", "); + sb.append("specific:"); + if (this.specific == null) { + sb.append("null"); + } else { + sb.append(this.specific); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (emitted == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'emitted' was not present! Struct: " + toString()); + } + if (transferred == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'transferred' was not present! Struct: " + toString()); + } + if (specific == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'specific' was not present! 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 ExecutorStatsStandardSchemeFactory implements SchemeFactory { + public ExecutorStatsStandardScheme getScheme() { + return new ExecutorStatsStandardScheme(); + } + } + + private static class ExecutorStatsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, ExecutorStats 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: // EMITTED + if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { + { + org.apache.thrift.protocol.TMap _map250 = iprot.readMapBegin(); + struct.emitted = new HashMap>(2*_map250.size); + for (int _i251 = 0; _i251 < _map250.size; ++_i251) + { + String _key252; // required + Map _val253; // required + _key252 = iprot.readString(); + { + org.apache.thrift.protocol.TMap _map254 = iprot.readMapBegin(); + _val253 = new HashMap(2*_map254.size); + for (int _i255 = 0; _i255 < _map254.size; ++_i255) + { + String _key256; // required + long _val257; // required + _key256 = iprot.readString(); + _val257 = iprot.readI64(); + _val253.put(_key256, _val257); + } + iprot.readMapEnd(); + } + struct.emitted.put(_key252, _val253); + } + iprot.readMapEnd(); + } + struct.setEmittedIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // TRANSFERRED + if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { + { + org.apache.thrift.protocol.TMap _map258 = iprot.readMapBegin(); + struct.transferred = new HashMap>(2*_map258.size); + for (int _i259 = 0; _i259 < _map258.size; ++_i259) + { + String _key260; // required + Map _val261; // required + _key260 = iprot.readString(); + { + org.apache.thrift.protocol.TMap _map262 = iprot.readMapBegin(); + _val261 = new HashMap(2*_map262.size); + for (int _i263 = 0; _i263 < _map262.size; ++_i263) + { + String _key264; // required + long _val265; // required + _key264 = iprot.readString(); + _val265 = iprot.readI64(); + _val261.put(_key264, _val265); + } + iprot.readMapEnd(); + } + struct.transferred.put(_key260, _val261); + } + iprot.readMapEnd(); + } + struct.setTransferredIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // SPECIFIC + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.specific = new ExecutorSpecificStats(); + struct.specific.read(iprot); + struct.setSpecificIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, ExecutorStats struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.emitted != null) { + oprot.writeFieldBegin(EMITTED_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.emitted.size())); + for (Map.Entry> _iter266 : struct.emitted.entrySet()) + { + oprot.writeString(_iter266.getKey()); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, _iter266.getValue().size())); + for (Map.Entry _iter267 : _iter266.getValue().entrySet()) + { + oprot.writeString(_iter267.getKey()); + oprot.writeI64(_iter267.getValue()); + } + oprot.writeMapEnd(); + } + } + oprot.writeMapEnd(); + } + oprot.writeFieldEnd(); + } + if (struct.transferred != null) { + oprot.writeFieldBegin(TRANSFERRED_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.transferred.size())); + for (Map.Entry> _iter268 : struct.transferred.entrySet()) + { + oprot.writeString(_iter268.getKey()); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, _iter268.getValue().size())); + for (Map.Entry _iter269 : _iter268.getValue().entrySet()) + { + oprot.writeString(_iter269.getKey()); + oprot.writeI64(_iter269.getValue()); + } + oprot.writeMapEnd(); + } + } + oprot.writeMapEnd(); + } + oprot.writeFieldEnd(); + } + if (struct.specific != null) { + oprot.writeFieldBegin(SPECIFIC_FIELD_DESC); + struct.specific.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class ExecutorStatsTupleSchemeFactory implements SchemeFactory { + public ExecutorStatsTupleScheme getScheme() { + return new ExecutorStatsTupleScheme(); + } + } + + private static class ExecutorStatsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, ExecutorStats struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + { + oprot.writeI32(struct.emitted.size()); + for (Map.Entry> _iter270 : struct.emitted.entrySet()) + { + oprot.writeString(_iter270.getKey()); + { + oprot.writeI32(_iter270.getValue().size()); + for (Map.Entry _iter271 : _iter270.getValue().entrySet()) + { + oprot.writeString(_iter271.getKey()); + oprot.writeI64(_iter271.getValue()); + } + } + } + } + { + oprot.writeI32(struct.transferred.size()); + for (Map.Entry> _iter272 : struct.transferred.entrySet()) + { + oprot.writeString(_iter272.getKey()); + { + oprot.writeI32(_iter272.getValue().size()); + for (Map.Entry _iter273 : _iter272.getValue().entrySet()) + { + oprot.writeString(_iter273.getKey()); + oprot.writeI64(_iter273.getValue()); + } + } + } + } + struct.specific.write(oprot); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, ExecutorStats struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + { + org.apache.thrift.protocol.TMap _map274 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32()); + struct.emitted = new HashMap>(2*_map274.size); + for (int _i275 = 0; _i275 < _map274.size; ++_i275) + { + String _key276; // required + Map _val277; // required + _key276 = iprot.readString(); + { + org.apache.thrift.protocol.TMap _map278 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, iprot.readI32()); + _val277 = new HashMap(2*_map278.size); + for (int _i279 = 0; _i279 < _map278.size; ++_i279) + { + String _key280; // required + long _val281; // required + _key280 = iprot.readString(); + _val281 = iprot.readI64(); + _val277.put(_key280, _val281); + } + } + struct.emitted.put(_key276, _val277); + } + } + struct.setEmittedIsSet(true); + { + org.apache.thrift.protocol.TMap _map282 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32()); + struct.transferred = new HashMap>(2*_map282.size); + for (int _i283 = 0; _i283 < _map282.size; ++_i283) + { + String _key284; // required + Map _val285; // required + _key284 = iprot.readString(); + { + org.apache.thrift.protocol.TMap _map286 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, iprot.readI32()); + _val285 = new HashMap(2*_map286.size); + for (int _i287 = 0; _i287 < _map286.size; ++_i287) + { + String _key288; // required + long _val289; // required + _key288 = iprot.readString(); + _val289 = iprot.readI64(); + _val285.put(_key288, _val289); + } + } + struct.transferred.put(_key284, _val285); + } + } + struct.setTransferredIsSet(true); + struct.specific = new ExecutorSpecificStats(); + struct.specific.read(iprot); + struct.setSpecificIsSet(true); + } + } + +} + diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/ExecutorSummary.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/ExecutorSummary.java new file mode 100644 index 000000000..e8a3f5b59 --- /dev/null +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/ExecutorSummary.java @@ -0,0 +1,906 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package com.intel.hibench.streambench.storm.thrift.generated; + +import org.apache.commons.lang.builder.HashCodeBuilder; +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 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 ExecutorSummary implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ExecutorSummary"); + + private static final org.apache.thrift.protocol.TField EXECUTOR_INFO_FIELD_DESC = new org.apache.thrift.protocol.TField("executor_info", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField COMPONENT_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("component_id", org.apache.thrift.protocol.TType.STRING, (short)2); + 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)3); + 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)4); + 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)5); + private static final org.apache.thrift.protocol.TField STATS_FIELD_DESC = new org.apache.thrift.protocol.TField("stats", org.apache.thrift.protocol.TType.STRUCT, (short)7); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new ExecutorSummaryStandardSchemeFactory()); + schemes.put(TupleScheme.class, new ExecutorSummaryTupleSchemeFactory()); + } + + public ExecutorInfo executor_info; // required + public String component_id; // required + public String host; // required + public int port; // required + public int uptime_secs; // required + public ExecutorStats stats; // 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 { + EXECUTOR_INFO((short)1, "executor_info"), + COMPONENT_ID((short)2, "component_id"), + HOST((short)3, "host"), + PORT((short)4, "port"), + UPTIME_SECS((short)5, "uptime_secs"), + STATS((short)7, "stats"); + + 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: // EXECUTOR_INFO + return EXECUTOR_INFO; + 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 7: // STATS + return STATS; + 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_SECS_ISSET_ID = 1; + private byte __isset_bitfield = 0; + private _Fields optionals[] = {_Fields.STATS}; + 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.EXECUTOR_INFO, new org.apache.thrift.meta_data.FieldMetaData("executor_info", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ExecutorInfo.class))); + tmpMap.put(_Fields.COMPONENT_ID, new org.apache.thrift.meta_data.FieldMetaData("component_id", 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.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.STATS, new org.apache.thrift.meta_data.FieldMetaData("stats", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ExecutorStats.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ExecutorSummary.class, metaDataMap); + } + + public ExecutorSummary() { + } + + public ExecutorSummary( + ExecutorInfo executor_info, + String component_id, + String host, + int port, + int uptime_secs) + { + this(); + this.executor_info = executor_info; + this.component_id = component_id; + this.host = host; + this.port = port; + setPortIsSet(true); + this.uptime_secs = uptime_secs; + setUptime_secsIsSet(true); + } + + /** + * Performs a deep copy on other. + */ + public ExecutorSummary(ExecutorSummary other) { + __isset_bitfield = other.__isset_bitfield; + if (other.isSetExecutor_info()) { + this.executor_info = new ExecutorInfo(other.executor_info); + } + if (other.isSetComponent_id()) { + this.component_id = other.component_id; + } + if (other.isSetHost()) { + this.host = other.host; + } + this.port = other.port; + this.uptime_secs = other.uptime_secs; + if (other.isSetStats()) { + this.stats = new ExecutorStats(other.stats); + } + } + + public ExecutorSummary deepCopy() { + return new ExecutorSummary(this); + } + + @Override + public void clear() { + this.executor_info = null; + this.component_id = null; + this.host = null; + setPortIsSet(false); + this.port = 0; + setUptime_secsIsSet(false); + this.uptime_secs = 0; + this.stats = null; + } + + public ExecutorInfo getExecutor_info() { + return this.executor_info; + } + + public ExecutorSummary setExecutor_info(ExecutorInfo executor_info) { + this.executor_info = executor_info; + return this; + } + + public void unsetExecutor_info() { + this.executor_info = null; + } + + /** Returns true if field executor_info is set (has been assigned a value) and false otherwise */ + public boolean isSetExecutor_info() { + return this.executor_info != null; + } + + public void setExecutor_infoIsSet(boolean value) { + if (!value) { + this.executor_info = null; + } + } + + public String getComponent_id() { + return this.component_id; + } + + public ExecutorSummary setComponent_id(String component_id) { + this.component_id = component_id; + return this; + } + + public void unsetComponent_id() { + this.component_id = null; + } + + /** Returns true if field component_id is set (has been assigned a value) and false otherwise */ + public boolean isSetComponent_id() { + return this.component_id != null; + } + + public void setComponent_idIsSet(boolean value) { + if (!value) { + this.component_id = null; + } + } + + public String getHost() { + return this.host; + } + + public ExecutorSummary setHost(String host) { + this.host = host; + return this; + } + + public void unsetHost() { + this.host = null; + } + + /** Returns true if field host is set (has been assigned a value) and false otherwise */ + public boolean isSetHost() { + return this.host != null; + } + + public void setHostIsSet(boolean value) { + if (!value) { + this.host = null; + } + } + + public int getPort() { + return this.port; + } + + public ExecutorSummary setPort(int port) { + this.port = port; + setPortIsSet(true); + return this; + } + + public void unsetPort() { + __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 isSetPort() { + return EncodingUtils.testBit(__isset_bitfield, __PORT_ISSET_ID); + } + + public void setPortIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __PORT_ISSET_ID, value); + } + + public int getUptime_secs() { + return this.uptime_secs; + } + + public ExecutorSummary setUptime_secs(int uptime_secs) { + this.uptime_secs = uptime_secs; + setUptime_secsIsSet(true); + return this; + } + + public void unsetUptime_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 isSetUptime_secs() { + return EncodingUtils.testBit(__isset_bitfield, __UPTIME_SECS_ISSET_ID); + } + + public void setUptime_secsIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __UPTIME_SECS_ISSET_ID, value); + } + + public ExecutorStats getStats() { + return this.stats; + } + + public ExecutorSummary setStats(ExecutorStats stats) { + this.stats = stats; + return this; + } + + public void unsetStats() { + this.stats = null; + } + + /** Returns true if field stats is set (has been assigned a value) and false otherwise */ + public boolean isSetStats() { + return this.stats != null; + } + + public void setStatsIsSet(boolean value) { + if (!value) { + this.stats = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case EXECUTOR_INFO: + if (value == null) { + unsetExecutor_info(); + } else { + setExecutor_info((ExecutorInfo)value); + } + break; + + case COMPONENT_ID: + if (value == null) { + unsetComponent_id(); + } else { + setComponent_id((String)value); + } + break; + + case HOST: + if (value == null) { + unsetHost(); + } else { + setHost((String)value); + } + break; + + case PORT: + if (value == null) { + unsetPort(); + } else { + setPort((Integer)value); + } + break; + + case UPTIME_SECS: + if (value == null) { + unsetUptime_secs(); + } else { + setUptime_secs((Integer)value); + } + break; + + case STATS: + if (value == null) { + unsetStats(); + } else { + setStats((ExecutorStats)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case EXECUTOR_INFO: + return getExecutor_info(); + + case COMPONENT_ID: + return getComponent_id(); + + case HOST: + return getHost(); + + case PORT: + return Integer.valueOf(getPort()); + + case UPTIME_SECS: + return Integer.valueOf(getUptime_secs()); + + case STATS: + return getStats(); + + } + 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 EXECUTOR_INFO: + return isSetExecutor_info(); + case COMPONENT_ID: + return isSetComponent_id(); + case HOST: + return isSetHost(); + case PORT: + return isSetPort(); + case UPTIME_SECS: + return isSetUptime_secs(); + case STATS: + return isSetStats(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof ExecutorSummary) + return this.equals((ExecutorSummary)that); + return false; + } + + public boolean equals(ExecutorSummary that) { + if (that == null) + return false; + + boolean this_present_executor_info = true && this.isSetExecutor_info(); + boolean that_present_executor_info = true && that.isSetExecutor_info(); + if (this_present_executor_info || that_present_executor_info) { + if (!(this_present_executor_info && that_present_executor_info)) + return false; + if (!this.executor_info.equals(that.executor_info)) + return false; + } + + boolean this_present_component_id = true && this.isSetComponent_id(); + boolean that_present_component_id = true && that.isSetComponent_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.isSetHost(); + boolean that_present_host = true && that.isSetHost(); + 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; + 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_stats = true && this.isSetStats(); + boolean that_present_stats = true && that.isSetStats(); + if (this_present_stats || that_present_stats) { + if (!(this_present_stats && that_present_stats)) + return false; + if (!this.stats.equals(that.stats)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_executor_info = true && (isSetExecutor_info()); + builder.append(present_executor_info); + if (present_executor_info) + builder.append(executor_info); + + boolean present_component_id = true && (isSetComponent_id()); + builder.append(present_component_id); + if (present_component_id) + builder.append(component_id); + + boolean present_host = true && (isSetHost()); + 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; + builder.append(present_uptime_secs); + if (present_uptime_secs) + builder.append(uptime_secs); + + boolean present_stats = true && (isSetStats()); + builder.append(present_stats); + if (present_stats) + builder.append(stats); + + return builder.toHashCode(); + } + + public int compareTo(ExecutorSummary other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + ExecutorSummary typedOther = (ExecutorSummary)other; + + lastComparison = Boolean.valueOf(isSetExecutor_info()).compareTo(typedOther.isSetExecutor_info()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetExecutor_info()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.executor_info, typedOther.executor_info); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetComponent_id()).compareTo(typedOther.isSetComponent_id()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetComponent_id()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.component_id, typedOther.component_id); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetHost()).compareTo(typedOther.isSetHost()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetHost()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.host, typedOther.host); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetPort()).compareTo(typedOther.isSetPort()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetPort()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.port, typedOther.port); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetUptime_secs()).compareTo(typedOther.isSetUptime_secs()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetUptime_secs()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.uptime_secs, typedOther.uptime_secs); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetStats()).compareTo(typedOther.isSetStats()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStats()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.stats, typedOther.stats); + 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("ExecutorSummary("); + boolean first = true; + + sb.append("executor_info:"); + if (this.executor_info == null) { + sb.append("null"); + } else { + sb.append(this.executor_info); + } + 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 (!first) sb.append(", "); + sb.append("uptime_secs:"); + sb.append(this.uptime_secs); + first = false; + if (isSetStats()) { + if (!first) sb.append(", "); + sb.append("stats:"); + if (this.stats == null) { + sb.append("null"); + } else { + sb.append(this.stats); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (executor_info == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'executor_info' was not present! Struct: " + toString()); + } + if (component_id == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'component_id' was not present! Struct: " + toString()); + } + if (host == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'host' was not present! Struct: " + toString()); + } + // alas, we cannot check 'port' because it's a primitive and you chose the non-beans generator. + // alas, we cannot check 'uptime_secs' because it's a primitive and you chose the non-beans generator. + // check for sub-struct validity + if (executor_info != null) { + executor_info.validate(); + } + if (stats != null) { + stats.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 ExecutorSummaryStandardSchemeFactory implements SchemeFactory { + public ExecutorSummaryStandardScheme getScheme() { + return new ExecutorSummaryStandardScheme(); + } + } + + private static class ExecutorSummaryStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, ExecutorSummary 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: // EXECUTOR_INFO + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.executor_info = new ExecutorInfo(); + struct.executor_info.read(iprot); + struct.setExecutor_infoIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // COMPONENT_ID + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.component_id = iprot.readString(); + struct.setComponent_idIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // HOST + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.host = iprot.readString(); + struct.setHostIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // PORT + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.port = iprot.readI32(); + struct.setPortIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 5: // UPTIME_SECS + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.uptime_secs = iprot.readI32(); + struct.setUptime_secsIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 7: // STATS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.stats = new ExecutorStats(); + struct.stats.read(iprot); + struct.setStatsIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + if (!struct.isSetPort()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'port' was not found in serialized data! Struct: " + toString()); + } + if (!struct.isSetUptime_secs()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'uptime_secs' was not found in serialized data! Struct: " + toString()); + } + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, ExecutorSummary struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.executor_info != null) { + oprot.writeFieldBegin(EXECUTOR_INFO_FIELD_DESC); + struct.executor_info.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.component_id != null) { + oprot.writeFieldBegin(COMPONENT_ID_FIELD_DESC); + oprot.writeString(struct.component_id); + 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(); + oprot.writeFieldBegin(UPTIME_SECS_FIELD_DESC); + oprot.writeI32(struct.uptime_secs); + oprot.writeFieldEnd(); + if (struct.stats != null) { + if (struct.isSetStats()) { + oprot.writeFieldBegin(STATS_FIELD_DESC); + struct.stats.write(oprot); + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class ExecutorSummaryTupleSchemeFactory implements SchemeFactory { + public ExecutorSummaryTupleScheme getScheme() { + return new ExecutorSummaryTupleScheme(); + } + } + + private static class ExecutorSummaryTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, ExecutorSummary struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.executor_info.write(oprot); + oprot.writeString(struct.component_id); + oprot.writeString(struct.host); + oprot.writeI32(struct.port); + oprot.writeI32(struct.uptime_secs); + BitSet optionals = new BitSet(); + if (struct.isSetStats()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetStats()) { + struct.stats.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, ExecutorSummary struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.executor_info = new ExecutorInfo(); + struct.executor_info.read(iprot); + struct.setExecutor_infoIsSet(true); + struct.component_id = iprot.readString(); + struct.setComponent_idIsSet(true); + struct.host = iprot.readString(); + struct.setHostIsSet(true); + struct.port = iprot.readI32(); + struct.setPortIsSet(true); + struct.uptime_secs = iprot.readI32(); + struct.setUptime_secsIsSet(true); + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.stats = new ExecutorStats(); + struct.stats.read(iprot); + struct.setStatsIsSet(true); + } + } + } + +} + diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/GlobalStreamId.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/GlobalStreamId.java new file mode 100644 index 000000000..b2ceb14d9 --- /dev/null +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/GlobalStreamId.java @@ -0,0 +1,488 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package com.intel.hibench.streambench.storm.thrift.generated; + +import org.apache.commons.lang.builder.HashCodeBuilder; +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 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.thrift.TBase, java.io.Serializable, Cloneable { + 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()); + } + + public String componentId; // required + public 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.isSetComponentId()) { + this.componentId = other.componentId; + } + if (other.isSetStreamId()) { + this.streamId = other.streamId; + } + } + + public GlobalStreamId deepCopy() { + return new GlobalStreamId(this); + } + + @Override + public void clear() { + this.componentId = null; + this.streamId = null; + } + + public String getComponentId() { + return this.componentId; + } + + public GlobalStreamId setComponentId(String componentId) { + this.componentId = componentId; + return this; + } + + public void unsetComponentId() { + this.componentId = null; + } + + /** Returns true if field componentId is set (has been assigned a value) and false otherwise */ + public boolean isSetComponentId() { + return this.componentId != null; + } + + public void setComponentIdIsSet(boolean value) { + if (!value) { + this.componentId = null; + } + } + + public String getStreamId() { + return this.streamId; + } + + public GlobalStreamId setStreamId(String streamId) { + this.streamId = streamId; + return this; + } + + public void unsetStreamId() { + this.streamId = null; + } + + /** Returns true if field streamId is set (has been assigned a value) and false otherwise */ + public boolean isSetStreamId() { + return this.streamId != null; + } + + public void setStreamIdIsSet(boolean value) { + if (!value) { + this.streamId = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case COMPONENT_ID: + if (value == null) { + unsetComponentId(); + } else { + setComponentId((String)value); + } + break; + + case STREAM_ID: + if (value == null) { + unsetStreamId(); + } else { + setStreamId((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case COMPONENT_ID: + return getComponentId(); + + case STREAM_ID: + return getStreamId(); + + } + 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 isSetComponentId(); + case STREAM_ID: + return isSetStreamId(); + } + 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.isSetComponentId(); + boolean that_present_componentId = true && that.isSetComponentId(); + 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.isSetStreamId(); + boolean that_present_streamId = true && that.isSetStreamId(); + 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 && (isSetComponentId()); + builder.append(present_componentId); + if (present_componentId) + builder.append(componentId); + + boolean present_streamId = true && (isSetStreamId()); + 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(isSetComponentId()).compareTo(typedOther.isSetComponentId()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetComponentId()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.componentId, typedOther.componentId); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetStreamId()).compareTo(typedOther.isSetStreamId()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStreamId()) { + lastComparison = org.apache.thrift.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.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 (componentId == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'componentId' was not present! Struct: " + toString()); + } + if (streamId == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'streamId' was not present! 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.setComponentIdIsSet(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.setStreamIdIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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.setComponentIdIsSet(true); + struct.streamId = iprot.readString(); + struct.setStreamIdIsSet(true); + } + } + +} + diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/Grouping.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/Grouping.java new file mode 100644 index 000000000..27f0225b3 --- /dev/null +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/Grouping.java @@ -0,0 +1,779 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package com.intel.hibench.streambench.storm.thrift.generated; + +import org.apache.commons.lang.builder.HashCodeBuilder; +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 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.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); + + /** 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"); + + 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; + 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))); + 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.setFields(value); + return x; + } + + public static Grouping shuffle(NullStruct value) { + Grouping x = new Grouping(); + x.setShuffle(value); + return x; + } + + public static Grouping all(NullStruct value) { + Grouping x = new Grouping(); + x.setAll(value); + return x; + } + + public static Grouping none(NullStruct value) { + Grouping x = new Grouping(); + x.setNone(value); + return x; + } + + public static Grouping direct(NullStruct value) { + Grouping x = new Grouping(); + x.setDirect(value); + return x; + } + + public static Grouping custom_object(JavaObject value) { + Grouping x = new Grouping(); + x.setCustom_object(value); + return x; + } + + public static Grouping custom_serialized(ByteBuffer value) { + Grouping x = new Grouping(); + x.setCustom_serialized(value); + return x; + } + + public static Grouping custom_serialized(byte[] value) { + Grouping x = new Grouping(); + x.setCustom_serialized(ByteBuffer.wrap(value)); + return x; + } + + public static Grouping local_or_shuffle(NullStruct value) { + Grouping x = new Grouping(); + x.setLocal_or_shuffle(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()); + 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); + for (int _i9 = 0; _i9 < _list8.size; ++_i9) + { + String _elem10; // required + _elem10 = iprot.readString(); + fields.add(_elem10); + } + 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; + } + default: + throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!"); + } + } else { + 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; + 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); + for (int _i13 = 0; _i13 < _list12.size; ++_i13) + { + String _elem14; // required + _elem14 = iprot.readString(); + fields.add(_elem14); + } + 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; + 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; + 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; + 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 getFields() { + 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 setFields(List value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.FIELDS; + value_ = value; + } + + public NullStruct getShuffle() { + 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 setShuffle(NullStruct value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.SHUFFLE; + value_ = value; + } + + public NullStruct getAll() { + 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 setAll(NullStruct value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.ALL; + value_ = value; + } + + public NullStruct getNone() { + 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 setNone(NullStruct value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.NONE; + value_ = value; + } + + public NullStruct getDirect() { + 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 setDirect(NullStruct value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.DIRECT; + value_ = value; + } + + public JavaObject getCustom_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 setCustom_object(JavaObject value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.CUSTOM_OBJECT; + value_ = value; + } + + public byte[] getCustom_serialized() { + setCustom_serialized(org.apache.thrift.TBaseHelper.rightSize(bufferForCustom_serialized())); + ByteBuffer b = bufferForCustom_serialized(); + return b == null ? null : b.array(); + } + + public ByteBuffer bufferForCustom_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 setCustom_serialized(byte[] value) { + setCustom_serialized(ByteBuffer.wrap(value)); + } + + public void setCustom_serialized(ByteBuffer value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.CUSTOM_SERIALIZED; + value_ = value; + } + + public NullStruct getLocal_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 setLocal_or_shuffle(NullStruct value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.LOCAL_OR_SHUFFLE; + value_ = value; + } + + public boolean isSetFields() { + return setField_ == _Fields.FIELDS; + } + + + public boolean isSetShuffle() { + return setField_ == _Fields.SHUFFLE; + } + + + public boolean isSetAll() { + return setField_ == _Fields.ALL; + } + + + public boolean isSetNone() { + return setField_ == _Fields.NONE; + } + + + public boolean isSetDirect() { + return setField_ == _Fields.DIRECT; + } + + + public boolean isSetCustom_object() { + return setField_ == _Fields.CUSTOM_OBJECT; + } + + + public boolean isSetCustom_serialized() { + return setField_ == _Fields.CUSTOM_SERIALIZED; + } + + + public boolean isSetLocal_or_shuffle() { + return setField_ == _Fields.LOCAL_OR_SHUFFLE; + } + + + 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() { + HashCodeBuilder hcb = new HashCodeBuilder(); + hcb.append(this.getClass().getName()); + org.apache.thrift.TFieldIdEnum setField = getSetField(); + if (setField != null) { + hcb.append(setField.getThriftFieldId()); + Object value = getFieldValue(); + if (value instanceof org.apache.thrift.TEnum) { + hcb.append(((org.apache.thrift.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.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/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/InvalidTopologyException.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/InvalidTopologyException.java new file mode 100644 index 000000000..bfa2d886b --- /dev/null +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/InvalidTopologyException.java @@ -0,0 +1,387 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package com.intel.hibench.streambench.storm.thrift.generated; + +import org.apache.commons.lang.builder.HashCodeBuilder; +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 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 TException implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + 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()); + } + + public 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.isSetMsg()) { + this.msg = other.msg; + } + } + + public InvalidTopologyException deepCopy() { + return new InvalidTopologyException(this); + } + + @Override + public void clear() { + this.msg = null; + } + + public String getMsg() { + return this.msg; + } + + public InvalidTopologyException setMsg(String msg) { + this.msg = msg; + return this; + } + + public void unsetMsg() { + this.msg = null; + } + + /** Returns true if field msg is set (has been assigned a value) and false otherwise */ + public boolean isSetMsg() { + return this.msg != null; + } + + public void setMsgIsSet(boolean value) { + if (!value) { + this.msg = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case MSG: + if (value == null) { + unsetMsg(); + } else { + setMsg((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case MSG: + return getMsg(); + + } + 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 isSetMsg(); + } + 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.isSetMsg(); + boolean that_present_msg = true && that.isSetMsg(); + 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 && (isSetMsg()); + 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(isSetMsg()).compareTo(typedOther.isSetMsg()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetMsg()) { + lastComparison = org.apache.thrift.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.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 (msg == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'msg' was not present! 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.setMsgIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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.setMsgIsSet(true); + } + } + +} + diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/JavaObject.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/JavaObject.java new file mode 100644 index 000000000..14b1f0929 --- /dev/null +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/JavaObject.java @@ -0,0 +1,542 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package com.intel.hibench.streambench.storm.thrift.generated; + +import org.apache.commons.lang.builder.HashCodeBuilder; +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 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.thrift.TBase, java.io.Serializable, Cloneable { + 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()); + } + + public String full_class_name; // required + public 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.isSetFull_class_name()) { + this.full_class_name = other.full_class_name; + } + if (other.isSetArgs_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 getFull_class_name() { + return this.full_class_name; + } + + public JavaObject setFull_class_name(String full_class_name) { + this.full_class_name = full_class_name; + return this; + } + + public void unsetFull_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 isSetFull_class_name() { + return this.full_class_name != null; + } + + public void setFull_class_nameIsSet(boolean value) { + if (!value) { + this.full_class_name = null; + } + } + + public int getArgs_listSize() { + return (this.args_list == null) ? 0 : this.args_list.size(); + } + + public java.util.Iterator getArgs_listIterator() { + return (this.args_list == null) ? null : this.args_list.iterator(); + } + + public void addToArgs_list(JavaObjectArg elem) { + if (this.args_list == null) { + this.args_list = new ArrayList(); + } + this.args_list.add(elem); + } + + public List getArgs_list() { + return this.args_list; + } + + public JavaObject setArgs_list(List args_list) { + this.args_list = args_list; + return this; + } + + public void unsetArgs_list() { + this.args_list = null; + } + + /** Returns true if field args_list is set (has been assigned a value) and false otherwise */ + public boolean isSetArgs_list() { + return this.args_list != null; + } + + public void setArgs_listIsSet(boolean value) { + if (!value) { + this.args_list = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case FULL_CLASS_NAME: + if (value == null) { + unsetFull_class_name(); + } else { + setFull_class_name((String)value); + } + break; + + case ARGS_LIST: + if (value == null) { + unsetArgs_list(); + } else { + setArgs_list((List)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case FULL_CLASS_NAME: + return getFull_class_name(); + + case ARGS_LIST: + return getArgs_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 isSetFull_class_name(); + case ARGS_LIST: + return isSetArgs_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.isSetFull_class_name(); + boolean that_present_full_class_name = true && that.isSetFull_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.isSetArgs_list(); + boolean that_present_args_list = true && that.isSetArgs_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 && (isSetFull_class_name()); + builder.append(present_full_class_name); + if (present_full_class_name) + builder.append(full_class_name); + + boolean present_args_list = true && (isSetArgs_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(isSetFull_class_name()).compareTo(typedOther.isSetFull_class_name()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetFull_class_name()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.full_class_name, typedOther.full_class_name); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetArgs_list()).compareTo(typedOther.isSetArgs_list()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetArgs_list()) { + lastComparison = org.apache.thrift.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.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 (full_class_name == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'full_class_name' was not present! Struct: " + toString()); + } + if (args_list == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'args_list' was not present! 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.setFull_class_nameIsSet(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); + for (int _i1 = 0; _i1 < _list0.size; ++_i1) + { + JavaObjectArg _elem2; // required + _elem2 = new JavaObjectArg(); + _elem2.read(iprot); + struct.args_list.add(_elem2); + } + iprot.readListEnd(); + } + struct.setArgs_listIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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.setFull_class_nameIsSet(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); + for (int _i6 = 0; _i6 < _list5.size; ++_i6) + { + JavaObjectArg _elem7; // required + _elem7 = new JavaObjectArg(); + _elem7.read(iprot); + struct.args_list.add(_elem7); + } + } + struct.setArgs_listIsSet(true); + } + } + +} + diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/JavaObjectArg.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/JavaObjectArg.java new file mode 100644 index 000000000..19fca5bb7 --- /dev/null +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/JavaObjectArg.java @@ -0,0 +1,610 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package com.intel.hibench.streambench.storm.thrift.generated; + +import org.apache.commons.lang.builder.HashCodeBuilder; +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 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.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.setInt_arg(value); + return x; + } + + public static JavaObjectArg long_arg(long value) { + JavaObjectArg x = new JavaObjectArg(); + x.setLong_arg(value); + return x; + } + + public static JavaObjectArg string_arg(String value) { + JavaObjectArg x = new JavaObjectArg(); + x.setString_arg(value); + return x; + } + + public static JavaObjectArg bool_arg(boolean value) { + JavaObjectArg x = new JavaObjectArg(); + x.setBool_arg(value); + return x; + } + + public static JavaObjectArg binary_arg(ByteBuffer value) { + JavaObjectArg x = new JavaObjectArg(); + x.setBinary_arg(value); + return x; + } + + public static JavaObjectArg binary_arg(byte[] value) { + JavaObjectArg x = new JavaObjectArg(); + x.setBinary_arg(ByteBuffer.wrap(value)); + return x; + } + + public static JavaObjectArg double_arg(double value) { + JavaObjectArg x = new JavaObjectArg(); + x.setDouble_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 { + 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 getInt_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 setInt_arg(int value) { + setField_ = _Fields.INT_ARG; + value_ = value; + } + + public long getLong_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 setLong_arg(long value) { + setField_ = _Fields.LONG_ARG; + value_ = value; + } + + public String getString_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 setString_arg(String value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.STRING_ARG; + value_ = value; + } + + public boolean getBool_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 setBool_arg(boolean value) { + setField_ = _Fields.BOOL_ARG; + value_ = value; + } + + public byte[] getBinary_arg() { + setBinary_arg(org.apache.thrift.TBaseHelper.rightSize(bufferForBinary_arg())); + ByteBuffer b = bufferForBinary_arg(); + return b == null ? null : b.array(); + } + + public ByteBuffer bufferForBinary_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 setBinary_arg(byte[] value) { + setBinary_arg(ByteBuffer.wrap(value)); + } + + public void setBinary_arg(ByteBuffer value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.BINARY_ARG; + value_ = value; + } + + public double getDouble_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 setDouble_arg(double value) { + setField_ = _Fields.DOUBLE_ARG; + value_ = value; + } + + public boolean isSetInt_arg() { + return setField_ == _Fields.INT_ARG; + } + + + public boolean isSetLong_arg() { + return setField_ == _Fields.LONG_ARG; + } + + + public boolean isSetString_arg() { + return setField_ == _Fields.STRING_ARG; + } + + + public boolean isSetBool_arg() { + return setField_ == _Fields.BOOL_ARG; + } + + + public boolean isSetBinary_arg() { + return setField_ == _Fields.BINARY_ARG; + } + + + public boolean isSetDouble_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() { + HashCodeBuilder hcb = new HashCodeBuilder(); + hcb.append(this.getClass().getName()); + org.apache.thrift.TFieldIdEnum setField = getSetField(); + if (setField != null) { + hcb.append(setField.getThriftFieldId()); + Object value = getFieldValue(); + if (value instanceof org.apache.thrift.TEnum) { + hcb.append(((org.apache.thrift.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.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/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/KillOptions.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/KillOptions.java new file mode 100644 index 000000000..a492adc77 --- /dev/null +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/KillOptions.java @@ -0,0 +1,389 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package com.intel.hibench.streambench.storm.thrift.generated; + +import org.apache.commons.lang.builder.HashCodeBuilder; +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 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.thrift.TBase, java.io.Serializable, Cloneable { + 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()); + } + + public 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 _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() { + setWait_secsIsSet(false); + this.wait_secs = 0; + } + + public int getWait_secs() { + return this.wait_secs; + } + + public KillOptions setWait_secs(int wait_secs) { + this.wait_secs = wait_secs; + setWait_secsIsSet(true); + return this; + } + + public void unsetWait_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 isSetWait_secs() { + return EncodingUtils.testBit(__isset_bitfield, __WAIT_SECS_ISSET_ID); + } + + public void setWait_secsIsSet(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) { + unsetWait_secs(); + } else { + setWait_secs((Integer)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case WAIT_SECS: + return Integer.valueOf(getWait_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 isSetWait_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.isSetWait_secs(); + boolean that_present_wait_secs = true && that.isSetWait_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 && (isSetWait_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(isSetWait_secs()).compareTo(typedOther.isSetWait_secs()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetWait_secs()) { + lastComparison = org.apache.thrift.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.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 (isSetWait_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.setWait_secsIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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.isSetWait_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.isSetWait_secs()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetWait_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.setWait_secsIsSet(true); + } + } + } + +} + diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/Nimbus.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/Nimbus.java new file mode 100644 index 000000000..25f1f5926 --- /dev/null +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/Nimbus.java @@ -0,0 +1,15944 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package com.intel.hibench.streambench.storm.thrift.generated; + +import org.apache.commons.lang.builder.HashCodeBuilder; +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 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, org.apache.thrift.TException; + + public void submitTopologyWithOpts(String name, String uploadedJarLocation, String jsonConf, StormTopology topology, SubmitOptions options) throws AlreadyAliveException, InvalidTopologyException, 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 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 String getNimbusConf() throws org.apache.thrift.TException; + + public ClusterSummary getClusterInfo() throws org.apache.thrift.TException; + + public TopologyInfo getTopologyInfo(String id) throws NotAliveException, org.apache.thrift.TException; + + public String getTopologyConf(String id) 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 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 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 getNimbusConf(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 getTopologyInfo(String id, 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 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 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, 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.setName(name); + args.setUploadedJarLocation(uploadedJarLocation); + args.setJsonConf(jsonConf); + args.setTopology(topology); + sendBase("submitTopology", args); + } + + public void recv_submitTopology() throws AlreadyAliveException, InvalidTopologyException, 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; + } + return; + } + + public void submitTopologyWithOpts(String name, String uploadedJarLocation, String jsonConf, StormTopology topology, SubmitOptions options) throws AlreadyAliveException, InvalidTopologyException, 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.setName(name); + args.setUploadedJarLocation(uploadedJarLocation); + args.setJsonConf(jsonConf); + args.setTopology(topology); + args.setOptions(options); + sendBase("submitTopologyWithOpts", args); + } + + public void recv_submitTopologyWithOpts() throws AlreadyAliveException, InvalidTopologyException, 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; + } + 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.setName(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.setName(name); + args.setOptions(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.setName(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.setName(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.setName(name); + args.setOptions(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 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.isSetSuccess()) { + 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.setLocation(location); + args.setChunk(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.setLocation(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.setFile(file); + sendBase("beginFileDownload", args); + } + + public String recv_beginFileDownload() throws org.apache.thrift.TException + { + beginFileDownload_result result = new beginFileDownload_result(); + receiveBase(result, "beginFileDownload"); + if (result.isSetSuccess()) { + 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.setId(id); + sendBase("downloadChunk", args); + } + + public ByteBuffer recv_downloadChunk() throws org.apache.thrift.TException + { + downloadChunk_result result = new downloadChunk_result(); + receiveBase(result, "downloadChunk"); + if (result.isSetSuccess()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "downloadChunk failed: unknown result"); + } + + 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.isSetSuccess()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getNimbusConf 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.isSetSuccess()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getClusterInfo 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.setId(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.isSetSuccess()) { + 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 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.setId(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.isSetSuccess()) { + 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 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.setId(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.isSetSuccess()) { + 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.setId(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.isSetSuccess()) { + 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 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.setName(name); + args.setUploadedJarLocation(uploadedJarLocation); + args.setJsonConf(jsonConf); + args.setTopology(topology); + args.write(prot); + prot.writeMessageEnd(); + } + + public void getResult() throws AlreadyAliveException, 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_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.setName(name); + args.setUploadedJarLocation(uploadedJarLocation); + args.setJsonConf(jsonConf); + args.setTopology(topology); + args.setOptions(options); + args.write(prot); + prot.writeMessageEnd(); + } + + public void getResult() throws AlreadyAliveException, 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_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.setName(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.setName(name); + args.setOptions(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.setName(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.setName(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.setName(name); + args.setOptions(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 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.setLocation(location); + args.setChunk(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.setLocation(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.setFile(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.setId(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 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 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 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.setId(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 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.setId(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 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.setId(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.setId(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 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("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("getTopologyConf", new getTopologyConf()); + processMap.put("getTopology", new getTopology()); + processMap.put("getUserTopology", new getUserTopology()); + 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; + } + 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; + } + 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 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 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 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 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 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 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 submitTopology_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + 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()); + } + + public String name; // required + public String uploadedJarLocation; // required + public String jsonConf; // required + public 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.isSetName()) { + this.name = other.name; + } + if (other.isSetUploadedJarLocation()) { + this.uploadedJarLocation = other.uploadedJarLocation; + } + if (other.isSetJsonConf()) { + this.jsonConf = other.jsonConf; + } + if (other.isSetTopology()) { + 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 getName() { + return this.name; + } + + public submitTopology_args setName(String name) { + this.name = name; + return this; + } + + public void unsetName() { + this.name = null; + } + + /** Returns true if field name is set (has been assigned a value) and false otherwise */ + public boolean isSetName() { + return this.name != null; + } + + public void setNameIsSet(boolean value) { + if (!value) { + this.name = null; + } + } + + public String getUploadedJarLocation() { + return this.uploadedJarLocation; + } + + public submitTopology_args setUploadedJarLocation(String uploadedJarLocation) { + this.uploadedJarLocation = uploadedJarLocation; + return this; + } + + public void unsetUploadedJarLocation() { + this.uploadedJarLocation = null; + } + + /** Returns true if field uploadedJarLocation is set (has been assigned a value) and false otherwise */ + public boolean isSetUploadedJarLocation() { + return this.uploadedJarLocation != null; + } + + public void setUploadedJarLocationIsSet(boolean value) { + if (!value) { + this.uploadedJarLocation = null; + } + } + + public String getJsonConf() { + return this.jsonConf; + } + + public submitTopology_args setJsonConf(String jsonConf) { + this.jsonConf = jsonConf; + return this; + } + + public void unsetJsonConf() { + this.jsonConf = null; + } + + /** Returns true if field jsonConf is set (has been assigned a value) and false otherwise */ + public boolean isSetJsonConf() { + return this.jsonConf != null; + } + + public void setJsonConfIsSet(boolean value) { + if (!value) { + this.jsonConf = null; + } + } + + public StormTopology getTopology() { + return this.topology; + } + + public submitTopology_args setTopology(StormTopology topology) { + this.topology = topology; + return this; + } + + public void unsetTopology() { + this.topology = null; + } + + /** Returns true if field topology is set (has been assigned a value) and false otherwise */ + public boolean isSetTopology() { + return this.topology != null; + } + + public void setTopologyIsSet(boolean value) { + if (!value) { + this.topology = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case NAME: + if (value == null) { + unsetName(); + } else { + setName((String)value); + } + break; + + case UPLOADED_JAR_LOCATION: + if (value == null) { + unsetUploadedJarLocation(); + } else { + setUploadedJarLocation((String)value); + } + break; + + case JSON_CONF: + if (value == null) { + unsetJsonConf(); + } else { + setJsonConf((String)value); + } + break; + + case TOPOLOGY: + if (value == null) { + unsetTopology(); + } else { + setTopology((StormTopology)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case NAME: + return getName(); + + case UPLOADED_JAR_LOCATION: + return getUploadedJarLocation(); + + case JSON_CONF: + return getJsonConf(); + + case TOPOLOGY: + return getTopology(); + + } + 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 isSetName(); + case UPLOADED_JAR_LOCATION: + return isSetUploadedJarLocation(); + case JSON_CONF: + return isSetJsonConf(); + case TOPOLOGY: + return isSetTopology(); + } + 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.isSetName(); + boolean that_present_name = true && that.isSetName(); + 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.isSetUploadedJarLocation(); + boolean that_present_uploadedJarLocation = true && that.isSetUploadedJarLocation(); + 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.isSetJsonConf(); + boolean that_present_jsonConf = true && that.isSetJsonConf(); + 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.isSetTopology(); + boolean that_present_topology = true && that.isSetTopology(); + 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 && (isSetName()); + builder.append(present_name); + if (present_name) + builder.append(name); + + boolean present_uploadedJarLocation = true && (isSetUploadedJarLocation()); + builder.append(present_uploadedJarLocation); + if (present_uploadedJarLocation) + builder.append(uploadedJarLocation); + + boolean present_jsonConf = true && (isSetJsonConf()); + builder.append(present_jsonConf); + if (present_jsonConf) + builder.append(jsonConf); + + boolean present_topology = true && (isSetTopology()); + 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(isSetName()).compareTo(typedOther.isSetName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.name, typedOther.name); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetUploadedJarLocation()).compareTo(typedOther.isSetUploadedJarLocation()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetUploadedJarLocation()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.uploadedJarLocation, typedOther.uploadedJarLocation); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetJsonConf()).compareTo(typedOther.isSetJsonConf()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetJsonConf()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.jsonConf, typedOther.jsonConf); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetTopology()).compareTo(typedOther.isSetTopology()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetTopology()) { + lastComparison = org.apache.thrift.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.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.setNameIsSet(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.setUploadedJarLocationIsSet(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.setJsonConfIsSet(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.setTopologyIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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.isSetName()) { + optionals.set(0); + } + if (struct.isSetUploadedJarLocation()) { + optionals.set(1); + } + if (struct.isSetJsonConf()) { + optionals.set(2); + } + if (struct.isSetTopology()) { + optionals.set(3); + } + oprot.writeBitSet(optionals, 4); + if (struct.isSetName()) { + oprot.writeString(struct.name); + } + if (struct.isSetUploadedJarLocation()) { + oprot.writeString(struct.uploadedJarLocation); + } + if (struct.isSetJsonConf()) { + oprot.writeString(struct.jsonConf); + } + if (struct.isSetTopology()) { + 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.setNameIsSet(true); + } + if (incoming.get(1)) { + struct.uploadedJarLocation = iprot.readString(); + struct.setUploadedJarLocationIsSet(true); + } + if (incoming.get(2)) { + struct.jsonConf = iprot.readString(); + struct.setJsonConfIsSet(true); + } + if (incoming.get(3)) { + struct.topology = new StormTopology(); + struct.topology.read(iprot); + struct.setTopologyIsSet(true); + } + } + } + + } + + public static class submitTopology_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + 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 Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new submitTopology_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new submitTopology_resultTupleSchemeFactory()); + } + + public AlreadyAliveException e; // required + public 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(submitTopology_result.class, metaDataMap); + } + + public submitTopology_result() { + } + + public submitTopology_result( + AlreadyAliveException e, + InvalidTopologyException ite) + { + this(); + this.e = e; + this.ite = ite; + } + + /** + * Performs a deep copy on other. + */ + public submitTopology_result(submitTopology_result other) { + if (other.isSetE()) { + this.e = new AlreadyAliveException(other.e); + } + if (other.isSetIte()) { + this.ite = new InvalidTopologyException(other.ite); + } + } + + public submitTopology_result deepCopy() { + return new submitTopology_result(this); + } + + @Override + public void clear() { + this.e = null; + this.ite = null; + } + + public AlreadyAliveException getE() { + return this.e; + } + + public submitTopology_result setE(AlreadyAliveException e) { + this.e = e; + return this; + } + + public void unsetE() { + this.e = null; + } + + /** Returns true if field e is set (has been assigned a value) and false otherwise */ + public boolean isSetE() { + return this.e != null; + } + + public void setEIsSet(boolean value) { + if (!value) { + this.e = null; + } + } + + public InvalidTopologyException getIte() { + return this.ite; + } + + public submitTopology_result setIte(InvalidTopologyException ite) { + this.ite = ite; + return this; + } + + public void unsetIte() { + this.ite = null; + } + + /** Returns true if field ite is set (has been assigned a value) and false otherwise */ + public boolean isSetIte() { + return this.ite != null; + } + + public void setIteIsSet(boolean value) { + if (!value) { + this.ite = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case E: + if (value == null) { + unsetE(); + } else { + setE((AlreadyAliveException)value); + } + break; + + case ITE: + if (value == null) { + unsetIte(); + } else { + setIte((InvalidTopologyException)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case E: + return getE(); + + case ITE: + return getIte(); + + } + 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 isSetE(); + case ITE: + return isSetIte(); + } + 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.isSetE(); + boolean that_present_e = true && that.isSetE(); + 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.isSetIte(); + boolean that_present_ite = true && that.isSetIte(); + 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 && (isSetE()); + builder.append(present_e); + if (present_e) + builder.append(e); + + boolean present_ite = true && (isSetIte()); + builder.append(present_ite); + if (present_ite) + builder.append(ite); + + 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(isSetE()).compareTo(typedOther.isSetE()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetE()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.e, typedOther.e); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetIte()).compareTo(typedOther.isSetIte()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetIte()) { + lastComparison = org.apache.thrift.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.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; + 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.setEIsSet(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.setIteIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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(); + } + 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.isSetE()) { + optionals.set(0); + } + if (struct.isSetIte()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.isSetE()) { + struct.e.write(oprot); + } + if (struct.isSetIte()) { + struct.ite.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(2); + if (incoming.get(0)) { + struct.e = new AlreadyAliveException(); + struct.e.read(iprot); + struct.setEIsSet(true); + } + if (incoming.get(1)) { + struct.ite = new InvalidTopologyException(); + struct.ite.read(iprot); + struct.setIteIsSet(true); + } + } + } + + } + + public static class submitTopologyWithOpts_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + 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()); + } + + public String name; // required + public String uploadedJarLocation; // required + public String jsonConf; // required + public StormTopology topology; // required + public 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.isSetName()) { + this.name = other.name; + } + if (other.isSetUploadedJarLocation()) { + this.uploadedJarLocation = other.uploadedJarLocation; + } + if (other.isSetJsonConf()) { + this.jsonConf = other.jsonConf; + } + if (other.isSetTopology()) { + this.topology = new StormTopology(other.topology); + } + if (other.isSetOptions()) { + 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 getName() { + return this.name; + } + + public submitTopologyWithOpts_args setName(String name) { + this.name = name; + return this; + } + + public void unsetName() { + this.name = null; + } + + /** Returns true if field name is set (has been assigned a value) and false otherwise */ + public boolean isSetName() { + return this.name != null; + } + + public void setNameIsSet(boolean value) { + if (!value) { + this.name = null; + } + } + + public String getUploadedJarLocation() { + return this.uploadedJarLocation; + } + + public submitTopologyWithOpts_args setUploadedJarLocation(String uploadedJarLocation) { + this.uploadedJarLocation = uploadedJarLocation; + return this; + } + + public void unsetUploadedJarLocation() { + this.uploadedJarLocation = null; + } + + /** Returns true if field uploadedJarLocation is set (has been assigned a value) and false otherwise */ + public boolean isSetUploadedJarLocation() { + return this.uploadedJarLocation != null; + } + + public void setUploadedJarLocationIsSet(boolean value) { + if (!value) { + this.uploadedJarLocation = null; + } + } + + public String getJsonConf() { + return this.jsonConf; + } + + public submitTopologyWithOpts_args setJsonConf(String jsonConf) { + this.jsonConf = jsonConf; + return this; + } + + public void unsetJsonConf() { + this.jsonConf = null; + } + + /** Returns true if field jsonConf is set (has been assigned a value) and false otherwise */ + public boolean isSetJsonConf() { + return this.jsonConf != null; + } + + public void setJsonConfIsSet(boolean value) { + if (!value) { + this.jsonConf = null; + } + } + + public StormTopology getTopology() { + return this.topology; + } + + public submitTopologyWithOpts_args setTopology(StormTopology topology) { + this.topology = topology; + return this; + } + + public void unsetTopology() { + this.topology = null; + } + + /** Returns true if field topology is set (has been assigned a value) and false otherwise */ + public boolean isSetTopology() { + return this.topology != null; + } + + public void setTopologyIsSet(boolean value) { + if (!value) { + this.topology = null; + } + } + + public SubmitOptions getOptions() { + return this.options; + } + + public submitTopologyWithOpts_args setOptions(SubmitOptions options) { + this.options = options; + return this; + } + + public void unsetOptions() { + this.options = null; + } + + /** Returns true if field options is set (has been assigned a value) and false otherwise */ + public boolean isSetOptions() { + return this.options != null; + } + + public void setOptionsIsSet(boolean value) { + if (!value) { + this.options = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case NAME: + if (value == null) { + unsetName(); + } else { + setName((String)value); + } + break; + + case UPLOADED_JAR_LOCATION: + if (value == null) { + unsetUploadedJarLocation(); + } else { + setUploadedJarLocation((String)value); + } + break; + + case JSON_CONF: + if (value == null) { + unsetJsonConf(); + } else { + setJsonConf((String)value); + } + break; + + case TOPOLOGY: + if (value == null) { + unsetTopology(); + } else { + setTopology((StormTopology)value); + } + break; + + case OPTIONS: + if (value == null) { + unsetOptions(); + } else { + setOptions((SubmitOptions)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case NAME: + return getName(); + + case UPLOADED_JAR_LOCATION: + return getUploadedJarLocation(); + + case JSON_CONF: + return getJsonConf(); + + case TOPOLOGY: + return getTopology(); + + case OPTIONS: + return getOptions(); + + } + 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 isSetName(); + case UPLOADED_JAR_LOCATION: + return isSetUploadedJarLocation(); + case JSON_CONF: + return isSetJsonConf(); + case TOPOLOGY: + return isSetTopology(); + case OPTIONS: + return isSetOptions(); + } + 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.isSetName(); + boolean that_present_name = true && that.isSetName(); + 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.isSetUploadedJarLocation(); + boolean that_present_uploadedJarLocation = true && that.isSetUploadedJarLocation(); + 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.isSetJsonConf(); + boolean that_present_jsonConf = true && that.isSetJsonConf(); + 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.isSetTopology(); + boolean that_present_topology = true && that.isSetTopology(); + 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.isSetOptions(); + boolean that_present_options = true && that.isSetOptions(); + 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 && (isSetName()); + builder.append(present_name); + if (present_name) + builder.append(name); + + boolean present_uploadedJarLocation = true && (isSetUploadedJarLocation()); + builder.append(present_uploadedJarLocation); + if (present_uploadedJarLocation) + builder.append(uploadedJarLocation); + + boolean present_jsonConf = true && (isSetJsonConf()); + builder.append(present_jsonConf); + if (present_jsonConf) + builder.append(jsonConf); + + boolean present_topology = true && (isSetTopology()); + builder.append(present_topology); + if (present_topology) + builder.append(topology); + + boolean present_options = true && (isSetOptions()); + 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(isSetName()).compareTo(typedOther.isSetName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.name, typedOther.name); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetUploadedJarLocation()).compareTo(typedOther.isSetUploadedJarLocation()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetUploadedJarLocation()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.uploadedJarLocation, typedOther.uploadedJarLocation); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetJsonConf()).compareTo(typedOther.isSetJsonConf()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetJsonConf()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.jsonConf, typedOther.jsonConf); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetTopology()).compareTo(typedOther.isSetTopology()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetTopology()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.topology, typedOther.topology); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetOptions()).compareTo(typedOther.isSetOptions()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetOptions()) { + lastComparison = org.apache.thrift.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.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.setNameIsSet(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.setUploadedJarLocationIsSet(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.setJsonConfIsSet(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.setTopologyIsSet(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.setOptionsIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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.isSetName()) { + optionals.set(0); + } + if (struct.isSetUploadedJarLocation()) { + optionals.set(1); + } + if (struct.isSetJsonConf()) { + optionals.set(2); + } + if (struct.isSetTopology()) { + optionals.set(3); + } + if (struct.isSetOptions()) { + optionals.set(4); + } + oprot.writeBitSet(optionals, 5); + if (struct.isSetName()) { + oprot.writeString(struct.name); + } + if (struct.isSetUploadedJarLocation()) { + oprot.writeString(struct.uploadedJarLocation); + } + if (struct.isSetJsonConf()) { + oprot.writeString(struct.jsonConf); + } + if (struct.isSetTopology()) { + struct.topology.write(oprot); + } + if (struct.isSetOptions()) { + 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.setNameIsSet(true); + } + if (incoming.get(1)) { + struct.uploadedJarLocation = iprot.readString(); + struct.setUploadedJarLocationIsSet(true); + } + if (incoming.get(2)) { + struct.jsonConf = iprot.readString(); + struct.setJsonConfIsSet(true); + } + if (incoming.get(3)) { + struct.topology = new StormTopology(); + struct.topology.read(iprot); + struct.setTopologyIsSet(true); + } + if (incoming.get(4)) { + struct.options = new SubmitOptions(); + struct.options.read(iprot); + struct.setOptionsIsSet(true); + } + } + } + + } + + public static class submitTopologyWithOpts_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + 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 Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new submitTopologyWithOpts_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new submitTopologyWithOpts_resultTupleSchemeFactory()); + } + + public AlreadyAliveException e; // required + public 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(submitTopologyWithOpts_result.class, metaDataMap); + } + + public submitTopologyWithOpts_result() { + } + + public submitTopologyWithOpts_result( + AlreadyAliveException e, + InvalidTopologyException ite) + { + this(); + this.e = e; + this.ite = ite; + } + + /** + * Performs a deep copy on other. + */ + public submitTopologyWithOpts_result(submitTopologyWithOpts_result other) { + if (other.isSetE()) { + this.e = new AlreadyAliveException(other.e); + } + if (other.isSetIte()) { + this.ite = new InvalidTopologyException(other.ite); + } + } + + public submitTopologyWithOpts_result deepCopy() { + return new submitTopologyWithOpts_result(this); + } + + @Override + public void clear() { + this.e = null; + this.ite = null; + } + + public AlreadyAliveException getE() { + return this.e; + } + + public submitTopologyWithOpts_result setE(AlreadyAliveException e) { + this.e = e; + return this; + } + + public void unsetE() { + this.e = null; + } + + /** Returns true if field e is set (has been assigned a value) and false otherwise */ + public boolean isSetE() { + return this.e != null; + } + + public void setEIsSet(boolean value) { + if (!value) { + this.e = null; + } + } + + public InvalidTopologyException getIte() { + return this.ite; + } + + public submitTopologyWithOpts_result setIte(InvalidTopologyException ite) { + this.ite = ite; + return this; + } + + public void unsetIte() { + this.ite = null; + } + + /** Returns true if field ite is set (has been assigned a value) and false otherwise */ + public boolean isSetIte() { + return this.ite != null; + } + + public void setIteIsSet(boolean value) { + if (!value) { + this.ite = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case E: + if (value == null) { + unsetE(); + } else { + setE((AlreadyAliveException)value); + } + break; + + case ITE: + if (value == null) { + unsetIte(); + } else { + setIte((InvalidTopologyException)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case E: + return getE(); + + case ITE: + return getIte(); + + } + 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 isSetE(); + case ITE: + return isSetIte(); + } + 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.isSetE(); + boolean that_present_e = true && that.isSetE(); + 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.isSetIte(); + boolean that_present_ite = true && that.isSetIte(); + 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 && (isSetE()); + builder.append(present_e); + if (present_e) + builder.append(e); + + boolean present_ite = true && (isSetIte()); + builder.append(present_ite); + if (present_ite) + builder.append(ite); + + 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(isSetE()).compareTo(typedOther.isSetE()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetE()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.e, typedOther.e); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetIte()).compareTo(typedOther.isSetIte()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetIte()) { + lastComparison = org.apache.thrift.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.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; + 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.setEIsSet(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.setIteIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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(); + } + 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.isSetE()) { + optionals.set(0); + } + if (struct.isSetIte()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.isSetE()) { + struct.e.write(oprot); + } + if (struct.isSetIte()) { + struct.ite.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(2); + if (incoming.get(0)) { + struct.e = new AlreadyAliveException(); + struct.e.read(iprot); + struct.setEIsSet(true); + } + if (incoming.get(1)) { + struct.ite = new InvalidTopologyException(); + struct.ite.read(iprot); + struct.setIteIsSet(true); + } + } + } + + } + + public static class killTopology_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + 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()); + } + + public 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.isSetName()) { + this.name = other.name; + } + } + + public killTopology_args deepCopy() { + return new killTopology_args(this); + } + + @Override + public void clear() { + this.name = null; + } + + public String getName() { + return this.name; + } + + public killTopology_args setName(String name) { + this.name = name; + return this; + } + + public void unsetName() { + this.name = null; + } + + /** Returns true if field name is set (has been assigned a value) and false otherwise */ + public boolean isSetName() { + return this.name != null; + } + + public void setNameIsSet(boolean value) { + if (!value) { + this.name = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case NAME: + if (value == null) { + unsetName(); + } else { + setName((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case NAME: + return getName(); + + } + 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 isSetName(); + } + 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.isSetName(); + boolean that_present_name = true && that.isSetName(); + 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 && (isSetName()); + 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(isSetName()).compareTo(typedOther.isSetName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetName()) { + lastComparison = org.apache.thrift.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.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.setNameIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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.isSetName()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetName()) { + 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.setNameIsSet(true); + } + } + } + + } + + public static class killTopology_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + 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()); + } + + public 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.isSetE()) { + this.e = new NotAliveException(other.e); + } + } + + public killTopology_result deepCopy() { + return new killTopology_result(this); + } + + @Override + public void clear() { + this.e = null; + } + + public NotAliveException getE() { + return this.e; + } + + public killTopology_result setE(NotAliveException e) { + this.e = e; + return this; + } + + public void unsetE() { + this.e = null; + } + + /** Returns true if field e is set (has been assigned a value) and false otherwise */ + public boolean isSetE() { + return this.e != null; + } + + public void setEIsSet(boolean value) { + if (!value) { + this.e = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case E: + if (value == null) { + unsetE(); + } else { + setE((NotAliveException)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case E: + return getE(); + + } + 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 isSetE(); + } + 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.isSetE(); + boolean that_present_e = true && that.isSetE(); + 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 && (isSetE()); + 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(isSetE()).compareTo(typedOther.isSetE()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetE()) { + lastComparison = org.apache.thrift.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.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.setEIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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.isSetE()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetE()) { + 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.setEIsSet(true); + } + } + } + + } + + public static class killTopologyWithOpts_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + 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()); + } + + public String name; // required + public 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.isSetName()) { + this.name = other.name; + } + if (other.isSetOptions()) { + 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 getName() { + return this.name; + } + + public killTopologyWithOpts_args setName(String name) { + this.name = name; + return this; + } + + public void unsetName() { + this.name = null; + } + + /** Returns true if field name is set (has been assigned a value) and false otherwise */ + public boolean isSetName() { + return this.name != null; + } + + public void setNameIsSet(boolean value) { + if (!value) { + this.name = null; + } + } + + public KillOptions getOptions() { + return this.options; + } + + public killTopologyWithOpts_args setOptions(KillOptions options) { + this.options = options; + return this; + } + + public void unsetOptions() { + this.options = null; + } + + /** Returns true if field options is set (has been assigned a value) and false otherwise */ + public boolean isSetOptions() { + return this.options != null; + } + + public void setOptionsIsSet(boolean value) { + if (!value) { + this.options = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case NAME: + if (value == null) { + unsetName(); + } else { + setName((String)value); + } + break; + + case OPTIONS: + if (value == null) { + unsetOptions(); + } else { + setOptions((KillOptions)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case NAME: + return getName(); + + case OPTIONS: + return getOptions(); + + } + 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 isSetName(); + case OPTIONS: + return isSetOptions(); + } + 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.isSetName(); + boolean that_present_name = true && that.isSetName(); + 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.isSetOptions(); + boolean that_present_options = true && that.isSetOptions(); + 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 && (isSetName()); + builder.append(present_name); + if (present_name) + builder.append(name); + + boolean present_options = true && (isSetOptions()); + 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(isSetName()).compareTo(typedOther.isSetName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.name, typedOther.name); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetOptions()).compareTo(typedOther.isSetOptions()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetOptions()) { + lastComparison = org.apache.thrift.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.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.setNameIsSet(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.setOptionsIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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.isSetName()) { + optionals.set(0); + } + if (struct.isSetOptions()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.isSetName()) { + oprot.writeString(struct.name); + } + if (struct.isSetOptions()) { + 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.setNameIsSet(true); + } + if (incoming.get(1)) { + struct.options = new KillOptions(); + struct.options.read(iprot); + struct.setOptionsIsSet(true); + } + } + } + + } + + public static class killTopologyWithOpts_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + 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()); + } + + public 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.isSetE()) { + this.e = new NotAliveException(other.e); + } + } + + public killTopologyWithOpts_result deepCopy() { + return new killTopologyWithOpts_result(this); + } + + @Override + public void clear() { + this.e = null; + } + + public NotAliveException getE() { + return this.e; + } + + public killTopologyWithOpts_result setE(NotAliveException e) { + this.e = e; + return this; + } + + public void unsetE() { + this.e = null; + } + + /** Returns true if field e is set (has been assigned a value) and false otherwise */ + public boolean isSetE() { + return this.e != null; + } + + public void setEIsSet(boolean value) { + if (!value) { + this.e = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case E: + if (value == null) { + unsetE(); + } else { + setE((NotAliveException)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case E: + return getE(); + + } + 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 isSetE(); + } + 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.isSetE(); + boolean that_present_e = true && that.isSetE(); + 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 && (isSetE()); + 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(isSetE()).compareTo(typedOther.isSetE()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetE()) { + lastComparison = org.apache.thrift.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.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.setEIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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.isSetE()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetE()) { + 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.setEIsSet(true); + } + } + } + + } + + public static class activate_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + 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()); + } + + public 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.isSetName()) { + this.name = other.name; + } + } + + public activate_args deepCopy() { + return new activate_args(this); + } + + @Override + public void clear() { + this.name = null; + } + + public String getName() { + return this.name; + } + + public activate_args setName(String name) { + this.name = name; + return this; + } + + public void unsetName() { + this.name = null; + } + + /** Returns true if field name is set (has been assigned a value) and false otherwise */ + public boolean isSetName() { + return this.name != null; + } + + public void setNameIsSet(boolean value) { + if (!value) { + this.name = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case NAME: + if (value == null) { + unsetName(); + } else { + setName((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case NAME: + return getName(); + + } + 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 isSetName(); + } + 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.isSetName(); + boolean that_present_name = true && that.isSetName(); + 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 && (isSetName()); + 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(isSetName()).compareTo(typedOther.isSetName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetName()) { + lastComparison = org.apache.thrift.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.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.setNameIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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.isSetName()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetName()) { + 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.setNameIsSet(true); + } + } + } + + } + + public static class activate_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + 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()); + } + + public 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.isSetE()) { + this.e = new NotAliveException(other.e); + } + } + + public activate_result deepCopy() { + return new activate_result(this); + } + + @Override + public void clear() { + this.e = null; + } + + public NotAliveException getE() { + return this.e; + } + + public activate_result setE(NotAliveException e) { + this.e = e; + return this; + } + + public void unsetE() { + this.e = null; + } + + /** Returns true if field e is set (has been assigned a value) and false otherwise */ + public boolean isSetE() { + return this.e != null; + } + + public void setEIsSet(boolean value) { + if (!value) { + this.e = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case E: + if (value == null) { + unsetE(); + } else { + setE((NotAliveException)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case E: + return getE(); + + } + 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 isSetE(); + } + 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.isSetE(); + boolean that_present_e = true && that.isSetE(); + 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 && (isSetE()); + 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(isSetE()).compareTo(typedOther.isSetE()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetE()) { + lastComparison = org.apache.thrift.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.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.setEIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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.isSetE()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetE()) { + 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.setEIsSet(true); + } + } + } + + } + + public static class deactivate_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + 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()); + } + + public 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.isSetName()) { + this.name = other.name; + } + } + + public deactivate_args deepCopy() { + return new deactivate_args(this); + } + + @Override + public void clear() { + this.name = null; + } + + public String getName() { + return this.name; + } + + public deactivate_args setName(String name) { + this.name = name; + return this; + } + + public void unsetName() { + this.name = null; + } + + /** Returns true if field name is set (has been assigned a value) and false otherwise */ + public boolean isSetName() { + return this.name != null; + } + + public void setNameIsSet(boolean value) { + if (!value) { + this.name = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case NAME: + if (value == null) { + unsetName(); + } else { + setName((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case NAME: + return getName(); + + } + 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 isSetName(); + } + 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.isSetName(); + boolean that_present_name = true && that.isSetName(); + 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 && (isSetName()); + 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(isSetName()).compareTo(typedOther.isSetName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetName()) { + lastComparison = org.apache.thrift.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.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.setNameIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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.isSetName()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetName()) { + 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.setNameIsSet(true); + } + } + } + + } + + public static class deactivate_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + 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()); + } + + public 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.isSetE()) { + this.e = new NotAliveException(other.e); + } + } + + public deactivate_result deepCopy() { + return new deactivate_result(this); + } + + @Override + public void clear() { + this.e = null; + } + + public NotAliveException getE() { + return this.e; + } + + public deactivate_result setE(NotAliveException e) { + this.e = e; + return this; + } + + public void unsetE() { + this.e = null; + } + + /** Returns true if field e is set (has been assigned a value) and false otherwise */ + public boolean isSetE() { + return this.e != null; + } + + public void setEIsSet(boolean value) { + if (!value) { + this.e = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case E: + if (value == null) { + unsetE(); + } else { + setE((NotAliveException)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case E: + return getE(); + + } + 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 isSetE(); + } + 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.isSetE(); + boolean that_present_e = true && that.isSetE(); + 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 && (isSetE()); + 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(isSetE()).compareTo(typedOther.isSetE()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetE()) { + lastComparison = org.apache.thrift.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.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.setEIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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.isSetE()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetE()) { + 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.setEIsSet(true); + } + } + } + + } + + public static class rebalance_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + 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()); + } + + public String name; // required + public 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.isSetName()) { + this.name = other.name; + } + if (other.isSetOptions()) { + 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 getName() { + return this.name; + } + + public rebalance_args setName(String name) { + this.name = name; + return this; + } + + public void unsetName() { + this.name = null; + } + + /** Returns true if field name is set (has been assigned a value) and false otherwise */ + public boolean isSetName() { + return this.name != null; + } + + public void setNameIsSet(boolean value) { + if (!value) { + this.name = null; + } + } + + public RebalanceOptions getOptions() { + return this.options; + } + + public rebalance_args setOptions(RebalanceOptions options) { + this.options = options; + return this; + } + + public void unsetOptions() { + this.options = null; + } + + /** Returns true if field options is set (has been assigned a value) and false otherwise */ + public boolean isSetOptions() { + return this.options != null; + } + + public void setOptionsIsSet(boolean value) { + if (!value) { + this.options = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case NAME: + if (value == null) { + unsetName(); + } else { + setName((String)value); + } + break; + + case OPTIONS: + if (value == null) { + unsetOptions(); + } else { + setOptions((RebalanceOptions)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case NAME: + return getName(); + + case OPTIONS: + return getOptions(); + + } + 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 isSetName(); + case OPTIONS: + return isSetOptions(); + } + 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.isSetName(); + boolean that_present_name = true && that.isSetName(); + 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.isSetOptions(); + boolean that_present_options = true && that.isSetOptions(); + 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 && (isSetName()); + builder.append(present_name); + if (present_name) + builder.append(name); + + boolean present_options = true && (isSetOptions()); + 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(isSetName()).compareTo(typedOther.isSetName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.name, typedOther.name); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetOptions()).compareTo(typedOther.isSetOptions()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetOptions()) { + lastComparison = org.apache.thrift.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.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.setNameIsSet(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.setOptionsIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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.isSetName()) { + optionals.set(0); + } + if (struct.isSetOptions()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.isSetName()) { + oprot.writeString(struct.name); + } + if (struct.isSetOptions()) { + 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.setNameIsSet(true); + } + if (incoming.get(1)) { + struct.options = new RebalanceOptions(); + struct.options.read(iprot); + struct.setOptionsIsSet(true); + } + } + } + + } + + public static class rebalance_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + 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()); + } + + public NotAliveException e; // required + public 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.isSetE()) { + this.e = new NotAliveException(other.e); + } + if (other.isSetIte()) { + 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 getE() { + return this.e; + } + + public rebalance_result setE(NotAliveException e) { + this.e = e; + return this; + } + + public void unsetE() { + this.e = null; + } + + /** Returns true if field e is set (has been assigned a value) and false otherwise */ + public boolean isSetE() { + return this.e != null; + } + + public void setEIsSet(boolean value) { + if (!value) { + this.e = null; + } + } + + public InvalidTopologyException getIte() { + return this.ite; + } + + public rebalance_result setIte(InvalidTopologyException ite) { + this.ite = ite; + return this; + } + + public void unsetIte() { + this.ite = null; + } + + /** Returns true if field ite is set (has been assigned a value) and false otherwise */ + public boolean isSetIte() { + return this.ite != null; + } + + public void setIteIsSet(boolean value) { + if (!value) { + this.ite = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case E: + if (value == null) { + unsetE(); + } else { + setE((NotAliveException)value); + } + break; + + case ITE: + if (value == null) { + unsetIte(); + } else { + setIte((InvalidTopologyException)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case E: + return getE(); + + case ITE: + return getIte(); + + } + 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 isSetE(); + case ITE: + return isSetIte(); + } + 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.isSetE(); + boolean that_present_e = true && that.isSetE(); + 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.isSetIte(); + boolean that_present_ite = true && that.isSetIte(); + 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 && (isSetE()); + builder.append(present_e); + if (present_e) + builder.append(e); + + boolean present_ite = true && (isSetIte()); + 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(isSetE()).compareTo(typedOther.isSetE()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetE()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.e, typedOther.e); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetIte()).compareTo(typedOther.isSetIte()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetIte()) { + lastComparison = org.apache.thrift.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.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.setEIsSet(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.setIteIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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.isSetE()) { + optionals.set(0); + } + if (struct.isSetIte()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.isSetE()) { + struct.e.write(oprot); + } + if (struct.isSetIte()) { + 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.setEIsSet(true); + } + if (incoming.get(1)) { + struct.ite = new InvalidTopologyException(); + struct.ite.read(iprot); + struct.setIteIsSet(true); + } + } + } + + } + + public static class beginFileUpload_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + 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() { + 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.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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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 { + 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()); + } + + public 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.isSetSuccess()) { + this.success = other.success; + } + } + + public beginFileUpload_result deepCopy() { + return new beginFileUpload_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public String getSuccess() { + return this.success; + } + + public beginFileUpload_result setSuccess(String success) { + this.success = success; + return this; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + } + 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 isSetSuccess(); + } + 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.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + 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 && (isSetSuccess()); + 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(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.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.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.setSuccessIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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.isSetSuccess()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetSuccess()) { + 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.setSuccessIsSet(true); + } + } + } + + } + + public static class uploadChunk_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + 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()); + } + + public String location; // required + public 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 = chunk; + } + + /** + * Performs a deep copy on other. + */ + public uploadChunk_args(uploadChunk_args other) { + if (other.isSetLocation()) { + this.location = other.location; + } + if (other.isSetChunk()) { + 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 getLocation() { + return this.location; + } + + public uploadChunk_args setLocation(String location) { + this.location = location; + return this; + } + + public void unsetLocation() { + this.location = null; + } + + /** Returns true if field location is set (has been assigned a value) and false otherwise */ + public boolean isSetLocation() { + return this.location != null; + } + + public void setLocationIsSet(boolean value) { + if (!value) { + this.location = null; + } + } + + public byte[] getChunk() { + setChunk(org.apache.thrift.TBaseHelper.rightSize(chunk)); + return chunk == null ? null : chunk.array(); + } + + public ByteBuffer bufferForChunk() { + return chunk; + } + + public uploadChunk_args setChunk(byte[] chunk) { + setChunk(chunk == null ? (ByteBuffer)null : ByteBuffer.wrap(chunk)); + return this; + } + + public uploadChunk_args setChunk(ByteBuffer chunk) { + this.chunk = chunk; + return this; + } + + public void unsetChunk() { + this.chunk = null; + } + + /** Returns true if field chunk is set (has been assigned a value) and false otherwise */ + public boolean isSetChunk() { + return this.chunk != null; + } + + public void setChunkIsSet(boolean value) { + if (!value) { + this.chunk = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case LOCATION: + if (value == null) { + unsetLocation(); + } else { + setLocation((String)value); + } + break; + + case CHUNK: + if (value == null) { + unsetChunk(); + } else { + setChunk((ByteBuffer)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case LOCATION: + return getLocation(); + + case CHUNK: + return getChunk(); + + } + 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 isSetLocation(); + case CHUNK: + return isSetChunk(); + } + 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.isSetLocation(); + boolean that_present_location = true && that.isSetLocation(); + 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.isSetChunk(); + boolean that_present_chunk = true && that.isSetChunk(); + 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 && (isSetLocation()); + builder.append(present_location); + if (present_location) + builder.append(location); + + boolean present_chunk = true && (isSetChunk()); + 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(isSetLocation()).compareTo(typedOther.isSetLocation()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetLocation()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.location, typedOther.location); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetChunk()).compareTo(typedOther.isSetChunk()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetChunk()) { + lastComparison = org.apache.thrift.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.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.setLocationIsSet(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.setChunkIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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.isSetLocation()) { + optionals.set(0); + } + if (struct.isSetChunk()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.isSetLocation()) { + oprot.writeString(struct.location); + } + if (struct.isSetChunk()) { + 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.setLocationIsSet(true); + } + if (incoming.get(1)) { + struct.chunk = iprot.readBinary(); + struct.setChunkIsSet(true); + } + } + } + + } + + public static class uploadChunk_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + 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() { + 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.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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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 { + 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()); + } + + public 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.isSetLocation()) { + this.location = other.location; + } + } + + public finishFileUpload_args deepCopy() { + return new finishFileUpload_args(this); + } + + @Override + public void clear() { + this.location = null; + } + + public String getLocation() { + return this.location; + } + + public finishFileUpload_args setLocation(String location) { + this.location = location; + return this; + } + + public void unsetLocation() { + this.location = null; + } + + /** Returns true if field location is set (has been assigned a value) and false otherwise */ + public boolean isSetLocation() { + return this.location != null; + } + + public void setLocationIsSet(boolean value) { + if (!value) { + this.location = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case LOCATION: + if (value == null) { + unsetLocation(); + } else { + setLocation((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case LOCATION: + return getLocation(); + + } + 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 isSetLocation(); + } + 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.isSetLocation(); + boolean that_present_location = true && that.isSetLocation(); + 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 && (isSetLocation()); + 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(isSetLocation()).compareTo(typedOther.isSetLocation()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetLocation()) { + lastComparison = org.apache.thrift.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.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.setLocationIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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.isSetLocation()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetLocation()) { + 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.setLocationIsSet(true); + } + } + } + + } + + public static class finishFileUpload_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + 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() { + 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.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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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 { + 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()); + } + + public 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.isSetFile()) { + this.file = other.file; + } + } + + public beginFileDownload_args deepCopy() { + return new beginFileDownload_args(this); + } + + @Override + public void clear() { + this.file = null; + } + + public String getFile() { + return this.file; + } + + public beginFileDownload_args setFile(String file) { + this.file = file; + return this; + } + + public void unsetFile() { + this.file = null; + } + + /** Returns true if field file is set (has been assigned a value) and false otherwise */ + public boolean isSetFile() { + return this.file != null; + } + + public void setFileIsSet(boolean value) { + if (!value) { + this.file = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case FILE: + if (value == null) { + unsetFile(); + } else { + setFile((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case FILE: + return getFile(); + + } + 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 isSetFile(); + } + 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.isSetFile(); + boolean that_present_file = true && that.isSetFile(); + 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 && (isSetFile()); + 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(isSetFile()).compareTo(typedOther.isSetFile()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetFile()) { + lastComparison = org.apache.thrift.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.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.setFileIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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.isSetFile()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetFile()) { + 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.setFileIsSet(true); + } + } + } + + } + + public static class beginFileDownload_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + 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()); + } + + public 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.isSetSuccess()) { + this.success = other.success; + } + } + + public beginFileDownload_result deepCopy() { + return new beginFileDownload_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public String getSuccess() { + return this.success; + } + + public beginFileDownload_result setSuccess(String success) { + this.success = success; + return this; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + } + 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 isSetSuccess(); + } + 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.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + 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 && (isSetSuccess()); + 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(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.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.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.setSuccessIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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.isSetSuccess()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetSuccess()) { + 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.setSuccessIsSet(true); + } + } + } + + } + + public static class downloadChunk_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + 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()); + } + + public 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.isSetId()) { + this.id = other.id; + } + } + + public downloadChunk_args deepCopy() { + return new downloadChunk_args(this); + } + + @Override + public void clear() { + this.id = null; + } + + public String getId() { + return this.id; + } + + public downloadChunk_args setId(String id) { + this.id = id; + return this; + } + + public void unsetId() { + this.id = null; + } + + /** Returns true if field id is set (has been assigned a value) and false otherwise */ + public boolean isSetId() { + return this.id != null; + } + + public void setIdIsSet(boolean value) { + if (!value) { + this.id = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case ID: + if (value == null) { + unsetId(); + } else { + setId((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case ID: + return getId(); + + } + 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 isSetId(); + } + 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.isSetId(); + boolean that_present_id = true && that.isSetId(); + 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 && (isSetId()); + 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(isSetId()).compareTo(typedOther.isSetId()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetId()) { + lastComparison = org.apache.thrift.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.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.setIdIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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.isSetId()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetId()) { + 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.setIdIsSet(true); + } + } + } + + } + + public static class downloadChunk_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + 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()); + } + + public 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 = success; + } + + /** + * Performs a deep copy on other. + */ + public downloadChunk_result(downloadChunk_result other) { + if (other.isSetSuccess()) { + 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[] getSuccess() { + setSuccess(org.apache.thrift.TBaseHelper.rightSize(success)); + return success == null ? null : success.array(); + } + + public ByteBuffer bufferForSuccess() { + return success; + } + + public downloadChunk_result setSuccess(byte[] success) { + setSuccess(success == null ? (ByteBuffer)null : ByteBuffer.wrap(success)); + return this; + } + + public downloadChunk_result setSuccess(ByteBuffer success) { + this.success = success; + return this; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((ByteBuffer)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + } + 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 isSetSuccess(); + } + 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.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + 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 && (isSetSuccess()); + 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(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.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.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.setSuccessIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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.isSetSuccess()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetSuccess()) { + 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.setSuccessIsSet(true); + } + } + } + + } + + public static class getNimbusConf_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + 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() { + 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.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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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 { + 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()); + } + + public 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.isSetSuccess()) { + this.success = other.success; + } + } + + public getNimbusConf_result deepCopy() { + return new getNimbusConf_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public String getSuccess() { + return this.success; + } + + public getNimbusConf_result setSuccess(String success) { + this.success = success; + return this; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + } + 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 isSetSuccess(); + } + 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.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + 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 && (isSetSuccess()); + 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(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.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.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.setSuccessIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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.isSetSuccess()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetSuccess()) { + 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.setSuccessIsSet(true); + } + } + } + + } + + public static class getClusterInfo_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + 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() { + 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.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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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 { + 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()); + } + + public 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.isSetSuccess()) { + this.success = new ClusterSummary(other.success); + } + } + + public getClusterInfo_result deepCopy() { + return new getClusterInfo_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public ClusterSummary getSuccess() { + return this.success; + } + + public getClusterInfo_result setSuccess(ClusterSummary success) { + this.success = success; + return this; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((ClusterSummary)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + } + 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 isSetSuccess(); + } + 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.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + 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 && (isSetSuccess()); + 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(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.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.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.setSuccessIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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.isSetSuccess()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetSuccess()) { + 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.setSuccessIsSet(true); + } + } + } + + } + + public static class getTopologyInfo_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + 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()); + } + + public 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.isSetId()) { + this.id = other.id; + } + } + + public getTopologyInfo_args deepCopy() { + return new getTopologyInfo_args(this); + } + + @Override + public void clear() { + this.id = null; + } + + public String getId() { + return this.id; + } + + public getTopologyInfo_args setId(String id) { + this.id = id; + return this; + } + + public void unsetId() { + this.id = null; + } + + /** Returns true if field id is set (has been assigned a value) and false otherwise */ + public boolean isSetId() { + return this.id != null; + } + + public void setIdIsSet(boolean value) { + if (!value) { + this.id = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case ID: + if (value == null) { + unsetId(); + } else { + setId((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case ID: + return getId(); + + } + 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 isSetId(); + } + 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.isSetId(); + boolean that_present_id = true && that.isSetId(); + 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 && (isSetId()); + 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(isSetId()).compareTo(typedOther.isSetId()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetId()) { + lastComparison = org.apache.thrift.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.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.setIdIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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.isSetId()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetId()) { + 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.setIdIsSet(true); + } + } + } + + } + + public static class getTopologyInfo_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + 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()); + } + + public TopologyInfo success; // required + public 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.isSetSuccess()) { + this.success = new TopologyInfo(other.success); + } + if (other.isSetE()) { + 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 getSuccess() { + return this.success; + } + + public getTopologyInfo_result setSuccess(TopologyInfo success) { + this.success = success; + return this; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public NotAliveException getE() { + return this.e; + } + + public getTopologyInfo_result setE(NotAliveException e) { + this.e = e; + return this; + } + + public void unsetE() { + this.e = null; + } + + /** Returns true if field e is set (has been assigned a value) and false otherwise */ + public boolean isSetE() { + return this.e != null; + } + + public void setEIsSet(boolean value) { + if (!value) { + this.e = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((TopologyInfo)value); + } + break; + + case E: + if (value == null) { + unsetE(); + } else { + setE((NotAliveException)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + case E: + return getE(); + + } + 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 isSetSuccess(); + case E: + return isSetE(); + } + 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.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + 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.isSetE(); + boolean that_present_e = true && that.isSetE(); + 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 && (isSetSuccess()); + builder.append(present_success); + if (present_success) + builder.append(success); + + boolean present_e = true && (isSetE()); + 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(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetE()).compareTo(typedOther.isSetE()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetE()) { + lastComparison = org.apache.thrift.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.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.setSuccessIsSet(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.setEIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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.isSetSuccess()) { + optionals.set(0); + } + if (struct.isSetE()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.isSetSuccess()) { + struct.success.write(oprot); + } + if (struct.isSetE()) { + 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.setSuccessIsSet(true); + } + if (incoming.get(1)) { + struct.e = new NotAliveException(); + struct.e.read(iprot); + struct.setEIsSet(true); + } + } + } + + } + + public static class getTopologyConf_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + 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()); + } + + public 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.isSetId()) { + this.id = other.id; + } + } + + public getTopologyConf_args deepCopy() { + return new getTopologyConf_args(this); + } + + @Override + public void clear() { + this.id = null; + } + + public String getId() { + return this.id; + } + + public getTopologyConf_args setId(String id) { + this.id = id; + return this; + } + + public void unsetId() { + this.id = null; + } + + /** Returns true if field id is set (has been assigned a value) and false otherwise */ + public boolean isSetId() { + return this.id != null; + } + + public void setIdIsSet(boolean value) { + if (!value) { + this.id = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case ID: + if (value == null) { + unsetId(); + } else { + setId((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case ID: + return getId(); + + } + 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 isSetId(); + } + 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.isSetId(); + boolean that_present_id = true && that.isSetId(); + 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 && (isSetId()); + 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(isSetId()).compareTo(typedOther.isSetId()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetId()) { + lastComparison = org.apache.thrift.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.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.setIdIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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.isSetId()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetId()) { + 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.setIdIsSet(true); + } + } + } + + } + + public static class getTopologyConf_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + 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()); + } + + public String success; // required + public 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.isSetSuccess()) { + this.success = other.success; + } + if (other.isSetE()) { + 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 getSuccess() { + return this.success; + } + + public getTopologyConf_result setSuccess(String success) { + this.success = success; + return this; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public NotAliveException getE() { + return this.e; + } + + public getTopologyConf_result setE(NotAliveException e) { + this.e = e; + return this; + } + + public void unsetE() { + this.e = null; + } + + /** Returns true if field e is set (has been assigned a value) and false otherwise */ + public boolean isSetE() { + return this.e != null; + } + + public void setEIsSet(boolean value) { + if (!value) { + this.e = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((String)value); + } + break; + + case E: + if (value == null) { + unsetE(); + } else { + setE((NotAliveException)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + case E: + return getE(); + + } + 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 isSetSuccess(); + case E: + return isSetE(); + } + 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.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + 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.isSetE(); + boolean that_present_e = true && that.isSetE(); + 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 && (isSetSuccess()); + builder.append(present_success); + if (present_success) + builder.append(success); + + boolean present_e = true && (isSetE()); + 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(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetE()).compareTo(typedOther.isSetE()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetE()) { + lastComparison = org.apache.thrift.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.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.setSuccessIsSet(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.setEIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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.isSetSuccess()) { + optionals.set(0); + } + if (struct.isSetE()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.isSetSuccess()) { + oprot.writeString(struct.success); + } + if (struct.isSetE()) { + 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.setSuccessIsSet(true); + } + if (incoming.get(1)) { + struct.e = new NotAliveException(); + struct.e.read(iprot); + struct.setEIsSet(true); + } + } + } + + } + + public static class getTopology_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + 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()); + } + + public 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.isSetId()) { + this.id = other.id; + } + } + + public getTopology_args deepCopy() { + return new getTopology_args(this); + } + + @Override + public void clear() { + this.id = null; + } + + public String getId() { + return this.id; + } + + public getTopology_args setId(String id) { + this.id = id; + return this; + } + + public void unsetId() { + this.id = null; + } + + /** Returns true if field id is set (has been assigned a value) and false otherwise */ + public boolean isSetId() { + return this.id != null; + } + + public void setIdIsSet(boolean value) { + if (!value) { + this.id = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case ID: + if (value == null) { + unsetId(); + } else { + setId((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case ID: + return getId(); + + } + 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 isSetId(); + } + 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.isSetId(); + boolean that_present_id = true && that.isSetId(); + 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 && (isSetId()); + 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(isSetId()).compareTo(typedOther.isSetId()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetId()) { + lastComparison = org.apache.thrift.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.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.setIdIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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.isSetId()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetId()) { + 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.setIdIsSet(true); + } + } + } + + } + + public static class getTopology_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + 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()); + } + + public StormTopology success; // required + public 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.isSetSuccess()) { + this.success = new StormTopology(other.success); + } + if (other.isSetE()) { + 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 getSuccess() { + return this.success; + } + + public getTopology_result setSuccess(StormTopology success) { + this.success = success; + return this; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public NotAliveException getE() { + return this.e; + } + + public getTopology_result setE(NotAliveException e) { + this.e = e; + return this; + } + + public void unsetE() { + this.e = null; + } + + /** Returns true if field e is set (has been assigned a value) and false otherwise */ + public boolean isSetE() { + return this.e != null; + } + + public void setEIsSet(boolean value) { + if (!value) { + this.e = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((StormTopology)value); + } + break; + + case E: + if (value == null) { + unsetE(); + } else { + setE((NotAliveException)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + case E: + return getE(); + + } + 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 isSetSuccess(); + case E: + return isSetE(); + } + 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.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + 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.isSetE(); + boolean that_present_e = true && that.isSetE(); + 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 && (isSetSuccess()); + builder.append(present_success); + if (present_success) + builder.append(success); + + boolean present_e = true && (isSetE()); + 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(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetE()).compareTo(typedOther.isSetE()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetE()) { + lastComparison = org.apache.thrift.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.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.setSuccessIsSet(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.setEIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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.isSetSuccess()) { + optionals.set(0); + } + if (struct.isSetE()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.isSetSuccess()) { + struct.success.write(oprot); + } + if (struct.isSetE()) { + 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.setSuccessIsSet(true); + } + if (incoming.get(1)) { + struct.e = new NotAliveException(); + struct.e.read(iprot); + struct.setEIsSet(true); + } + } + } + + } + + public static class getUserTopology_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + 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()); + } + + public 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.isSetId()) { + this.id = other.id; + } + } + + public getUserTopology_args deepCopy() { + return new getUserTopology_args(this); + } + + @Override + public void clear() { + this.id = null; + } + + public String getId() { + return this.id; + } + + public getUserTopology_args setId(String id) { + this.id = id; + return this; + } + + public void unsetId() { + this.id = null; + } + + /** Returns true if field id is set (has been assigned a value) and false otherwise */ + public boolean isSetId() { + return this.id != null; + } + + public void setIdIsSet(boolean value) { + if (!value) { + this.id = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case ID: + if (value == null) { + unsetId(); + } else { + setId((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case ID: + return getId(); + + } + 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 isSetId(); + } + 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.isSetId(); + boolean that_present_id = true && that.isSetId(); + 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 && (isSetId()); + 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(isSetId()).compareTo(typedOther.isSetId()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetId()) { + lastComparison = org.apache.thrift.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.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.setIdIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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.isSetId()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetId()) { + 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.setIdIsSet(true); + } + } + } + + } + + public static class getUserTopology_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + 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()); + } + + public StormTopology success; // required + public 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.isSetSuccess()) { + this.success = new StormTopology(other.success); + } + if (other.isSetE()) { + 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 getSuccess() { + return this.success; + } + + public getUserTopology_result setSuccess(StormTopology success) { + this.success = success; + return this; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public NotAliveException getE() { + return this.e; + } + + public getUserTopology_result setE(NotAliveException e) { + this.e = e; + return this; + } + + public void unsetE() { + this.e = null; + } + + /** Returns true if field e is set (has been assigned a value) and false otherwise */ + public boolean isSetE() { + return this.e != null; + } + + public void setEIsSet(boolean value) { + if (!value) { + this.e = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((StormTopology)value); + } + break; + + case E: + if (value == null) { + unsetE(); + } else { + setE((NotAliveException)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + case E: + return getE(); + + } + 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 isSetSuccess(); + case E: + return isSetE(); + } + 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.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + 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.isSetE(); + boolean that_present_e = true && that.isSetE(); + 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 && (isSetSuccess()); + builder.append(present_success); + if (present_success) + builder.append(success); + + boolean present_e = true && (isSetE()); + 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(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetE()).compareTo(typedOther.isSetE()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetE()) { + lastComparison = org.apache.thrift.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.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.setSuccessIsSet(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.setEIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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.isSetSuccess()) { + optionals.set(0); + } + if (struct.isSetE()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.isSetSuccess()) { + struct.success.write(oprot); + } + if (struct.isSetE()) { + 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.setSuccessIsSet(true); + } + if (incoming.get(1)) { + struct.e = new NotAliveException(); + struct.e.read(iprot); + struct.setEIsSet(true); + } + } + } + + } + +} diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/NotAliveException.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/NotAliveException.java new file mode 100644 index 000000000..9467a17fb --- /dev/null +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/NotAliveException.java @@ -0,0 +1,387 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package com.intel.hibench.streambench.storm.thrift.generated; + +import org.apache.commons.lang.builder.HashCodeBuilder; +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 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 TException implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + 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()); + } + + public 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.isSetMsg()) { + this.msg = other.msg; + } + } + + public NotAliveException deepCopy() { + return new NotAliveException(this); + } + + @Override + public void clear() { + this.msg = null; + } + + public String getMsg() { + return this.msg; + } + + public NotAliveException setMsg(String msg) { + this.msg = msg; + return this; + } + + public void unsetMsg() { + this.msg = null; + } + + /** Returns true if field msg is set (has been assigned a value) and false otherwise */ + public boolean isSetMsg() { + return this.msg != null; + } + + public void setMsgIsSet(boolean value) { + if (!value) { + this.msg = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case MSG: + if (value == null) { + unsetMsg(); + } else { + setMsg((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case MSG: + return getMsg(); + + } + 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 isSetMsg(); + } + 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.isSetMsg(); + boolean that_present_msg = true && that.isSetMsg(); + 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 && (isSetMsg()); + 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(isSetMsg()).compareTo(typedOther.isSetMsg()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetMsg()) { + lastComparison = org.apache.thrift.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.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 (msg == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'msg' was not present! 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.setMsgIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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.setMsgIsSet(true); + } + } + +} + diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/NullStruct.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/NullStruct.java new file mode 100644 index 000000000..42c73324a --- /dev/null +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/NullStruct.java @@ -0,0 +1,281 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package com.intel.hibench.streambench.storm.thrift.generated; + +import org.apache.commons.lang.builder.HashCodeBuilder; +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 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.thrift.TBase, java.io.Serializable, Cloneable { + 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() { + 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.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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/RebalanceOptions.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/RebalanceOptions.java new file mode 100644 index 000000000..c39008b8e --- /dev/null +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/RebalanceOptions.java @@ -0,0 +1,659 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package com.intel.hibench.streambench.storm.thrift.generated; + +import org.apache.commons.lang.builder.HashCodeBuilder; +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 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.thrift.TBase, java.io.Serializable, Cloneable { + 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 NUM_WORKERS_FIELD_DESC = new org.apache.thrift.protocol.TField("num_workers", org.apache.thrift.protocol.TType.I32, (short)2); + private static final org.apache.thrift.protocol.TField NUM_EXECUTORS_FIELD_DESC = new org.apache.thrift.protocol.TField("num_executors", org.apache.thrift.protocol.TType.MAP, (short)3); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new RebalanceOptionsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new RebalanceOptionsTupleSchemeFactory()); + } + + public int wait_secs; // optional + public int num_workers; // optional + public Map num_executors; // 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"), + NUM_WORKERS((short)2, "num_workers"), + NUM_EXECUTORS((short)3, "num_executors"); + + 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; + case 3: // NUM_EXECUTORS + return NUM_EXECUTORS; + 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 byte __isset_bitfield = 0; + private _Fields optionals[] = {_Fields.WAIT_SECS,_Fields.NUM_WORKERS,_Fields.NUM_EXECUTORS}; + 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.NUM_WORKERS, new org.apache.thrift.meta_data.FieldMetaData("num_workers", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + tmpMap.put(_Fields.NUM_EXECUTORS, new org.apache.thrift.meta_data.FieldMetaData("num_executors", 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.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)))); + 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.num_workers = other.num_workers; + if (other.isSetNum_executors()) { + Map __this__num_executors = new HashMap(); + for (Map.Entry other_element : other.num_executors.entrySet()) { + + String other_element_key = other_element.getKey(); + Integer other_element_value = other_element.getValue(); + + String __this__num_executors_copy_key = other_element_key; + + Integer __this__num_executors_copy_value = other_element_value; + + __this__num_executors.put(__this__num_executors_copy_key, __this__num_executors_copy_value); + } + this.num_executors = __this__num_executors; + } + } + + public RebalanceOptions deepCopy() { + return new RebalanceOptions(this); + } + + @Override + public void clear() { + setWait_secsIsSet(false); + this.wait_secs = 0; + setNum_workersIsSet(false); + this.num_workers = 0; + this.num_executors = null; + } + + public int getWait_secs() { + return this.wait_secs; + } + + public RebalanceOptions setWait_secs(int wait_secs) { + this.wait_secs = wait_secs; + setWait_secsIsSet(true); + return this; + } + + public void unsetWait_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 isSetWait_secs() { + return EncodingUtils.testBit(__isset_bitfield, __WAIT_SECS_ISSET_ID); + } + + public void setWait_secsIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __WAIT_SECS_ISSET_ID, value); + } + + public int getNum_workers() { + return this.num_workers; + } + + public RebalanceOptions setNum_workers(int num_workers) { + this.num_workers = num_workers; + setNum_workersIsSet(true); + return this; + } + + public void unsetNum_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 isSetNum_workers() { + return EncodingUtils.testBit(__isset_bitfield, __NUM_WORKERS_ISSET_ID); + } + + public void setNum_workersIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __NUM_WORKERS_ISSET_ID, value); + } + + public int getNum_executorsSize() { + return (this.num_executors == null) ? 0 : this.num_executors.size(); + } + + public void putToNum_executors(String key, int val) { + if (this.num_executors == null) { + this.num_executors = new HashMap(); + } + this.num_executors.put(key, val); + } + + public Map getNum_executors() { + return this.num_executors; + } + + public RebalanceOptions setNum_executors(Map num_executors) { + this.num_executors = num_executors; + return this; + } + + public void unsetNum_executors() { + this.num_executors = null; + } + + /** Returns true if field num_executors is set (has been assigned a value) and false otherwise */ + public boolean isSetNum_executors() { + return this.num_executors != null; + } + + public void setNum_executorsIsSet(boolean value) { + if (!value) { + this.num_executors = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case WAIT_SECS: + if (value == null) { + unsetWait_secs(); + } else { + setWait_secs((Integer)value); + } + break; + + case NUM_WORKERS: + if (value == null) { + unsetNum_workers(); + } else { + setNum_workers((Integer)value); + } + break; + + case NUM_EXECUTORS: + if (value == null) { + unsetNum_executors(); + } else { + setNum_executors((Map)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case WAIT_SECS: + return Integer.valueOf(getWait_secs()); + + case NUM_WORKERS: + return Integer.valueOf(getNum_workers()); + + case NUM_EXECUTORS: + return getNum_executors(); + + } + 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 isSetWait_secs(); + case NUM_WORKERS: + return isSetNum_workers(); + case NUM_EXECUTORS: + return isSetNum_executors(); + } + 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.isSetWait_secs(); + boolean that_present_wait_secs = true && that.isSetWait_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.isSetNum_workers(); + boolean that_present_num_workers = true && that.isSetNum_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; + } + + boolean this_present_num_executors = true && this.isSetNum_executors(); + boolean that_present_num_executors = true && that.isSetNum_executors(); + if (this_present_num_executors || that_present_num_executors) { + if (!(this_present_num_executors && that_present_num_executors)) + return false; + if (!this.num_executors.equals(that.num_executors)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_wait_secs = true && (isSetWait_secs()); + builder.append(present_wait_secs); + if (present_wait_secs) + builder.append(wait_secs); + + boolean present_num_workers = true && (isSetNum_workers()); + builder.append(present_num_workers); + if (present_num_workers) + builder.append(num_workers); + + boolean present_num_executors = true && (isSetNum_executors()); + builder.append(present_num_executors); + if (present_num_executors) + builder.append(num_executors); + + 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(isSetWait_secs()).compareTo(typedOther.isSetWait_secs()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetWait_secs()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.wait_secs, typedOther.wait_secs); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetNum_workers()).compareTo(typedOther.isSetNum_workers()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetNum_workers()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.num_workers, typedOther.num_workers); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetNum_executors()).compareTo(typedOther.isSetNum_executors()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetNum_executors()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.num_executors, typedOther.num_executors); + 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 (isSetWait_secs()) { + sb.append("wait_secs:"); + sb.append(this.wait_secs); + first = false; + } + if (isSetNum_workers()) { + if (!first) sb.append(", "); + sb.append("num_workers:"); + sb.append(this.num_workers); + first = false; + } + if (isSetNum_executors()) { + if (!first) sb.append(", "); + sb.append("num_executors:"); + if (this.num_executors == null) { + sb.append("null"); + } else { + sb.append(this.num_executors); + } + 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.setWait_secsIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // NUM_WORKERS + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.num_workers = iprot.readI32(); + struct.setNum_workersIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // NUM_EXECUTORS + if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { + { + org.apache.thrift.protocol.TMap _map316 = iprot.readMapBegin(); + struct.num_executors = new HashMap(2*_map316.size); + for (int _i317 = 0; _i317 < _map316.size; ++_i317) + { + String _key318; // required + int _val319; // required + _key318 = iprot.readString(); + _val319 = iprot.readI32(); + struct.num_executors.put(_key318, _val319); + } + iprot.readMapEnd(); + } + struct.setNum_executorsIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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.isSetWait_secs()) { + oprot.writeFieldBegin(WAIT_SECS_FIELD_DESC); + oprot.writeI32(struct.wait_secs); + oprot.writeFieldEnd(); + } + if (struct.isSetNum_workers()) { + oprot.writeFieldBegin(NUM_WORKERS_FIELD_DESC); + oprot.writeI32(struct.num_workers); + oprot.writeFieldEnd(); + } + if (struct.num_executors != null) { + if (struct.isSetNum_executors()) { + oprot.writeFieldBegin(NUM_EXECUTORS_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, struct.num_executors.size())); + for (Map.Entry _iter320 : struct.num_executors.entrySet()) + { + oprot.writeString(_iter320.getKey()); + oprot.writeI32(_iter320.getValue()); + } + oprot.writeMapEnd(); + } + 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.isSetWait_secs()) { + optionals.set(0); + } + if (struct.isSetNum_workers()) { + optionals.set(1); + } + if (struct.isSetNum_executors()) { + optionals.set(2); + } + oprot.writeBitSet(optionals, 3); + if (struct.isSetWait_secs()) { + oprot.writeI32(struct.wait_secs); + } + if (struct.isSetNum_workers()) { + oprot.writeI32(struct.num_workers); + } + if (struct.isSetNum_executors()) { + { + oprot.writeI32(struct.num_executors.size()); + for (Map.Entry _iter321 : struct.num_executors.entrySet()) + { + oprot.writeString(_iter321.getKey()); + oprot.writeI32(_iter321.getValue()); + } + } + } + } + + @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.setWait_secsIsSet(true); + } + if (incoming.get(1)) { + struct.num_workers = iprot.readI32(); + struct.setNum_workersIsSet(true); + } + if (incoming.get(2)) { + { + org.apache.thrift.protocol.TMap _map322 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, iprot.readI32()); + struct.num_executors = new HashMap(2*_map322.size); + for (int _i323 = 0; _i323 < _map322.size; ++_i323) + { + String _key324; // required + int _val325; // required + _key324 = iprot.readString(); + _val325 = iprot.readI32(); + struct.num_executors.put(_key324, _val325); + } + } + struct.setNum_executorsIsSet(true); + } + } + } + +} + diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/ShellComponent.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/ShellComponent.java new file mode 100644 index 000000000..8e8b5d741 --- /dev/null +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/ShellComponent.java @@ -0,0 +1,499 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package com.intel.hibench.streambench.storm.thrift.generated; + +import org.apache.commons.lang.builder.HashCodeBuilder; +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 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.thrift.TBase, java.io.Serializable, Cloneable { + 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()); + } + + public String execution_command; // required + public 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.isSetExecution_command()) { + this.execution_command = other.execution_command; + } + if (other.isSetScript()) { + this.script = other.script; + } + } + + public ShellComponent deepCopy() { + return new ShellComponent(this); + } + + @Override + public void clear() { + this.execution_command = null; + this.script = null; + } + + public String getExecution_command() { + return this.execution_command; + } + + public ShellComponent setExecution_command(String execution_command) { + this.execution_command = execution_command; + return this; + } + + public void unsetExecution_command() { + this.execution_command = null; + } + + /** Returns true if field execution_command is set (has been assigned a value) and false otherwise */ + public boolean isSetExecution_command() { + return this.execution_command != null; + } + + public void setExecution_commandIsSet(boolean value) { + if (!value) { + this.execution_command = null; + } + } + + public String getScript() { + return this.script; + } + + public ShellComponent setScript(String script) { + this.script = script; + return this; + } + + public void unsetScript() { + this.script = null; + } + + /** Returns true if field script is set (has been assigned a value) and false otherwise */ + public boolean isSetScript() { + return this.script != null; + } + + public void setScriptIsSet(boolean value) { + if (!value) { + this.script = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case EXECUTION_COMMAND: + if (value == null) { + unsetExecution_command(); + } else { + setExecution_command((String)value); + } + break; + + case SCRIPT: + if (value == null) { + unsetScript(); + } else { + setScript((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case EXECUTION_COMMAND: + return getExecution_command(); + + case SCRIPT: + return getScript(); + + } + 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 isSetExecution_command(); + case SCRIPT: + return isSetScript(); + } + 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.isSetExecution_command(); + boolean that_present_execution_command = true && that.isSetExecution_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.isSetScript(); + boolean that_present_script = true && that.isSetScript(); + 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 && (isSetExecution_command()); + builder.append(present_execution_command); + if (present_execution_command) + builder.append(execution_command); + + boolean present_script = true && (isSetScript()); + 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(isSetExecution_command()).compareTo(typedOther.isSetExecution_command()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetExecution_command()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.execution_command, typedOther.execution_command); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetScript()).compareTo(typedOther.isSetScript()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetScript()) { + lastComparison = org.apache.thrift.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.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.setExecution_commandIsSet(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.setScriptIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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.isSetExecution_command()) { + optionals.set(0); + } + if (struct.isSetScript()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.isSetExecution_command()) { + oprot.writeString(struct.execution_command); + } + if (struct.isSetScript()) { + 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.setExecution_commandIsSet(true); + } + if (incoming.get(1)) { + struct.script = iprot.readString(); + struct.setScriptIsSet(true); + } + } + } + +} + diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/SpoutSpec.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/SpoutSpec.java new file mode 100644 index 000000000..d530666c3 --- /dev/null +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/SpoutSpec.java @@ -0,0 +1,495 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package com.intel.hibench.streambench.storm.thrift.generated; + +import org.apache.commons.lang.builder.HashCodeBuilder; +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 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.thrift.TBase, java.io.Serializable, Cloneable { + 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()); + } + + public ComponentObject spout_object; // required + public 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.isSetSpout_object()) { + this.spout_object = new ComponentObject(other.spout_object); + } + if (other.isSetCommon()) { + 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 getSpout_object() { + return this.spout_object; + } + + public SpoutSpec setSpout_object(ComponentObject spout_object) { + this.spout_object = spout_object; + return this; + } + + public void unsetSpout_object() { + this.spout_object = null; + } + + /** Returns true if field spout_object is set (has been assigned a value) and false otherwise */ + public boolean isSetSpout_object() { + return this.spout_object != null; + } + + public void setSpout_objectIsSet(boolean value) { + if (!value) { + this.spout_object = null; + } + } + + public ComponentCommon getCommon() { + return this.common; + } + + public SpoutSpec setCommon(ComponentCommon common) { + this.common = common; + return this; + } + + public void unsetCommon() { + this.common = null; + } + + /** Returns true if field common is set (has been assigned a value) and false otherwise */ + public boolean isSetCommon() { + return this.common != null; + } + + public void setCommonIsSet(boolean value) { + if (!value) { + this.common = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SPOUT_OBJECT: + if (value == null) { + unsetSpout_object(); + } else { + setSpout_object((ComponentObject)value); + } + break; + + case COMMON: + if (value == null) { + unsetCommon(); + } else { + setCommon((ComponentCommon)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SPOUT_OBJECT: + return getSpout_object(); + + case COMMON: + return getCommon(); + + } + 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 isSetSpout_object(); + case COMMON: + return isSetCommon(); + } + 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.isSetSpout_object(); + boolean that_present_spout_object = true && that.isSetSpout_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.isSetCommon(); + boolean that_present_common = true && that.isSetCommon(); + 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 && (isSetSpout_object()); + builder.append(present_spout_object); + if (present_spout_object) + builder.append(spout_object); + + boolean present_common = true && (isSetCommon()); + 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(isSetSpout_object()).compareTo(typedOther.isSetSpout_object()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSpout_object()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.spout_object, typedOther.spout_object); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetCommon()).compareTo(typedOther.isSetCommon()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetCommon()) { + lastComparison = org.apache.thrift.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.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 (spout_object == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'spout_object' was not present! Struct: " + toString()); + } + if (common == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'common' was not present! 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.setSpout_objectIsSet(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.setCommonIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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.setSpout_objectIsSet(true); + struct.common = new ComponentCommon(); + struct.common.read(iprot); + struct.setCommonIsSet(true); + } + } + +} + diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/SpoutStats.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/SpoutStats.java new file mode 100644 index 000000000..5f93068db --- /dev/null +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/SpoutStats.java @@ -0,0 +1,931 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package com.intel.hibench.streambench.storm.thrift.generated; + +import org.apache.commons.lang.builder.HashCodeBuilder; +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 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.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("SpoutStats"); + + private static final org.apache.thrift.protocol.TField ACKED_FIELD_DESC = new org.apache.thrift.protocol.TField("acked", org.apache.thrift.protocol.TType.MAP, (short)1); + private static final org.apache.thrift.protocol.TField FAILED_FIELD_DESC = new org.apache.thrift.protocol.TField("failed", org.apache.thrift.protocol.TType.MAP, (short)2); + private static final org.apache.thrift.protocol.TField COMPLETE_MS_AVG_FIELD_DESC = new org.apache.thrift.protocol.TField("complete_ms_avg", org.apache.thrift.protocol.TType.MAP, (short)3); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new SpoutStatsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new SpoutStatsTupleSchemeFactory()); + } + + public Map> acked; // required + public Map> failed; // required + public 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.thrift.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.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.ACKED, new org.apache.thrift.meta_data.FieldMetaData("acked", 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.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.I64))))); + tmpMap.put(_Fields.FAILED, new org.apache.thrift.meta_data.FieldMetaData("failed", 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.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.I64))))); + tmpMap.put(_Fields.COMPLETE_MS_AVG, new org.apache.thrift.meta_data.FieldMetaData("complete_ms_avg", 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.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.DOUBLE))))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.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.isSetAcked()) { + 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.isSetFailed()) { + 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.isSetComplete_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 getAckedSize() { + return (this.acked == null) ? 0 : this.acked.size(); + } + + public void putToAcked(String key, Map val) { + if (this.acked == null) { + this.acked = new HashMap>(); + } + this.acked.put(key, val); + } + + public Map> getAcked() { + return this.acked; + } + + public SpoutStats setAcked(Map> acked) { + this.acked = acked; + return this; + } + + public void unsetAcked() { + this.acked = null; + } + + /** Returns true if field acked is set (has been assigned a value) and false otherwise */ + public boolean isSetAcked() { + return this.acked != null; + } + + public void setAckedIsSet(boolean value) { + if (!value) { + this.acked = null; + } + } + + public int getFailedSize() { + return (this.failed == null) ? 0 : this.failed.size(); + } + + public void putToFailed(String key, Map val) { + if (this.failed == null) { + this.failed = new HashMap>(); + } + this.failed.put(key, val); + } + + public Map> getFailed() { + return this.failed; + } + + public SpoutStats setFailed(Map> failed) { + this.failed = failed; + return this; + } + + public void unsetFailed() { + this.failed = null; + } + + /** Returns true if field failed is set (has been assigned a value) and false otherwise */ + public boolean isSetFailed() { + return this.failed != null; + } + + public void setFailedIsSet(boolean value) { + if (!value) { + this.failed = null; + } + } + + public int getComplete_ms_avgSize() { + return (this.complete_ms_avg == null) ? 0 : this.complete_ms_avg.size(); + } + + public void putToComplete_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> getComplete_ms_avg() { + return this.complete_ms_avg; + } + + public SpoutStats setComplete_ms_avg(Map> complete_ms_avg) { + this.complete_ms_avg = complete_ms_avg; + return this; + } + + public void unsetComplete_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 isSetComplete_ms_avg() { + return this.complete_ms_avg != null; + } + + public void setComplete_ms_avgIsSet(boolean value) { + if (!value) { + this.complete_ms_avg = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case ACKED: + if (value == null) { + unsetAcked(); + } else { + setAcked((Map>)value); + } + break; + + case FAILED: + if (value == null) { + unsetFailed(); + } else { + setFailed((Map>)value); + } + break; + + case COMPLETE_MS_AVG: + if (value == null) { + unsetComplete_ms_avg(); + } else { + setComplete_ms_avg((Map>)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case ACKED: + return getAcked(); + + case FAILED: + return getFailed(); + + case COMPLETE_MS_AVG: + return getComplete_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 isSetAcked(); + case FAILED: + return isSetFailed(); + case COMPLETE_MS_AVG: + return isSetComplete_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.isSetAcked(); + boolean that_present_acked = true && that.isSetAcked(); + 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.isSetFailed(); + boolean that_present_failed = true && that.isSetFailed(); + 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.isSetComplete_ms_avg(); + boolean that_present_complete_ms_avg = true && that.isSetComplete_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 && (isSetAcked()); + builder.append(present_acked); + if (present_acked) + builder.append(acked); + + boolean present_failed = true && (isSetFailed()); + builder.append(present_failed); + if (present_failed) + builder.append(failed); + + boolean present_complete_ms_avg = true && (isSetComplete_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(isSetAcked()).compareTo(typedOther.isSetAcked()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetAcked()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.acked, typedOther.acked); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetFailed()).compareTo(typedOther.isSetFailed()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetFailed()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.failed, typedOther.failed); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetComplete_ms_avg()).compareTo(typedOther.isSetComplete_ms_avg()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetComplete_ms_avg()) { + lastComparison = org.apache.thrift.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.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("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.thrift.TException { + // check for required fields + if (acked == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'acked' was not present! Struct: " + toString()); + } + if (failed == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'failed' was not present! Struct: " + toString()); + } + if (complete_ms_avg == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'complete_ms_avg' was not present! 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 SpoutStatsStandardSchemeFactory implements SchemeFactory { + public SpoutStatsStandardScheme getScheme() { + return new SpoutStatsStandardScheme(); + } + } + + private static class SpoutStatsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, SpoutStats 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: // ACKED + if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { + { + org.apache.thrift.protocol.TMap _map190 = iprot.readMapBegin(); + struct.acked = new HashMap>(2*_map190.size); + for (int _i191 = 0; _i191 < _map190.size; ++_i191) + { + String _key192; // required + Map _val193; // required + _key192 = iprot.readString(); + { + org.apache.thrift.protocol.TMap _map194 = iprot.readMapBegin(); + _val193 = new HashMap(2*_map194.size); + for (int _i195 = 0; _i195 < _map194.size; ++_i195) + { + String _key196; // required + long _val197; // required + _key196 = iprot.readString(); + _val197 = iprot.readI64(); + _val193.put(_key196, _val197); + } + iprot.readMapEnd(); + } + struct.acked.put(_key192, _val193); + } + iprot.readMapEnd(); + } + struct.setAckedIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // FAILED + if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { + { + org.apache.thrift.protocol.TMap _map198 = iprot.readMapBegin(); + struct.failed = new HashMap>(2*_map198.size); + for (int _i199 = 0; _i199 < _map198.size; ++_i199) + { + String _key200; // required + Map _val201; // required + _key200 = iprot.readString(); + { + org.apache.thrift.protocol.TMap _map202 = iprot.readMapBegin(); + _val201 = new HashMap(2*_map202.size); + for (int _i203 = 0; _i203 < _map202.size; ++_i203) + { + String _key204; // required + long _val205; // required + _key204 = iprot.readString(); + _val205 = iprot.readI64(); + _val201.put(_key204, _val205); + } + iprot.readMapEnd(); + } + struct.failed.put(_key200, _val201); + } + iprot.readMapEnd(); + } + struct.setFailedIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // COMPLETE_MS_AVG + if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { + { + org.apache.thrift.protocol.TMap _map206 = iprot.readMapBegin(); + struct.complete_ms_avg = new HashMap>(2*_map206.size); + for (int _i207 = 0; _i207 < _map206.size; ++_i207) + { + String _key208; // required + Map _val209; // required + _key208 = iprot.readString(); + { + org.apache.thrift.protocol.TMap _map210 = iprot.readMapBegin(); + _val209 = new HashMap(2*_map210.size); + for (int _i211 = 0; _i211 < _map210.size; ++_i211) + { + String _key212; // required + double _val213; // required + _key212 = iprot.readString(); + _val213 = iprot.readDouble(); + _val209.put(_key212, _val213); + } + iprot.readMapEnd(); + } + struct.complete_ms_avg.put(_key208, _val209); + } + iprot.readMapEnd(); + } + struct.setComplete_ms_avgIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, SpoutStats struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.acked != null) { + oprot.writeFieldBegin(ACKED_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.acked.size())); + for (Map.Entry> _iter214 : struct.acked.entrySet()) + { + oprot.writeString(_iter214.getKey()); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, _iter214.getValue().size())); + for (Map.Entry _iter215 : _iter214.getValue().entrySet()) + { + oprot.writeString(_iter215.getKey()); + oprot.writeI64(_iter215.getValue()); + } + oprot.writeMapEnd(); + } + } + oprot.writeMapEnd(); + } + oprot.writeFieldEnd(); + } + if (struct.failed != null) { + oprot.writeFieldBegin(FAILED_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.failed.size())); + for (Map.Entry> _iter216 : struct.failed.entrySet()) + { + oprot.writeString(_iter216.getKey()); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, _iter216.getValue().size())); + for (Map.Entry _iter217 : _iter216.getValue().entrySet()) + { + oprot.writeString(_iter217.getKey()); + oprot.writeI64(_iter217.getValue()); + } + oprot.writeMapEnd(); + } + } + oprot.writeMapEnd(); + } + oprot.writeFieldEnd(); + } + if (struct.complete_ms_avg != null) { + oprot.writeFieldBegin(COMPLETE_MS_AVG_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.complete_ms_avg.size())); + for (Map.Entry> _iter218 : struct.complete_ms_avg.entrySet()) + { + oprot.writeString(_iter218.getKey()); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.DOUBLE, _iter218.getValue().size())); + for (Map.Entry _iter219 : _iter218.getValue().entrySet()) + { + oprot.writeString(_iter219.getKey()); + oprot.writeDouble(_iter219.getValue()); + } + oprot.writeMapEnd(); + } + } + oprot.writeMapEnd(); + } + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class SpoutStatsTupleSchemeFactory implements SchemeFactory { + public SpoutStatsTupleScheme getScheme() { + return new SpoutStatsTupleScheme(); + } + } + + private static class SpoutStatsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, SpoutStats struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + { + oprot.writeI32(struct.acked.size()); + for (Map.Entry> _iter220 : struct.acked.entrySet()) + { + oprot.writeString(_iter220.getKey()); + { + oprot.writeI32(_iter220.getValue().size()); + for (Map.Entry _iter221 : _iter220.getValue().entrySet()) + { + oprot.writeString(_iter221.getKey()); + oprot.writeI64(_iter221.getValue()); + } + } + } + } + { + oprot.writeI32(struct.failed.size()); + for (Map.Entry> _iter222 : struct.failed.entrySet()) + { + oprot.writeString(_iter222.getKey()); + { + oprot.writeI32(_iter222.getValue().size()); + for (Map.Entry _iter223 : _iter222.getValue().entrySet()) + { + oprot.writeString(_iter223.getKey()); + oprot.writeI64(_iter223.getValue()); + } + } + } + } + { + oprot.writeI32(struct.complete_ms_avg.size()); + for (Map.Entry> _iter224 : struct.complete_ms_avg.entrySet()) + { + oprot.writeString(_iter224.getKey()); + { + oprot.writeI32(_iter224.getValue().size()); + for (Map.Entry _iter225 : _iter224.getValue().entrySet()) + { + oprot.writeString(_iter225.getKey()); + oprot.writeDouble(_iter225.getValue()); + } + } + } + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, SpoutStats struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + { + org.apache.thrift.protocol.TMap _map226 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32()); + struct.acked = new HashMap>(2*_map226.size); + for (int _i227 = 0; _i227 < _map226.size; ++_i227) + { + String _key228; // required + Map _val229; // required + _key228 = iprot.readString(); + { + org.apache.thrift.protocol.TMap _map230 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, iprot.readI32()); + _val229 = new HashMap(2*_map230.size); + for (int _i231 = 0; _i231 < _map230.size; ++_i231) + { + String _key232; // required + long _val233; // required + _key232 = iprot.readString(); + _val233 = iprot.readI64(); + _val229.put(_key232, _val233); + } + } + struct.acked.put(_key228, _val229); + } + } + struct.setAckedIsSet(true); + { + org.apache.thrift.protocol.TMap _map234 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32()); + struct.failed = new HashMap>(2*_map234.size); + for (int _i235 = 0; _i235 < _map234.size; ++_i235) + { + String _key236; // required + Map _val237; // required + _key236 = iprot.readString(); + { + org.apache.thrift.protocol.TMap _map238 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, iprot.readI32()); + _val237 = new HashMap(2*_map238.size); + for (int _i239 = 0; _i239 < _map238.size; ++_i239) + { + String _key240; // required + long _val241; // required + _key240 = iprot.readString(); + _val241 = iprot.readI64(); + _val237.put(_key240, _val241); + } + } + struct.failed.put(_key236, _val237); + } + } + struct.setFailedIsSet(true); + { + org.apache.thrift.protocol.TMap _map242 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32()); + struct.complete_ms_avg = new HashMap>(2*_map242.size); + for (int _i243 = 0; _i243 < _map242.size; ++_i243) + { + String _key244; // required + Map _val245; // required + _key244 = iprot.readString(); + { + org.apache.thrift.protocol.TMap _map246 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.DOUBLE, iprot.readI32()); + _val245 = 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(); + _val245.put(_key248, _val249); + } + } + struct.complete_ms_avg.put(_key244, _val245); + } + } + struct.setComplete_ms_avgIsSet(true); + } + } + +} + diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/StateSpoutSpec.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/StateSpoutSpec.java new file mode 100644 index 000000000..c1f2c8d94 --- /dev/null +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/StateSpoutSpec.java @@ -0,0 +1,495 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package com.intel.hibench.streambench.storm.thrift.generated; + +import org.apache.commons.lang.builder.HashCodeBuilder; +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 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.thrift.TBase, java.io.Serializable, Cloneable { + 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()); + } + + public ComponentObject state_spout_object; // required + public 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.isSetState_spout_object()) { + this.state_spout_object = new ComponentObject(other.state_spout_object); + } + if (other.isSetCommon()) { + 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 getState_spout_object() { + return this.state_spout_object; + } + + public StateSpoutSpec setState_spout_object(ComponentObject state_spout_object) { + this.state_spout_object = state_spout_object; + return this; + } + + public void unsetState_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 isSetState_spout_object() { + return this.state_spout_object != null; + } + + public void setState_spout_objectIsSet(boolean value) { + if (!value) { + this.state_spout_object = null; + } + } + + public ComponentCommon getCommon() { + return this.common; + } + + public StateSpoutSpec setCommon(ComponentCommon common) { + this.common = common; + return this; + } + + public void unsetCommon() { + this.common = null; + } + + /** Returns true if field common is set (has been assigned a value) and false otherwise */ + public boolean isSetCommon() { + return this.common != null; + } + + public void setCommonIsSet(boolean value) { + if (!value) { + this.common = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case STATE_SPOUT_OBJECT: + if (value == null) { + unsetState_spout_object(); + } else { + setState_spout_object((ComponentObject)value); + } + break; + + case COMMON: + if (value == null) { + unsetCommon(); + } else { + setCommon((ComponentCommon)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case STATE_SPOUT_OBJECT: + return getState_spout_object(); + + case COMMON: + return getCommon(); + + } + 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 isSetState_spout_object(); + case COMMON: + return isSetCommon(); + } + 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.isSetState_spout_object(); + boolean that_present_state_spout_object = true && that.isSetState_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.isSetCommon(); + boolean that_present_common = true && that.isSetCommon(); + 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 && (isSetState_spout_object()); + builder.append(present_state_spout_object); + if (present_state_spout_object) + builder.append(state_spout_object); + + boolean present_common = true && (isSetCommon()); + 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(isSetState_spout_object()).compareTo(typedOther.isSetState_spout_object()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetState_spout_object()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.state_spout_object, typedOther.state_spout_object); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetCommon()).compareTo(typedOther.isSetCommon()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetCommon()) { + lastComparison = org.apache.thrift.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.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 (state_spout_object == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'state_spout_object' was not present! Struct: " + toString()); + } + if (common == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'common' was not present! 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.setState_spout_objectIsSet(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.setCommonIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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.setState_spout_objectIsSet(true); + struct.common = new ComponentCommon(); + struct.common.read(iprot); + struct.setCommonIsSet(true); + } + } + +} + diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/StormTopology.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/StormTopology.java new file mode 100644 index 000000000..5a8c616db --- /dev/null +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/StormTopology.java @@ -0,0 +1,784 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package com.intel.hibench.streambench.storm.thrift.generated; + +import org.apache.commons.lang.builder.HashCodeBuilder; +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 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.thrift.TBase, java.io.Serializable, Cloneable { + 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()); + } + + public Map spouts; // required + public Map bolts; // required + public 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.isSetSpouts()) { + 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.isSetBolts()) { + 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.isSetState_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 getSpoutsSize() { + return (this.spouts == null) ? 0 : this.spouts.size(); + } + + public void putToSpouts(String key, SpoutSpec val) { + if (this.spouts == null) { + this.spouts = new HashMap(); + } + this.spouts.put(key, val); + } + + public Map getSpouts() { + return this.spouts; + } + + public StormTopology setSpouts(Map spouts) { + this.spouts = spouts; + return this; + } + + public void unsetSpouts() { + this.spouts = null; + } + + /** Returns true if field spouts is set (has been assigned a value) and false otherwise */ + public boolean isSetSpouts() { + return this.spouts != null; + } + + public void setSpoutsIsSet(boolean value) { + if (!value) { + this.spouts = null; + } + } + + public int getBoltsSize() { + return (this.bolts == null) ? 0 : this.bolts.size(); + } + + public void putToBolts(String key, Bolt val) { + if (this.bolts == null) { + this.bolts = new HashMap(); + } + this.bolts.put(key, val); + } + + public Map getBolts() { + return this.bolts; + } + + public StormTopology setBolts(Map bolts) { + this.bolts = bolts; + return this; + } + + public void unsetBolts() { + this.bolts = null; + } + + /** Returns true if field bolts is set (has been assigned a value) and false otherwise */ + public boolean isSetBolts() { + return this.bolts != null; + } + + public void setBoltsIsSet(boolean value) { + if (!value) { + this.bolts = null; + } + } + + public int getState_spoutsSize() { + return (this.state_spouts == null) ? 0 : this.state_spouts.size(); + } + + public void putToState_spouts(String key, StateSpoutSpec val) { + if (this.state_spouts == null) { + this.state_spouts = new HashMap(); + } + this.state_spouts.put(key, val); + } + + public Map getState_spouts() { + return this.state_spouts; + } + + public StormTopology setState_spouts(Map state_spouts) { + this.state_spouts = state_spouts; + return this; + } + + public void unsetState_spouts() { + this.state_spouts = null; + } + + /** Returns true if field state_spouts is set (has been assigned a value) and false otherwise */ + public boolean isSetState_spouts() { + return this.state_spouts != null; + } + + public void setState_spoutsIsSet(boolean value) { + if (!value) { + this.state_spouts = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SPOUTS: + if (value == null) { + unsetSpouts(); + } else { + setSpouts((Map)value); + } + break; + + case BOLTS: + if (value == null) { + unsetBolts(); + } else { + setBolts((Map)value); + } + break; + + case STATE_SPOUTS: + if (value == null) { + unsetState_spouts(); + } else { + setState_spouts((Map)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SPOUTS: + return getSpouts(); + + case BOLTS: + return getBolts(); + + case STATE_SPOUTS: + return getState_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 isSetSpouts(); + case BOLTS: + return isSetBolts(); + case STATE_SPOUTS: + return isSetState_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.isSetSpouts(); + boolean that_present_spouts = true && that.isSetSpouts(); + 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.isSetBolts(); + boolean that_present_bolts = true && that.isSetBolts(); + 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.isSetState_spouts(); + boolean that_present_state_spouts = true && that.isSetState_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 && (isSetSpouts()); + builder.append(present_spouts); + if (present_spouts) + builder.append(spouts); + + boolean present_bolts = true && (isSetBolts()); + builder.append(present_bolts); + if (present_bolts) + builder.append(bolts); + + boolean present_state_spouts = true && (isSetState_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(isSetSpouts()).compareTo(typedOther.isSetSpouts()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSpouts()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.spouts, typedOther.spouts); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetBolts()).compareTo(typedOther.isSetBolts()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetBolts()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.bolts, typedOther.bolts); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetState_spouts()).compareTo(typedOther.isSetState_spouts()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetState_spouts()) { + lastComparison = org.apache.thrift.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.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 (spouts == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'spouts' was not present! Struct: " + toString()); + } + if (bolts == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'bolts' was not present! Struct: " + toString()); + } + if (state_spouts == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'state_spouts' was not present! 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); + for (int _i45 = 0; _i45 < _map44.size; ++_i45) + { + String _key46; // required + SpoutSpec _val47; // required + _key46 = iprot.readString(); + _val47 = new SpoutSpec(); + _val47.read(iprot); + struct.spouts.put(_key46, _val47); + } + iprot.readMapEnd(); + } + struct.setSpoutsIsSet(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); + for (int _i49 = 0; _i49 < _map48.size; ++_i49) + { + String _key50; // required + Bolt _val51; // required + _key50 = iprot.readString(); + _val51 = new Bolt(); + _val51.read(iprot); + struct.bolts.put(_key50, _val51); + } + iprot.readMapEnd(); + } + struct.setBoltsIsSet(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); + for (int _i53 = 0; _i53 < _map52.size; ++_i53) + { + String _key54; // required + StateSpoutSpec _val55; // required + _key54 = iprot.readString(); + _val55 = new StateSpoutSpec(); + _val55.read(iprot); + struct.state_spouts.put(_key54, _val55); + } + iprot.readMapEnd(); + } + struct.setState_spoutsIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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); + for (int _i63 = 0; _i63 < _map62.size; ++_i63) + { + String _key64; // required + SpoutSpec _val65; // required + _key64 = iprot.readString(); + _val65 = new SpoutSpec(); + _val65.read(iprot); + struct.spouts.put(_key64, _val65); + } + } + struct.setSpoutsIsSet(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); + for (int _i67 = 0; _i67 < _map66.size; ++_i67) + { + String _key68; // required + Bolt _val69; // required + _key68 = iprot.readString(); + _val69 = new Bolt(); + _val69.read(iprot); + struct.bolts.put(_key68, _val69); + } + } + struct.setBoltsIsSet(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); + for (int _i71 = 0; _i71 < _map70.size; ++_i71) + { + String _key72; // required + StateSpoutSpec _val73; // required + _key72 = iprot.readString(); + _val73 = new StateSpoutSpec(); + _val73.read(iprot); + struct.state_spouts.put(_key72, _val73); + } + } + struct.setState_spoutsIsSet(true); + } + } + +} + diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/StreamInfo.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/StreamInfo.java new file mode 100644 index 000000000..88f7a279f --- /dev/null +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/StreamInfo.java @@ -0,0 +1,539 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package com.intel.hibench.streambench.storm.thrift.generated; + +import org.apache.commons.lang.builder.HashCodeBuilder; +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 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.thrift.TBase, java.io.Serializable, Cloneable { + 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()); + } + + public List output_fields; // required + public 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; + setDirectIsSet(true); + } + + /** + * Performs a deep copy on other. + */ + public StreamInfo(StreamInfo other) { + __isset_bitfield = other.__isset_bitfield; + if (other.isSetOutput_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; + setDirectIsSet(false); + this.direct = false; + } + + public int getOutput_fieldsSize() { + return (this.output_fields == null) ? 0 : this.output_fields.size(); + } + + public java.util.Iterator getOutput_fieldsIterator() { + return (this.output_fields == null) ? null : this.output_fields.iterator(); + } + + public void addToOutput_fields(String elem) { + if (this.output_fields == null) { + this.output_fields = new ArrayList(); + } + this.output_fields.add(elem); + } + + public List getOutput_fields() { + return this.output_fields; + } + + public StreamInfo setOutput_fields(List output_fields) { + this.output_fields = output_fields; + return this; + } + + public void unsetOutput_fields() { + this.output_fields = null; + } + + /** Returns true if field output_fields is set (has been assigned a value) and false otherwise */ + public boolean isSetOutput_fields() { + return this.output_fields != null; + } + + public void setOutput_fieldsIsSet(boolean value) { + if (!value) { + this.output_fields = null; + } + } + + public boolean isDirect() { + return this.direct; + } + + public StreamInfo setDirect(boolean direct) { + this.direct = direct; + setDirectIsSet(true); + return this; + } + + public void unsetDirect() { + __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 isSetDirect() { + return EncodingUtils.testBit(__isset_bitfield, __DIRECT_ISSET_ID); + } + + public void setDirectIsSet(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) { + unsetOutput_fields(); + } else { + setOutput_fields((List)value); + } + break; + + case DIRECT: + if (value == null) { + unsetDirect(); + } else { + setDirect((Boolean)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case OUTPUT_FIELDS: + return getOutput_fields(); + + case DIRECT: + return Boolean.valueOf(isDirect()); + + } + 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 isSetOutput_fields(); + case DIRECT: + return isSetDirect(); + } + 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.isSetOutput_fields(); + boolean that_present_output_fields = true && that.isSetOutput_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 && (isSetOutput_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(isSetOutput_fields()).compareTo(typedOther.isSetOutput_fields()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetOutput_fields()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.output_fields, typedOther.output_fields); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetDirect()).compareTo(typedOther.isSetDirect()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetDirect()) { + lastComparison = org.apache.thrift.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.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 (output_fields == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'output_fields' was not present! Struct: " + toString()); + } + // alas, we cannot check 'direct' because it's a primitive and you chose the non-beans generator. + // 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); + for (int _i17 = 0; _i17 < _list16.size; ++_i17) + { + String _elem18; // required + _elem18 = iprot.readString(); + struct.output_fields.add(_elem18); + } + iprot.readListEnd(); + } + struct.setOutput_fieldsIsSet(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.setDirectIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + if (!struct.isSetDirect()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'direct' was not found in serialized data! Struct: " + toString()); + } + 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); + for (int _i22 = 0; _i22 < _list21.size; ++_i22) + { + String _elem23; // required + _elem23 = iprot.readString(); + struct.output_fields.add(_elem23); + } + } + struct.setOutput_fieldsIsSet(true); + struct.direct = iprot.readBool(); + struct.setDirectIsSet(true); + } + } + +} + diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/SubmitOptions.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/SubmitOptions.java new file mode 100644 index 000000000..a617ab6f2 --- /dev/null +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/SubmitOptions.java @@ -0,0 +1,403 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package com.intel.hibench.streambench.storm.thrift.generated; + +import org.apache.commons.lang.builder.HashCodeBuilder; +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 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.thrift.TBase, java.io.Serializable, Cloneable { + 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()); + } + + /** + * + * @see TopologyInitialStatus + */ + public 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.isSetInitial_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 getInitial_status() { + return this.initial_status; + } + + /** + * + * @see TopologyInitialStatus + */ + public SubmitOptions setInitial_status(TopologyInitialStatus initial_status) { + this.initial_status = initial_status; + return this; + } + + public void unsetInitial_status() { + this.initial_status = null; + } + + /** Returns true if field initial_status is set (has been assigned a value) and false otherwise */ + public boolean isSetInitial_status() { + return this.initial_status != null; + } + + public void setInitial_statusIsSet(boolean value) { + if (!value) { + this.initial_status = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case INITIAL_STATUS: + if (value == null) { + unsetInitial_status(); + } else { + setInitial_status((TopologyInitialStatus)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case INITIAL_STATUS: + return getInitial_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 isSetInitial_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.isSetInitial_status(); + boolean that_present_initial_status = true && that.isSetInitial_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 && (isSetInitial_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(isSetInitial_status()).compareTo(typedOther.isSetInitial_status()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetInitial_status()) { + lastComparison = org.apache.thrift.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.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 (initial_status == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'initial_status' was not present! 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 = TopologyInitialStatus.findByValue(iprot.readI32()); + struct.setInitial_statusIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + 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 = TopologyInitialStatus.findByValue(iprot.readI32()); + struct.setInitial_statusIsSet(true); + } + } + +} + diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/SupervisorSummary.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/SupervisorSummary.java new file mode 100644 index 000000000..5bf077fa1 --- /dev/null +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/SupervisorSummary.java @@ -0,0 +1,780 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package com.intel.hibench.streambench.storm.thrift.generated; + +import org.apache.commons.lang.builder.HashCodeBuilder; +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 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.thrift.TBase, java.io.Serializable, Cloneable { + 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 UPTIME_SECS_FIELD_DESC = new org.apache.thrift.protocol.TField("uptime_secs", org.apache.thrift.protocol.TType.I32, (short)2); + 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)3); + 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)4); + 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)5); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new SupervisorSummaryStandardSchemeFactory()); + schemes.put(TupleScheme.class, new SupervisorSummaryTupleSchemeFactory()); + } + + public String host; // required + public int uptime_secs; // required + public int num_workers; // required + public int num_used_workers; // required + public String supervisor_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 { + HOST((short)1, "host"), + UPTIME_SECS((short)2, "uptime_secs"), + NUM_WORKERS((short)3, "num_workers"), + NUM_USED_WORKERS((short)4, "num_used_workers"), + SUPERVISOR_ID((short)5, "supervisor_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: // HOST + return HOST; + case 2: // UPTIME_SECS + return UPTIME_SECS; + case 3: // NUM_WORKERS + return NUM_WORKERS; + case 4: // NUM_USED_WORKERS + return NUM_USED_WORKERS; + case 5: // SUPERVISOR_ID + return SUPERVISOR_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 + 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.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))); + 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))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(SupervisorSummary.class, metaDataMap); + } + + public SupervisorSummary() { + } + + public SupervisorSummary( + String host, + int uptime_secs, + int num_workers, + int num_used_workers, + String supervisor_id) + { + this(); + this.host = host; + this.uptime_secs = uptime_secs; + setUptime_secsIsSet(true); + this.num_workers = num_workers; + setNum_workersIsSet(true); + this.num_used_workers = num_used_workers; + setNum_used_workersIsSet(true); + this.supervisor_id = supervisor_id; + } + + /** + * Performs a deep copy on other. + */ + public SupervisorSummary(SupervisorSummary other) { + __isset_bitfield = other.__isset_bitfield; + if (other.isSetHost()) { + this.host = other.host; + } + this.uptime_secs = other.uptime_secs; + this.num_workers = other.num_workers; + this.num_used_workers = other.num_used_workers; + if (other.isSetSupervisor_id()) { + this.supervisor_id = other.supervisor_id; + } + } + + public SupervisorSummary deepCopy() { + return new SupervisorSummary(this); + } + + @Override + public void clear() { + this.host = null; + setUptime_secsIsSet(false); + this.uptime_secs = 0; + setNum_workersIsSet(false); + this.num_workers = 0; + setNum_used_workersIsSet(false); + this.num_used_workers = 0; + this.supervisor_id = null; + } + + public String getHost() { + return this.host; + } + + public SupervisorSummary setHost(String host) { + this.host = host; + return this; + } + + public void unsetHost() { + this.host = null; + } + + /** Returns true if field host is set (has been assigned a value) and false otherwise */ + public boolean isSetHost() { + return this.host != null; + } + + public void setHostIsSet(boolean value) { + if (!value) { + this.host = null; + } + } + + public int getUptime_secs() { + return this.uptime_secs; + } + + public SupervisorSummary setUptime_secs(int uptime_secs) { + this.uptime_secs = uptime_secs; + setUptime_secsIsSet(true); + return this; + } + + public void unsetUptime_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 isSetUptime_secs() { + return EncodingUtils.testBit(__isset_bitfield, __UPTIME_SECS_ISSET_ID); + } + + public void setUptime_secsIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __UPTIME_SECS_ISSET_ID, value); + } + + public int getNum_workers() { + return this.num_workers; + } + + public SupervisorSummary setNum_workers(int num_workers) { + this.num_workers = num_workers; + setNum_workersIsSet(true); + return this; + } + + public void unsetNum_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 isSetNum_workers() { + return EncodingUtils.testBit(__isset_bitfield, __NUM_WORKERS_ISSET_ID); + } + + public void setNum_workersIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __NUM_WORKERS_ISSET_ID, value); + } + + public int getNum_used_workers() { + return this.num_used_workers; + } + + public SupervisorSummary setNum_used_workers(int num_used_workers) { + this.num_used_workers = num_used_workers; + setNum_used_workersIsSet(true); + return this; + } + + public void unsetNum_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 isSetNum_used_workers() { + return EncodingUtils.testBit(__isset_bitfield, __NUM_USED_WORKERS_ISSET_ID); + } + + public void setNum_used_workersIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __NUM_USED_WORKERS_ISSET_ID, value); + } + + public String getSupervisor_id() { + return this.supervisor_id; + } + + public SupervisorSummary setSupervisor_id(String supervisor_id) { + this.supervisor_id = supervisor_id; + return this; + } + + public void unsetSupervisor_id() { + this.supervisor_id = null; + } + + /** Returns true if field supervisor_id is set (has been assigned a value) and false otherwise */ + public boolean isSetSupervisor_id() { + return this.supervisor_id != null; + } + + public void setSupervisor_idIsSet(boolean value) { + if (!value) { + this.supervisor_id = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case HOST: + if (value == null) { + unsetHost(); + } else { + setHost((String)value); + } + break; + + case UPTIME_SECS: + if (value == null) { + unsetUptime_secs(); + } else { + setUptime_secs((Integer)value); + } + break; + + case NUM_WORKERS: + if (value == null) { + unsetNum_workers(); + } else { + setNum_workers((Integer)value); + } + break; + + case NUM_USED_WORKERS: + if (value == null) { + unsetNum_used_workers(); + } else { + setNum_used_workers((Integer)value); + } + break; + + case SUPERVISOR_ID: + if (value == null) { + unsetSupervisor_id(); + } else { + setSupervisor_id((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case HOST: + return getHost(); + + case UPTIME_SECS: + return Integer.valueOf(getUptime_secs()); + + case NUM_WORKERS: + return Integer.valueOf(getNum_workers()); + + case NUM_USED_WORKERS: + return Integer.valueOf(getNum_used_workers()); + + case SUPERVISOR_ID: + return getSupervisor_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 HOST: + return isSetHost(); + case UPTIME_SECS: + return isSetUptime_secs(); + case NUM_WORKERS: + return isSetNum_workers(); + case NUM_USED_WORKERS: + return isSetNum_used_workers(); + case SUPERVISOR_ID: + return isSetSupervisor_id(); + } + 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.isSetHost(); + boolean that_present_host = true && that.isSetHost(); + 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; + 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; + } + + boolean this_present_supervisor_id = true && this.isSetSupervisor_id(); + boolean that_present_supervisor_id = true && that.isSetSupervisor_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; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_host = true && (isSetHost()); + builder.append(present_host); + if (present_host) + builder.append(host); + + 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); + + boolean present_supervisor_id = true && (isSetSupervisor_id()); + builder.append(present_supervisor_id); + if (present_supervisor_id) + builder.append(supervisor_id); + + 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(isSetHost()).compareTo(typedOther.isSetHost()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetHost()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.host, typedOther.host); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetUptime_secs()).compareTo(typedOther.isSetUptime_secs()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetUptime_secs()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.uptime_secs, typedOther.uptime_secs); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetNum_workers()).compareTo(typedOther.isSetNum_workers()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetNum_workers()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.num_workers, typedOther.num_workers); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetNum_used_workers()).compareTo(typedOther.isSetNum_used_workers()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetNum_used_workers()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.num_used_workers, typedOther.num_used_workers); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetSupervisor_id()).compareTo(typedOther.isSetSupervisor_id()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSupervisor_id()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.supervisor_id, typedOther.supervisor_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("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("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; + if (!first) sb.append(", "); + sb.append("supervisor_id:"); + if (this.supervisor_id == null) { + sb.append("null"); + } else { + sb.append(this.supervisor_id); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (host == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'host' was not present! Struct: " + toString()); + } + // alas, we cannot check 'uptime_secs' because it's a primitive and you chose the non-beans generator. + // alas, we cannot check 'num_workers' because it's a primitive and you chose the non-beans generator. + // alas, we cannot check 'num_used_workers' because it's a primitive and you chose the non-beans generator. + if (supervisor_id == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'supervisor_id' was not present! 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.setHostIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // UPTIME_SECS + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.uptime_secs = iprot.readI32(); + struct.setUptime_secsIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // NUM_WORKERS + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.num_workers = iprot.readI32(); + struct.setNum_workersIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // NUM_USED_WORKERS + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.num_used_workers = iprot.readI32(); + struct.setNum_used_workersIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 5: // SUPERVISOR_ID + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.supervisor_id = iprot.readString(); + struct.setSupervisor_idIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + if (!struct.isSetUptime_secs()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'uptime_secs' was not found in serialized data! Struct: " + toString()); + } + if (!struct.isSetNum_workers()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'num_workers' was not found in serialized data! Struct: " + toString()); + } + if (!struct.isSetNum_used_workers()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'num_used_workers' was not found in serialized data! Struct: " + toString()); + } + 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(); + } + 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(); + if (struct.supervisor_id != null) { + oprot.writeFieldBegin(SUPERVISOR_ID_FIELD_DESC); + oprot.writeString(struct.supervisor_id); + 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.writeI32(struct.uptime_secs); + oprot.writeI32(struct.num_workers); + oprot.writeI32(struct.num_used_workers); + oprot.writeString(struct.supervisor_id); + } + + @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.setHostIsSet(true); + struct.uptime_secs = iprot.readI32(); + struct.setUptime_secsIsSet(true); + struct.num_workers = iprot.readI32(); + struct.setNum_workersIsSet(true); + struct.num_used_workers = iprot.readI32(); + struct.setNum_used_workersIsSet(true); + struct.supervisor_id = iprot.readString(); + struct.setSupervisor_idIsSet(true); + } + } + +} + diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/TopologyInfo.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/TopologyInfo.java new file mode 100644 index 000000000..36fb36b26 --- /dev/null +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/TopologyInfo.java @@ -0,0 +1,1046 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package com.intel.hibench.streambench.storm.thrift.generated; + +import org.apache.commons.lang.builder.HashCodeBuilder; +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 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.thrift.TBase, java.io.Serializable, Cloneable { + 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 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 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 EXECUTORS_FIELD_DESC = new org.apache.thrift.protocol.TField("executors", org.apache.thrift.protocol.TType.LIST, (short)4); + 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)5); + private static final org.apache.thrift.protocol.TField ERRORS_FIELD_DESC = new org.apache.thrift.protocol.TField("errors", org.apache.thrift.protocol.TType.MAP, (short)6); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TopologyInfoStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TopologyInfoTupleSchemeFactory()); + } + + public String id; // required + public String name; // required + public int uptime_secs; // required + public List executors; // required + public String status; // required + public Map> errors; // 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"), + NAME((short)2, "name"), + UPTIME_SECS((short)3, "uptime_secs"), + EXECUTORS((short)4, "executors"), + STATUS((short)5, "status"), + 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: // ID + return ID; + case 2: // NAME + return NAME; + case 3: // UPTIME_SECS + return UPTIME_SECS; + case 4: // EXECUTORS + return EXECUTORS; + case 5: // STATUS + return STATUS; + 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 __UPTIME_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.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.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.EXECUTORS, new org.apache.thrift.meta_data.FieldMetaData("executors", 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, ExecutorSummary.class)))); + 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.ERRORS, new org.apache.thrift.meta_data.FieldMetaData("errors", 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.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(TopologyInfo.class, metaDataMap); + } + + public TopologyInfo() { + } + + public TopologyInfo( + String id, + String name, + int uptime_secs, + List executors, + String status, + Map> errors) + { + this(); + this.id = id; + this.name = name; + this.uptime_secs = uptime_secs; + setUptime_secsIsSet(true); + this.executors = executors; + this.status = status; + this.errors = errors; + } + + /** + * Performs a deep copy on other. + */ + public TopologyInfo(TopologyInfo other) { + __isset_bitfield = other.__isset_bitfield; + if (other.isSetId()) { + this.id = other.id; + } + if (other.isSetName()) { + this.name = other.name; + } + this.uptime_secs = other.uptime_secs; + if (other.isSetExecutors()) { + List __this__executors = new ArrayList(); + for (ExecutorSummary other_element : other.executors) { + __this__executors.add(new ExecutorSummary(other_element)); + } + this.executors = __this__executors; + } + if (other.isSetStatus()) { + this.status = other.status; + } + if (other.isSetErrors()) { + Map> __this__errors = new HashMap>(); + for (Map.Entry> other_element : other.errors.entrySet()) { + + String other_element_key = other_element.getKey(); + List other_element_value = other_element.getValue(); + + String __this__errors_copy_key = other_element_key; + + List __this__errors_copy_value = new ArrayList(); + for (ErrorInfo other_element_value_element : other_element_value) { + __this__errors_copy_value.add(new ErrorInfo(other_element_value_element)); + } + + __this__errors.put(__this__errors_copy_key, __this__errors_copy_value); + } + this.errors = __this__errors; + } + } + + public TopologyInfo deepCopy() { + return new TopologyInfo(this); + } + + @Override + public void clear() { + this.id = null; + this.name = null; + setUptime_secsIsSet(false); + this.uptime_secs = 0; + this.executors = null; + this.status = null; + this.errors = null; + } + + public String getId() { + return this.id; + } + + public TopologyInfo setId(String id) { + this.id = id; + return this; + } + + public void unsetId() { + this.id = null; + } + + /** Returns true if field id is set (has been assigned a value) and false otherwise */ + public boolean isSetId() { + return this.id != null; + } + + public void setIdIsSet(boolean value) { + if (!value) { + this.id = null; + } + } + + public String getName() { + return this.name; + } + + public TopologyInfo setName(String name) { + this.name = name; + return this; + } + + public void unsetName() { + this.name = null; + } + + /** Returns true if field name is set (has been assigned a value) and false otherwise */ + public boolean isSetName() { + return this.name != null; + } + + public void setNameIsSet(boolean value) { + if (!value) { + this.name = null; + } + } + + public int getUptime_secs() { + return this.uptime_secs; + } + + public TopologyInfo setUptime_secs(int uptime_secs) { + this.uptime_secs = uptime_secs; + setUptime_secsIsSet(true); + return this; + } + + public void unsetUptime_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 isSetUptime_secs() { + return EncodingUtils.testBit(__isset_bitfield, __UPTIME_SECS_ISSET_ID); + } + + public void setUptime_secsIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __UPTIME_SECS_ISSET_ID, value); + } + + public int getExecutorsSize() { + return (this.executors == null) ? 0 : this.executors.size(); + } + + public java.util.Iterator getExecutorsIterator() { + return (this.executors == null) ? null : this.executors.iterator(); + } + + public void addToExecutors(ExecutorSummary elem) { + if (this.executors == null) { + this.executors = new ArrayList(); + } + this.executors.add(elem); + } + + public List getExecutors() { + return this.executors; + } + + public TopologyInfo setExecutors(List executors) { + this.executors = executors; + return this; + } + + public void unsetExecutors() { + this.executors = null; + } + + /** Returns true if field executors is set (has been assigned a value) and false otherwise */ + public boolean isSetExecutors() { + return this.executors != null; + } + + public void setExecutorsIsSet(boolean value) { + if (!value) { + this.executors = null; + } + } + + public String getStatus() { + return this.status; + } + + public TopologyInfo setStatus(String status) { + this.status = status; + return this; + } + + public void unsetStatus() { + this.status = null; + } + + /** Returns true if field status is set (has been assigned a value) and false otherwise */ + public boolean isSetStatus() { + return this.status != null; + } + + public void setStatusIsSet(boolean value) { + if (!value) { + this.status = null; + } + } + + public int getErrorsSize() { + return (this.errors == null) ? 0 : this.errors.size(); + } + + public void putToErrors(String key, List val) { + if (this.errors == null) { + this.errors = new HashMap>(); + } + this.errors.put(key, val); + } + + public Map> getErrors() { + return this.errors; + } + + public TopologyInfo setErrors(Map> errors) { + this.errors = errors; + return this; + } + + public void unsetErrors() { + this.errors = null; + } + + /** Returns true if field errors is set (has been assigned a value) and false otherwise */ + public boolean isSetErrors() { + return this.errors != null; + } + + public void setErrorsIsSet(boolean value) { + if (!value) { + this.errors = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case ID: + if (value == null) { + unsetId(); + } else { + setId((String)value); + } + break; + + case NAME: + if (value == null) { + unsetName(); + } else { + setName((String)value); + } + break; + + case UPTIME_SECS: + if (value == null) { + unsetUptime_secs(); + } else { + setUptime_secs((Integer)value); + } + break; + + case EXECUTORS: + if (value == null) { + unsetExecutors(); + } else { + setExecutors((List)value); + } + break; + + case STATUS: + if (value == null) { + unsetStatus(); + } else { + setStatus((String)value); + } + break; + + case ERRORS: + if (value == null) { + unsetErrors(); + } else { + setErrors((Map>)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case ID: + return getId(); + + case NAME: + return getName(); + + case UPTIME_SECS: + return Integer.valueOf(getUptime_secs()); + + case EXECUTORS: + return getExecutors(); + + case STATUS: + return getStatus(); + + case ERRORS: + return getErrors(); + + } + 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 isSetId(); + case NAME: + return isSetName(); + case UPTIME_SECS: + return isSetUptime_secs(); + case EXECUTORS: + return isSetExecutors(); + case STATUS: + return isSetStatus(); + case ERRORS: + return isSetErrors(); + } + 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.isSetId(); + boolean that_present_id = true && that.isSetId(); + 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.isSetName(); + boolean that_present_name = true && that.isSetName(); + 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_executors = true && this.isSetExecutors(); + boolean that_present_executors = true && that.isSetExecutors(); + if (this_present_executors || that_present_executors) { + if (!(this_present_executors && that_present_executors)) + return false; + if (!this.executors.equals(that.executors)) + return false; + } + + boolean this_present_status = true && this.isSetStatus(); + boolean that_present_status = true && that.isSetStatus(); + 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_errors = true && this.isSetErrors(); + boolean that_present_errors = true && that.isSetErrors(); + 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() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_id = true && (isSetId()); + builder.append(present_id); + if (present_id) + builder.append(id); + + boolean present_name = true && (isSetName()); + 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_executors = true && (isSetExecutors()); + builder.append(present_executors); + if (present_executors) + builder.append(executors); + + boolean present_status = true && (isSetStatus()); + builder.append(present_status); + if (present_status) + builder.append(status); + + boolean present_errors = true && (isSetErrors()); + builder.append(present_errors); + if (present_errors) + builder.append(errors); + + 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(isSetId()).compareTo(typedOther.isSetId()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetId()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.id, typedOther.id); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetName()).compareTo(typedOther.isSetName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.name, typedOther.name); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetUptime_secs()).compareTo(typedOther.isSetUptime_secs()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetUptime_secs()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.uptime_secs, typedOther.uptime_secs); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetExecutors()).compareTo(typedOther.isSetExecutors()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetExecutors()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.executors, typedOther.executors); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStatus()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetErrors()).compareTo(typedOther.isSetErrors()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetErrors()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.errors, typedOther.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("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("executors:"); + if (this.executors == null) { + sb.append("null"); + } else { + sb.append(this.executors); + } + 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("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 (id == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'id' was not present! Struct: " + toString()); + } + if (name == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'name' was not present! Struct: " + toString()); + } + // alas, we cannot check 'uptime_secs' because it's a primitive and you chose the non-beans generator. + if (executors == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'executors' was not present! Struct: " + toString()); + } + if (status == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' was not present! Struct: " + toString()); + } + if (errors == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'errors' was not present! 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 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: // ID + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.id = iprot.readString(); + struct.setIdIsSet(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.setNameIsSet(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.setUptime_secsIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // EXECUTORS + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list290 = iprot.readListBegin(); + struct.executors = new ArrayList(_list290.size); + for (int _i291 = 0; _i291 < _list290.size; ++_i291) + { + ExecutorSummary _elem292; // required + _elem292 = new ExecutorSummary(); + _elem292.read(iprot); + struct.executors.add(_elem292); + } + iprot.readListEnd(); + } + struct.setExecutorsIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 5: // STATUS + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.status = iprot.readString(); + struct.setStatusIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 6: // ERRORS + if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { + { + org.apache.thrift.protocol.TMap _map293 = iprot.readMapBegin(); + struct.errors = new HashMap>(2*_map293.size); + for (int _i294 = 0; _i294 < _map293.size; ++_i294) + { + String _key295; // required + List _val296; // required + _key295 = iprot.readString(); + { + org.apache.thrift.protocol.TList _list297 = iprot.readListBegin(); + _val296 = new ArrayList(_list297.size); + for (int _i298 = 0; _i298 < _list297.size; ++_i298) + { + ErrorInfo _elem299; // required + _elem299 = new ErrorInfo(); + _elem299.read(iprot); + _val296.add(_elem299); + } + iprot.readListEnd(); + } + struct.errors.put(_key295, _val296); + } + iprot.readMapEnd(); + } + struct.setErrorsIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + if (!struct.isSetUptime_secs()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'uptime_secs' was not found in serialized data! Struct: " + toString()); + } + 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.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(); + } + oprot.writeFieldBegin(UPTIME_SECS_FIELD_DESC); + oprot.writeI32(struct.uptime_secs); + oprot.writeFieldEnd(); + if (struct.executors != null) { + oprot.writeFieldBegin(EXECUTORS_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.executors.size())); + for (ExecutorSummary _iter300 : struct.executors) + { + _iter300.write(oprot); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + if (struct.status != null) { + oprot.writeFieldBegin(STATUS_FIELD_DESC); + oprot.writeString(struct.status); + oprot.writeFieldEnd(); + } + if (struct.errors != null) { + oprot.writeFieldBegin(ERRORS_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.LIST, struct.errors.size())); + for (Map.Entry> _iter301 : struct.errors.entrySet()) + { + oprot.writeString(_iter301.getKey()); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, _iter301.getValue().size())); + for (ErrorInfo _iter302 : _iter301.getValue()) + { + _iter302.write(oprot); + } + oprot.writeListEnd(); + } + } + oprot.writeMapEnd(); + } + 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; + oprot.writeString(struct.id); + oprot.writeString(struct.name); + oprot.writeI32(struct.uptime_secs); + { + oprot.writeI32(struct.executors.size()); + for (ExecutorSummary _iter303 : struct.executors) + { + _iter303.write(oprot); + } + } + oprot.writeString(struct.status); + { + oprot.writeI32(struct.errors.size()); + for (Map.Entry> _iter304 : struct.errors.entrySet()) + { + oprot.writeString(_iter304.getKey()); + { + oprot.writeI32(_iter304.getValue().size()); + for (ErrorInfo _iter305 : _iter304.getValue()) + { + _iter305.write(oprot); + } + } + } + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TopologyInfo struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.id = iprot.readString(); + struct.setIdIsSet(true); + struct.name = iprot.readString(); + struct.setNameIsSet(true); + struct.uptime_secs = iprot.readI32(); + struct.setUptime_secsIsSet(true); + { + org.apache.thrift.protocol.TList _list306 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.executors = new ArrayList(_list306.size); + for (int _i307 = 0; _i307 < _list306.size; ++_i307) + { + ExecutorSummary _elem308; // required + _elem308 = new ExecutorSummary(); + _elem308.read(iprot); + struct.executors.add(_elem308); + } + } + struct.setExecutorsIsSet(true); + struct.status = iprot.readString(); + struct.setStatusIsSet(true); + { + org.apache.thrift.protocol.TMap _map309 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.LIST, iprot.readI32()); + struct.errors = new HashMap>(2*_map309.size); + for (int _i310 = 0; _i310 < _map309.size; ++_i310) + { + String _key311; // required + List _val312; // required + _key311 = iprot.readString(); + { + org.apache.thrift.protocol.TList _list313 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + _val312 = new ArrayList(_list313.size); + for (int _i314 = 0; _i314 < _list313.size; ++_i314) + { + ErrorInfo _elem315; // required + _elem315 = new ErrorInfo(); + _elem315.read(iprot); + _val312.add(_elem315); + } + } + struct.errors.put(_key311, _val312); + } + } + struct.setErrorsIsSet(true); + } + } + +} + diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/TopologyInitialStatus.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/TopologyInitialStatus.java new file mode 100644 index 000000000..5ed30f953 --- /dev/null +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/TopologyInitialStatus.java @@ -0,0 +1,45 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package com.intel.hibench.streambench.storm.thrift.generated; + + +import java.util.Map; +import java.util.HashMap; +import org.apache.thrift.TEnum; + +public enum TopologyInitialStatus implements org.apache.thrift.TEnum { + ACTIVE(1), + INACTIVE(2); + + private final int value; + + private TopologyInitialStatus(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 TopologyInitialStatus findByValue(int value) { + switch (value) { + case 1: + return ACTIVE; + case 2: + return INACTIVE; + default: + return null; + } + } +} diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/TopologySummary.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/TopologySummary.java new file mode 100644 index 000000000..968065eac --- /dev/null +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/thrift/generated/TopologySummary.java @@ -0,0 +1,977 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package com.intel.hibench.streambench.storm.thrift.generated; + +import org.apache.commons.lang.builder.HashCodeBuilder; +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 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.thrift.TBase, java.io.Serializable, Cloneable { + 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 NUM_TASKS_FIELD_DESC = new org.apache.thrift.protocol.TField("num_tasks", org.apache.thrift.protocol.TType.I32, (short)3); + private static final org.apache.thrift.protocol.TField NUM_EXECUTORS_FIELD_DESC = new org.apache.thrift.protocol.TField("num_executors", org.apache.thrift.protocol.TType.I32, (short)4); + 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)5); + 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)6); + 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)7); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TopologySummaryStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TopologySummaryTupleSchemeFactory()); + } + + public String id; // required + public String name; // required + public int num_tasks; // required + public int num_executors; // required + public int num_workers; // required + public int uptime_secs; // required + public 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.thrift.TFieldIdEnum { + ID((short)1, "id"), + NAME((short)2, "name"), + NUM_TASKS((short)3, "num_tasks"), + NUM_EXECUTORS((short)4, "num_executors"), + NUM_WORKERS((short)5, "num_workers"), + UPTIME_SECS((short)6, "uptime_secs"), + STATUS((short)7, "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: // ID + return ID; + case 2: // NAME + return NAME; + case 3: // NUM_TASKS + return NUM_TASKS; + case 4: // NUM_EXECUTORS + return NUM_EXECUTORS; + case 5: // NUM_WORKERS + return NUM_WORKERS; + case 6: // UPTIME_SECS + return UPTIME_SECS; + case 7: // 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 __NUM_TASKS_ISSET_ID = 0; + private static final int __NUM_EXECUTORS_ISSET_ID = 1; + private static final int __NUM_WORKERS_ISSET_ID = 2; + private static final int __UPTIME_SECS_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.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.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_EXECUTORS, new org.apache.thrift.meta_data.FieldMetaData("num_executors", 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.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.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))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TopologySummary.class, metaDataMap); + } + + public TopologySummary() { + } + + public TopologySummary( + String id, + String name, + int num_tasks, + int num_executors, + int num_workers, + int uptime_secs, + String status) + { + this(); + this.id = id; + this.name = name; + this.num_tasks = num_tasks; + setNum_tasksIsSet(true); + this.num_executors = num_executors; + setNum_executorsIsSet(true); + this.num_workers = num_workers; + setNum_workersIsSet(true); + this.uptime_secs = uptime_secs; + setUptime_secsIsSet(true); + this.status = status; + } + + /** + * Performs a deep copy on other. + */ + public TopologySummary(TopologySummary other) { + __isset_bitfield = other.__isset_bitfield; + if (other.isSetId()) { + this.id = other.id; + } + if (other.isSetName()) { + this.name = other.name; + } + this.num_tasks = other.num_tasks; + this.num_executors = other.num_executors; + this.num_workers = other.num_workers; + this.uptime_secs = other.uptime_secs; + if (other.isSetStatus()) { + this.status = other.status; + } + } + + public TopologySummary deepCopy() { + return new TopologySummary(this); + } + + @Override + public void clear() { + this.id = null; + this.name = null; + setNum_tasksIsSet(false); + this.num_tasks = 0; + setNum_executorsIsSet(false); + this.num_executors = 0; + setNum_workersIsSet(false); + this.num_workers = 0; + setUptime_secsIsSet(false); + this.uptime_secs = 0; + this.status = null; + } + + public String getId() { + return this.id; + } + + public TopologySummary setId(String id) { + this.id = id; + return this; + } + + public void unsetId() { + this.id = null; + } + + /** Returns true if field id is set (has been assigned a value) and false otherwise */ + public boolean isSetId() { + return this.id != null; + } + + public void setIdIsSet(boolean value) { + if (!value) { + this.id = null; + } + } + + public String getName() { + return this.name; + } + + public TopologySummary setName(String name) { + this.name = name; + return this; + } + + public void unsetName() { + this.name = null; + } + + /** Returns true if field name is set (has been assigned a value) and false otherwise */ + public boolean isSetName() { + return this.name != null; + } + + public void setNameIsSet(boolean value) { + if (!value) { + this.name = null; + } + } + + public int getNum_tasks() { + return this.num_tasks; + } + + public TopologySummary setNum_tasks(int num_tasks) { + this.num_tasks = num_tasks; + setNum_tasksIsSet(true); + return this; + } + + public void unsetNum_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 isSetNum_tasks() { + return EncodingUtils.testBit(__isset_bitfield, __NUM_TASKS_ISSET_ID); + } + + public void setNum_tasksIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __NUM_TASKS_ISSET_ID, value); + } + + public int getNum_executors() { + return this.num_executors; + } + + public TopologySummary setNum_executors(int num_executors) { + this.num_executors = num_executors; + setNum_executorsIsSet(true); + return this; + } + + public void unsetNum_executors() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __NUM_EXECUTORS_ISSET_ID); + } + + /** Returns true if field num_executors is set (has been assigned a value) and false otherwise */ + public boolean isSetNum_executors() { + return EncodingUtils.testBit(__isset_bitfield, __NUM_EXECUTORS_ISSET_ID); + } + + public void setNum_executorsIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __NUM_EXECUTORS_ISSET_ID, value); + } + + public int getNum_workers() { + return this.num_workers; + } + + public TopologySummary setNum_workers(int num_workers) { + this.num_workers = num_workers; + setNum_workersIsSet(true); + return this; + } + + public void unsetNum_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 isSetNum_workers() { + return EncodingUtils.testBit(__isset_bitfield, __NUM_WORKERS_ISSET_ID); + } + + public void setNum_workersIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __NUM_WORKERS_ISSET_ID, value); + } + + public int getUptime_secs() { + return this.uptime_secs; + } + + public TopologySummary setUptime_secs(int uptime_secs) { + this.uptime_secs = uptime_secs; + setUptime_secsIsSet(true); + return this; + } + + public void unsetUptime_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 isSetUptime_secs() { + return EncodingUtils.testBit(__isset_bitfield, __UPTIME_SECS_ISSET_ID); + } + + public void setUptime_secsIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __UPTIME_SECS_ISSET_ID, value); + } + + public String getStatus() { + return this.status; + } + + public TopologySummary setStatus(String status) { + this.status = status; + return this; + } + + public void unsetStatus() { + this.status = null; + } + + /** Returns true if field status is set (has been assigned a value) and false otherwise */ + public boolean isSetStatus() { + return this.status != null; + } + + public void setStatusIsSet(boolean value) { + if (!value) { + this.status = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case ID: + if (value == null) { + unsetId(); + } else { + setId((String)value); + } + break; + + case NAME: + if (value == null) { + unsetName(); + } else { + setName((String)value); + } + break; + + case NUM_TASKS: + if (value == null) { + unsetNum_tasks(); + } else { + setNum_tasks((Integer)value); + } + break; + + case NUM_EXECUTORS: + if (value == null) { + unsetNum_executors(); + } else { + setNum_executors((Integer)value); + } + break; + + case NUM_WORKERS: + if (value == null) { + unsetNum_workers(); + } else { + setNum_workers((Integer)value); + } + break; + + case UPTIME_SECS: + if (value == null) { + unsetUptime_secs(); + } else { + setUptime_secs((Integer)value); + } + break; + + case STATUS: + if (value == null) { + unsetStatus(); + } else { + setStatus((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case ID: + return getId(); + + case NAME: + return getName(); + + case NUM_TASKS: + return Integer.valueOf(getNum_tasks()); + + case NUM_EXECUTORS: + return Integer.valueOf(getNum_executors()); + + case NUM_WORKERS: + return Integer.valueOf(getNum_workers()); + + case UPTIME_SECS: + return Integer.valueOf(getUptime_secs()); + + case STATUS: + return getStatus(); + + } + 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 isSetId(); + case NAME: + return isSetName(); + case NUM_TASKS: + return isSetNum_tasks(); + case NUM_EXECUTORS: + return isSetNum_executors(); + case NUM_WORKERS: + return isSetNum_workers(); + case UPTIME_SECS: + return isSetUptime_secs(); + case STATUS: + return isSetStatus(); + } + 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.isSetId(); + boolean that_present_id = true && that.isSetId(); + 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.isSetName(); + boolean that_present_name = true && that.isSetName(); + 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_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_executors = true; + boolean that_present_num_executors = true; + if (this_present_num_executors || that_present_num_executors) { + if (!(this_present_num_executors && that_present_num_executors)) + return false; + if (this.num_executors != that.num_executors) + 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_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_status = true && this.isSetStatus(); + boolean that_present_status = true && that.isSetStatus(); + 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_id = true && (isSetId()); + builder.append(present_id); + if (present_id) + builder.append(id); + + boolean present_name = true && (isSetName()); + builder.append(present_name); + if (present_name) + builder.append(name); + + boolean present_num_tasks = true; + builder.append(present_num_tasks); + if (present_num_tasks) + builder.append(num_tasks); + + boolean present_num_executors = true; + builder.append(present_num_executors); + if (present_num_executors) + builder.append(num_executors); + + boolean present_num_workers = true; + builder.append(present_num_workers); + if (present_num_workers) + builder.append(num_workers); + + boolean present_uptime_secs = true; + builder.append(present_uptime_secs); + if (present_uptime_secs) + builder.append(uptime_secs); + + boolean present_status = true && (isSetStatus()); + builder.append(present_status); + if (present_status) + builder.append(status); + + 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(isSetId()).compareTo(typedOther.isSetId()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetId()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.id, typedOther.id); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetName()).compareTo(typedOther.isSetName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.name, typedOther.name); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetNum_tasks()).compareTo(typedOther.isSetNum_tasks()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetNum_tasks()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.num_tasks, typedOther.num_tasks); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetNum_executors()).compareTo(typedOther.isSetNum_executors()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetNum_executors()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.num_executors, typedOther.num_executors); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetNum_workers()).compareTo(typedOther.isSetNum_workers()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetNum_workers()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.num_workers, typedOther.num_workers); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetUptime_secs()).compareTo(typedOther.isSetUptime_secs()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetUptime_secs()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.uptime_secs, typedOther.uptime_secs); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStatus()) { + lastComparison = org.apache.thrift.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.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("num_tasks:"); + sb.append(this.num_tasks); + first = false; + if (!first) sb.append(", "); + sb.append("num_executors:"); + sb.append(this.num_executors); + first = false; + if (!first) sb.append(", "); + sb.append("num_workers:"); + sb.append(this.num_workers); + first = false; + if (!first) sb.append(", "); + sb.append("uptime_secs:"); + sb.append(this.uptime_secs); + first = false; + 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.thrift.TException { + // check for required fields + if (id == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'id' was not present! Struct: " + toString()); + } + if (name == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'name' was not present! Struct: " + toString()); + } + // alas, we cannot check 'num_tasks' because it's a primitive and you chose the non-beans generator. + // alas, we cannot check 'num_executors' because it's a primitive and you chose the non-beans generator. + // alas, we cannot check 'num_workers' because it's a primitive and you chose the non-beans generator. + // alas, we cannot check 'uptime_secs' because it's a primitive and you chose the non-beans generator. + if (status == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' was not present! 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.setIdIsSet(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.setNameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // NUM_TASKS + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.num_tasks = iprot.readI32(); + struct.setNum_tasksIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // NUM_EXECUTORS + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.num_executors = iprot.readI32(); + struct.setNum_executorsIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 5: // NUM_WORKERS + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.num_workers = iprot.readI32(); + struct.setNum_workersIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 6: // UPTIME_SECS + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.uptime_secs = iprot.readI32(); + struct.setUptime_secsIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 7: // STATUS + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.status = iprot.readString(); + struct.setStatusIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + if (!struct.isSetNum_tasks()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'num_tasks' was not found in serialized data! Struct: " + toString()); + } + if (!struct.isSetNum_executors()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'num_executors' was not found in serialized data! Struct: " + toString()); + } + if (!struct.isSetNum_workers()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'num_workers' was not found in serialized data! Struct: " + toString()); + } + if (!struct.isSetUptime_secs()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'uptime_secs' was not found in serialized data! Struct: " + toString()); + } + 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(); + } + oprot.writeFieldBegin(NUM_TASKS_FIELD_DESC); + oprot.writeI32(struct.num_tasks); + oprot.writeFieldEnd(); + oprot.writeFieldBegin(NUM_EXECUTORS_FIELD_DESC); + oprot.writeI32(struct.num_executors); + oprot.writeFieldEnd(); + oprot.writeFieldBegin(NUM_WORKERS_FIELD_DESC); + oprot.writeI32(struct.num_workers); + oprot.writeFieldEnd(); + oprot.writeFieldBegin(UPTIME_SECS_FIELD_DESC); + oprot.writeI32(struct.uptime_secs); + oprot.writeFieldEnd(); + if (struct.status != null) { + oprot.writeFieldBegin(STATUS_FIELD_DESC); + oprot.writeString(struct.status); + 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.writeI32(struct.num_tasks); + oprot.writeI32(struct.num_executors); + oprot.writeI32(struct.num_workers); + oprot.writeI32(struct.uptime_secs); + oprot.writeString(struct.status); + } + + @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.setIdIsSet(true); + struct.name = iprot.readString(); + struct.setNameIsSet(true); + struct.num_tasks = iprot.readI32(); + struct.setNum_tasksIsSet(true); + struct.num_executors = iprot.readI32(); + struct.setNum_executorsIsSet(true); + struct.num_workers = iprot.readI32(); + struct.setNum_workersIsSet(true); + struct.uptime_secs = iprot.readI32(); + struct.setUptime_secsIsSet(true); + struct.status = iprot.readString(); + struct.setStatusIsSet(true); + } + } + +} + diff --git a/src/streambench/common/src/main/java/com/intel/hibench/streambench/common/UserVisit.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/topologies/AbstractStormSpoutTops.java similarity index 54% rename from src/streambench/common/src/main/java/com/intel/hibench/streambench/common/UserVisit.java rename to src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/topologies/AbstractStormSpoutTops.java index 7fb688b06..f31e00f5f 100644 --- a/src/streambench/common/src/main/java/com/intel/hibench/streambench/common/UserVisit.java +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/topologies/AbstractStormSpoutTops.java @@ -15,34 +15,26 @@ * limitations under the License. */ -package com.intel.hibench.streambench.common; +package com.intel.hibench.streambench.storm.topologies; -import java.io.Serializable; - -/** - * I only add three fields here, we can add more later if we need them. - */ -public class UserVisit implements Serializable{ - String ip; - String sessionId; // TODO: Not sure about the meaning of this field - String browser; +import backtype.storm.topology.TopologyBuilder; +import backtype.storm.topology.base.BaseRichSpout; +import com.intel.hibench.streambench.storm.spout.ConstructSpoutUtil; +import com.intel.hibench.streambench.storm.util.StormBenchConfig; +public class AbstractStormSpoutTops { + protected StormBenchConfig config; - public UserVisit(String ip, String sessionId, String browser) { - this.browser = browser; - this.ip = ip; - this.sessionId = sessionId; - } + public AbstractStormSpoutTops(StormBenchConfig c) { + config=c; + } - public String getBrowser() { - return browser; - } + public void setSpout(TopologyBuilder builder){ + BaseRichSpout spout= ConstructSpoutUtil.constructSpout(); + builder.setSpout("spout", spout, config.spoutThreads); + } - public String getIp() { - return ip; - } + public void setBolt(TopologyBuilder builder){ - public String getSessionId() { - return sessionId; - } + } } diff --git a/src/streambench/common/src/main/scala/com/intel/hibench/streambench/common/metrics/LatencyCollector.scala b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/topologies/AbstractTridentSpoutTops.java similarity index 80% rename from src/streambench/common/src/main/scala/com/intel/hibench/streambench/common/metrics/LatencyCollector.scala rename to src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/topologies/AbstractTridentSpoutTops.java index 2beffd7df..6c30af33c 100644 --- a/src/streambench/common/src/main/scala/com/intel/hibench/streambench/common/metrics/LatencyCollector.scala +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/topologies/AbstractTridentSpoutTops.java @@ -15,10 +15,12 @@ * limitations under the License. */ -package com.intel.hibench.streambench.common.metrics +package com.intel.hibench.streambench.storm.topologies; -trait LatencyCollector { +import storm.trident.TridentTopology; - def start(): Unit +public class AbstractTridentSpoutTops { + public void setTopology(TridentTopology topology) { + } } diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/topologies/SingleSpoutTops.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/topologies/SingleSpoutTops.java index 6fa40a456..f747cf117 100644 --- a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/topologies/SingleSpoutTops.java +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/topologies/SingleSpoutTops.java @@ -17,44 +17,37 @@ package com.intel.hibench.streambench.storm.topologies; -import com.intel.hibench.streambench.storm.spout.KafkaSpoutFactory; -import com.intel.hibench.streambench.storm.util.StormBenchConfig; -import org.apache.storm.Config; -import org.apache.storm.StormSubmitter; -import org.apache.storm.topology.TopologyBuilder; -import org.apache.storm.topology.base.BaseRichSpout; - -public abstract class SingleSpoutTops { - - protected StormBenchConfig config; - - public SingleSpoutTops(StormBenchConfig config) { - this.config = config; - } - - public void run() throws Exception { - StormSubmitter.submitTopology(config.benchName, getConf(), getBuilder().createTopology()); - } - - private Config getConf() { - Config conf = new Config(); - conf.setNumWorkers(config.workerCount); - conf.put(Config.TOPOLOGY_BACKPRESSURE_ENABLE, false); - if (!config.ackon) { - conf.setNumAckers(0); +import backtype.storm.topology.*; +import backtype.storm.*; + +import com.intel.hibench.streambench.storm.util.*; + +public class SingleSpoutTops extends AbstractStormSpoutTops { + + public SingleSpoutTops(StormBenchConfig c) { + super(c); + } + + public void run() throws Exception { + StormSubmitter.submitTopology(config.benchName, getConf(), getBuilder().createTopology()); + } + + public Config getConf() { + Config conf = new Config(); + conf.setMaxTaskParallelism(200); + conf.put("topology.spout.max.batch.size", 64 * 1024); + conf.setNumWorkers(config.workerCount); + if (!config.ackon) + conf.setNumAckers(0); + return conf; + } + + public TopologyBuilder getBuilder() { + TopologyBuilder builder = new TopologyBuilder(); + setSpout(builder); + setBolt(builder); + return builder; } - return conf; - } - - public TopologyBuilder getBuilder() { - TopologyBuilder builder = new TopologyBuilder(); - BaseRichSpout spout = KafkaSpoutFactory.getSpout(config); - builder.setSpout("spout", spout, config.spoutThreads); - setBolts(builder); - return builder; - } - - public abstract void setBolts(TopologyBuilder builder); } diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/topologies/SingleTridentSpoutTops.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/topologies/SingleTridentSpoutTops.java index 990bfab3f..eb2d8d90c 100644 --- a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/topologies/SingleTridentSpoutTops.java +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/topologies/SingleTridentSpoutTops.java @@ -17,28 +17,37 @@ package com.intel.hibench.streambench.storm.topologies; -import com.intel.hibench.streambench.storm.util.StormBenchConfig; -import org.apache.storm.Config; -import org.apache.storm.StormSubmitter; -import org.apache.storm.trident.TridentTopology; +import backtype.storm.*; +import storm.trident.TridentTopology; -public abstract class SingleTridentSpoutTops { +import com.intel.hibench.streambench.storm.util.*; - protected StormBenchConfig config; +public class SingleTridentSpoutTops extends AbstractTridentSpoutTops { - public SingleTridentSpoutTops(StormBenchConfig config) { - this.config = config; - } + protected StormBenchConfig config; - public void run() throws Exception { - StormSubmitter.submitTopology(config.benchName, getConf(), createTopology().build()); - } + public SingleTridentSpoutTops(StormBenchConfig c) { + this.config = c; + } - private Config getConf() { - Config conf = new Config(); - conf.setNumWorkers(config.workerCount); - return conf; - } + public void run() throws Exception { + StormSubmitter.submitTopology(config.benchName, getConf(), createTridentTopology().build()); + } + + public Config getConf() { + Config conf = new Config(); + conf.setMaxTaskParallelism(200); + conf.put("topology.spout.max.batch.size", 64 * 1024); + conf.setNumWorkers(config.workerCount); + if (!config.ackon) + conf.setNumAckers(0); + return conf; + } + + public TridentTopology createTridentTopology() { + TridentTopology topology = new TridentTopology(); + setTopology(topology); + return topology; + } - public abstract TridentTopology createTopology(); } diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/trident/Sketch.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/trident/Sketch.java new file mode 100644 index 000000000..9c44cc65f --- /dev/null +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/trident/Sketch.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.intel.hibench.streambench.storm.trident; + +import backtype.storm.Config; +import backtype.storm.LocalCluster; +import backtype.storm.LocalDRPC; +import backtype.storm.StormSubmitter; +import backtype.storm.generated.StormTopology; +import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Values; +import backtype.storm.utils.*; +import backtype.storm.topology.base.BaseRichSpout; + +import storm.trident.TridentState; +import storm.trident.TridentTopology; +import storm.trident.operation.BaseFunction; +import storm.trident.operation.TridentCollector; +import storm.trident.operation.builtin.Count; +import storm.trident.operation.builtin.FilterNull; +import storm.trident.operation.builtin.MapGet; +import storm.trident.operation.builtin.Sum; +import storm.trident.testing.FixedBatchSpout; +import storm.trident.testing.MemoryMapState; +import storm.trident.tuple.TridentTuple; +import storm.kafka.trident.*; + + +import com.intel.hibench.streambench.storm.util.*; +import com.intel.hibench.streambench.storm.spout.*; +import com.intel.hibench.streambench.storm.topologies.*; + +public class Sketch extends BaseFunction { + private int fieldIndex; + private String separator; + + public Sketch(int fieldIndex, String separator) { + this.fieldIndex = fieldIndex; + this.separator = separator; + } + + @Override + public void execute(TridentTuple tuple, TridentCollector collector){ + String record = tuple.getString(0); + String[] fields = record.split(separator); + if (fields.length > fieldIndex) + collector.emit(new Values(fields[fieldIndex])); + } +} diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/trident/TridentDistinctCount.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/trident/TridentDistinctCount.java new file mode 100644 index 000000000..6afacd08c --- /dev/null +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/trident/TridentDistinctCount.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.intel.hibench.streambench.storm.trident; + +import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Values; +import storm.trident.TridentTopology; +import storm.trident.operation.BaseFunction; +import storm.trident.operation.TridentCollector; + +import storm.trident.tuple.TridentTuple; +import storm.kafka.trident.*; + +import com.intel.hibench.streambench.storm.util.*; +import com.intel.hibench.streambench.storm.spout.*; +import com.intel.hibench.streambench.storm.topologies.*; + +import java.util.Set; +import java.util.HashSet; + +public class TridentDistinctCount extends SingleTridentSpoutTops { + + public TridentDistinctCount(StormBenchConfig config) { + super(config); + } + + @Override + public void setTopology(TridentTopology topology) { + OpaqueTridentKafkaSpout spout = ConstructSpoutUtil.constructTridentSpout(); + + topology + .newStream("bg0", spout) + .each(spout.getOutputFields(), new Sketch(config.fieldIndex, config.separator), new Fields("field")) + .parallelismHint(config.spoutThreads) + .partitionBy(new Fields("field")) + .each(new Fields("field"), new DistinctCount(), new Fields("size")) + .parallelismHint(config.workerCount); + } + + public static class DistinctCount extends BaseFunction { + Set set = new HashSet(); + + @Override + public void execute(TridentTuple tuple, TridentCollector collector) { + String word = tuple.getString(0); + set.add(word); + BenchLogUtil.logMsg("Distinct count:"+set.size()); + collector.emit(new Values(set.size())); + } + + + } +} diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/trident/TridentGrep.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/trident/TridentGrep.java new file mode 100644 index 000000000..e61a0c33a --- /dev/null +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/trident/TridentGrep.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.intel.hibench.streambench.storm.trident; + +import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Values; + +import storm.trident.TridentTopology; +import storm.trident.operation.BaseFunction; +import storm.trident.operation.TridentCollector; + +import storm.trident.tuple.TridentTuple; +import storm.kafka.trident.*; + +import com.intel.hibench.streambench.storm.util.*; +import com.intel.hibench.streambench.storm.spout.*; +import com.intel.hibench.streambench.storm.topologies.*; + +public class TridentGrep extends SingleTridentSpoutTops { + + public TridentGrep(StormBenchConfig config){ + super(config); + } + + @Override + public void setTopology(TridentTopology topology) { + OpaqueTridentKafkaSpout spout = ConstructSpoutUtil.constructTridentSpout(); + + topology + .newStream("bg0", spout) + .each(spout.getOutputFields(), new Grep(config.pattern), new Fields("tuple")) + .parallelismHint(config.workerCount); + } + + public static class Grep extends BaseFunction { + private String pattern; + + public Grep(String pattern) { + this.pattern = pattern; + } + + @Override + public void execute(TridentTuple tuple, TridentCollector collector){ + String val = tuple.getString(0); + if (val.contains(pattern)) + // BenchLogUtil.logMsg(val); + collector.emit(new Values(val)); + } + } +} diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/trident/TridentIdentity.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/trident/TridentIdentity.java index 223b203aa..aa887773a 100644 --- a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/trident/TridentIdentity.java +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/trident/TridentIdentity.java @@ -17,30 +17,37 @@ package com.intel.hibench.streambench.storm.trident; -import com.intel.hibench.streambench.storm.spout.KafkaSpoutFactory; -import com.intel.hibench.streambench.storm.topologies.SingleTridentSpoutTops; -import com.intel.hibench.streambench.storm.trident.functions.Identity; -import com.intel.hibench.streambench.storm.util.StormBenchConfig; -import org.apache.storm.trident.TridentTopology; -import org.apache.storm.trident.spout.ITridentDataSource; +import backtype.storm.tuple.*; -public class TridentIdentity extends SingleTridentSpoutTops { +import storm.trident.TridentTopology; +import storm.trident.operation.BaseFunction; +import storm.trident.operation.TridentCollector; +import storm.trident.tuple.TridentTuple; +import storm.kafka.trident.*; + +import com.intel.hibench.streambench.storm.util.*; +import com.intel.hibench.streambench.storm.spout.*; +import com.intel.hibench.streambench.storm.topologies.*; - public TridentIdentity(StormBenchConfig config) { +public class TridentIdentity extends SingleTridentSpoutTops { + + public TridentIdentity(StormBenchConfig config){ super(config); } @Override - public TridentTopology createTopology() { - ITridentDataSource source = KafkaSpoutFactory.getTridentSpout(config, true); + public void setTopology(TridentTopology topology) { + OpaqueTridentKafkaSpout spout = ConstructSpoutUtil.constructTridentSpout(); - TridentTopology topology = new TridentTopology(); - - topology.newStream("kafka", source) - .map(new Identity(config) - ) - .parallelismHint(config.boltThreads); - return topology; + topology + .newStream("bg0", spout) + .each(spout.getOutputFields(), new Identity(), new Fields("tuple")) + .parallelismHint(config.workerCount); + } + public static class Identity extends BaseFunction { + @Override + public void execute(TridentTuple tuple, TridentCollector collector){ + collector.emit(new Values(tuple.getValues())); + } } - } diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/trident/TridentNumericCalc.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/trident/TridentNumericCalc.java new file mode 100644 index 000000000..adc6894bb --- /dev/null +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/trident/TridentNumericCalc.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.intel.hibench.streambench.storm.trident; + +import java.io.Serializable; + +import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Values; + +import com.intel.hibench.streambench.storm.spout.ConstructSpoutUtil; +import com.intel.hibench.streambench.storm.topologies.SingleTridentSpoutTops; +import com.intel.hibench.streambench.storm.util.StormBenchConfig; +import storm.trident.TridentTopology; +import storm.trident.operation.*; +import storm.trident.testing.MemoryMapState; +import storm.trident.tuple.TridentTuple; +import storm.kafka.trident.*; + + +class Numeric implements Serializable { + public Long max = 0L; + public Long min = 10000L; + public Long sum = 0L; + public Long count = 0L; + public Numeric() {} + public Numeric(Long max, Long min, Long sum, Long count) { + this.max = max; + this.min = min; + this.sum = sum; + this.count = count; + } +} + +class NumericCalc implements CombinerAggregator, Serializable { + + @Override + public Numeric init(TridentTuple tuple) { + if (tuple.contains("number")) { + Long val = tuple.getLong(0); + return new Numeric(val, val, val, 1L); + } + return new Numeric(); + } + + @Override + public Numeric combine(Numeric val1, Numeric val2) { + if (val1.max < val2.max) val1.max = val2.max; + if (val1.min > val2.min) val1.min = val2.min; + val1.sum += val2.sum; + val1.count += val2.count; + System.out.println(val1.max + " " + val1.min + " " + val1.sum + " " + val1.count); + return val1; + } + + @Override + public Numeric zero() { + return new Numeric(); + } +} + +public class TridentNumericCalc extends SingleTridentSpoutTops implements Serializable { + + public TridentNumericCalc(StormBenchConfig config){ + super(config); + } + + @Override + public void setTopology(TridentTopology topology) { + OpaqueTridentKafkaSpout spout = ConstructSpoutUtil.constructTridentSpout(); + + topology + .newStream("bg0", spout) + .parallelismHint(config.spoutThreads) + .each(spout.getOutputFields(), new Trim(config.separator, config.fieldIndex), new Fields("number")) + .persistentAggregate(new MemoryMapState.Factory(), new Fields("number"), new NumericCalc(), new Fields("res")) + ; + } + + public static class Trim extends BaseFunction { + String separator; + int fieldIndex; + + public Trim(String separator, int fieldIndex) { + this.separator = separator; + this.fieldIndex = fieldIndex; + } + + @Override + public void execute(TridentTuple tuple, TridentCollector collector) { + String record = tuple.getString(0); + String[] fields = record.trim().split(separator); + if (fields.length > fieldIndex) { + Long val = Long.parseLong(fields[fieldIndex]); + collector.emit(new Values(val)); + } + } + } + +} diff --git a/src/streambench/common/src/main/java/com/intel/hibench/streambench/common/KMeansData.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/trident/TridentProject.java similarity index 50% rename from src/streambench/common/src/main/java/com/intel/hibench/streambench/common/KMeansData.java rename to src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/trident/TridentProject.java index 1768e12bb..a9fb1fff9 100644 --- a/src/streambench/common/src/main/java/com/intel/hibench/streambench/common/KMeansData.java +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/trident/TridentProject.java @@ -15,22 +15,30 @@ * limitations under the License. */ -package com.intel.hibench.streambench.common; +package com.intel.hibench.streambench.storm.trident; -public class KMeansData { - long id; - double[] data; +import backtype.storm.tuple.Fields; +import storm.trident.TridentTopology; +import storm.kafka.trident.*; - public KMeansData(long id, double[] data) { - this.id = id; - this.data = data; - } +import com.intel.hibench.streambench.storm.util.*; +import com.intel.hibench.streambench.storm.spout.*; +import com.intel.hibench.streambench.storm.topologies.*; + +public class TridentProject extends SingleTridentSpoutTops { - public double[] getData() { - return data; + public TridentProject(StormBenchConfig config) { + super(config); } - public long getId() { - return id; + @Override + public void setTopology(TridentTopology topology) { + OpaqueTridentKafkaSpout spout = ConstructSpoutUtil.constructTridentSpout(); + + topology + .newStream("bg0", spout) + .each(spout.getOutputFields(), new Sketch(config.fieldIndex, config.separator), new Fields("field")) + .parallelismHint(config.workerCount); } + } diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/trident/TridentRepartition.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/trident/TridentRepartition.java deleted file mode 100644 index b9be06d95..000000000 --- a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/trident/TridentRepartition.java +++ /dev/null @@ -1,46 +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.intel.hibench.streambench.storm.trident; - -import com.intel.hibench.streambench.storm.spout.KafkaSpoutFactory; -import com.intel.hibench.streambench.storm.topologies.SingleTridentSpoutTops; -import com.intel.hibench.streambench.storm.trident.functions.Identity; -import com.intel.hibench.streambench.storm.util.StormBenchConfig; -import org.apache.storm.trident.TridentTopology; -import org.apache.storm.trident.spout.ITridentDataSource; - -public class TridentRepartition extends SingleTridentSpoutTops { - - public TridentRepartition(StormBenchConfig config) { - super(config); - } - - @Override - public TridentTopology createTopology() { - ITridentDataSource source = KafkaSpoutFactory.getTridentSpout(config, true); - - TridentTopology topology = new TridentTopology(); - - topology.newStream("kafka", source) - .parallelismHint(config.spoutThreads) - .shuffle() - .map(new Identity(config)) - .parallelismHint(config.boltThreads); - return topology; - } -} diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/trident/TridentSample.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/trident/TridentSample.java new file mode 100644 index 000000000..5b90cb8d0 --- /dev/null +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/trident/TridentSample.java @@ -0,0 +1,81 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.intel.hibench.streambench.storm.trident; + +import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Values; +import storm.trident.TridentTopology; +import storm.trident.operation.BaseFunction; +import storm.trident.operation.TridentCollector; + +import storm.trident.tuple.TridentTuple; +import storm.kafka.trident.*; + + +import com.intel.hibench.streambench.storm.util.*; +import com.intel.hibench.streambench.storm.spout.*; +import com.intel.hibench.streambench.storm.topologies.*; + +import java.util.Random; + +public class TridentSample extends SingleTridentSpoutTops { + private double probability; + public TridentSample(StormBenchConfig config){ + super(config); + } + + @Override + public void setTopology(TridentTopology topology) { + OpaqueTridentKafkaSpout spout = ConstructSpoutUtil.constructTridentSpout(); + + topology + .newStream("bg0", spout) + .each(spout.getOutputFields(), new Sample(config.prob), new Fields("tuple")) + .parallelismHint(config.workerCount); + } + public static class Sample extends BaseFunction { + private double probability; + private int count = 0; + private ThreadLocal rand = null; + + public Sample(double prob) { + probability = prob; + rand = threadRandom(1); + } + + @Override + public void execute(TridentTuple tuple, TridentCollector collector){ + double randVal = rand.get().nextDouble(); + if (randVal <= probability) { + count += 1; + collector.emit(new Values(tuple.getString(0))); + BenchLogUtil.logMsg(" count:" + count); + } + } + + public static ThreadLocal threadRandom(final long seed) { + return new ThreadLocal(){ + @Override + protected Random initialValue() { + return new Random(seed); + } + }; + } + + } +} diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/trident/TridentWindow.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/trident/TridentWindow.java deleted file mode 100644 index b7651eeca..000000000 --- a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/trident/TridentWindow.java +++ /dev/null @@ -1,103 +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.intel.hibench.streambench.storm.trident; - -import com.intel.hibench.streambench.common.metrics.KafkaReporter; -import com.intel.hibench.streambench.common.metrics.LatencyReporter; -import com.intel.hibench.streambench.storm.spout.KafkaSpoutFactory; -import com.intel.hibench.streambench.storm.topologies.SingleTridentSpoutTops; -import com.intel.hibench.streambench.storm.trident.functions.Parser; -import com.intel.hibench.streambench.storm.util.StormBenchConfig; -import org.apache.storm.topology.base.BaseWindowedBolt; -import org.apache.storm.trident.TridentTopology; -import org.apache.storm.trident.operation.BaseAggregator; -import org.apache.storm.trident.operation.TridentCollector; -import org.apache.storm.trident.operation.TridentOperationContext; -import org.apache.storm.trident.spout.ITridentDataSource; -import org.apache.storm.trident.tuple.TridentTuple; -import org.apache.storm.trident.windowing.InMemoryWindowsStoreFactory; -import org.apache.storm.tuple.Fields; -import org.apache.storm.tuple.Values; - -import java.util.Map; -import java.util.concurrent.TimeUnit; - -public class TridentWindow extends SingleTridentSpoutTops { - - public TridentWindow(StormBenchConfig config) { - super(config); - } - - @Override - public TridentTopology createTopology() { - ITridentDataSource source = KafkaSpoutFactory.getTridentSpout(config, true); - - TridentTopology topology = new TridentTopology(); - topology.newStream("kafka", source) - .each(new Fields("str"), new Parser(), new Fields("ip", "time")) - .project(new Fields("ip", "time")) - .parallelismHint(config.spoutThreads) - .groupBy(new Fields("ip")).toStream() - .slidingWindow(new BaseWindowedBolt.Duration((int) config.windowDuration, TimeUnit.MILLISECONDS), - new BaseWindowedBolt.Duration((int) config.windowSlideStep, TimeUnit.MILLISECONDS), - new InMemoryWindowsStoreFactory(), - new Fields("ip", "time"), new Count(config), new Fields("ip", "count")) - .parallelismHint(config.boltThreads); - return topology; - } - - private static class Count extends BaseAggregator { - - private final StormBenchConfig config; - private LatencyReporter reporter = null; - - Count(StormBenchConfig config) { - this.config = config; - } - - static class State { - String ip; - long minTime = Long.MAX_VALUE; - long count = 0L; - } - - @Override - public void prepare(Map conf, TridentOperationContext context) { - this.reporter = new KafkaReporter(config.reporterTopic, config.brokerList); - } - - @Override - public State init(Object batchId, TridentCollector tridentCollector) { - return new State(); - } - - @Override - public void aggregate(State state, TridentTuple tridentTuple, TridentCollector tridentCollector) { - state.ip = tridentTuple.getString(0); - state.count++; - state.minTime = Math.min(tridentTuple.getLong(1), state.minTime); - } - - @Override - public void complete(State state, TridentCollector tridentCollector) { - tridentCollector.emit(new Values(state.ip, state.count)); - for (int i = 0; i < state.count; i++) { - reporter.report(state.minTime, System.currentTimeMillis()); - } - } - } -} diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/trident/TridentWordcount.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/trident/TridentWordcount.java index c68f50b9f..466ce7e98 100644 --- a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/trident/TridentWordcount.java +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/trident/TridentWordcount.java @@ -17,67 +17,73 @@ package com.intel.hibench.streambench.storm.trident; -import com.intel.hibench.streambench.common.metrics.KafkaReporter; -import com.intel.hibench.streambench.common.metrics.LatencyReporter; -import com.intel.hibench.streambench.storm.spout.KafkaSpoutFactory; +import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Values; + +import com.intel.hibench.streambench.storm.spout.ConstructSpoutUtil; import com.intel.hibench.streambench.storm.topologies.SingleTridentSpoutTops; -import com.intel.hibench.streambench.storm.trident.functions.Parser; +import com.intel.hibench.streambench.storm.util.BenchLogUtil; import com.intel.hibench.streambench.storm.util.StormBenchConfig; -import org.apache.storm.trident.TridentTopology; -import org.apache.storm.trident.operation.ReducerAggregator; -import org.apache.storm.trident.spout.ITridentDataSource; -import org.apache.storm.trident.testing.MemoryMapState; -import org.apache.storm.trident.tuple.TridentTuple; -import org.apache.storm.tuple.Fields; +import storm.trident.TridentTopology; +import storm.trident.operation.BaseFunction; +import storm.trident.operation.TridentCollector; +import storm.trident.tuple.TridentTuple; +import storm.kafka.trident.*; -public class TridentWordcount extends SingleTridentSpoutTops { - public TridentWordcount(StormBenchConfig config) { +import java.util.Map; +import java.util.HashMap; + +public class TridentWordcount extends SingleTridentSpoutTops { + + public TridentWordcount(StormBenchConfig config){ super(config); } @Override - public TridentTopology createTopology() { - ITridentDataSource source = KafkaSpoutFactory.getTridentSpout(config, true); + public void setTopology(TridentTopology topology) { + OpaqueTridentKafkaSpout spout = ConstructSpoutUtil.constructTridentSpout(); - TridentTopology topology = new TridentTopology(); - topology.newStream("kafka", source) - .each(new Fields("str"), new Parser(), new Fields("ip", "time")) - .project(new Fields("ip", "time")) - .parallelismHint(config.spoutThreads) - .groupBy(new Fields("ip")) - .persistentAggregate(new MemoryMapState.Factory(), new Fields("ip", "time"), new Count(config), - new Fields("word", "count")) - .parallelismHint(config.boltThreads); - return topology; + topology + .newStream("bg0", spout) + .each(spout.getOutputFields(), new Split(config.separator), new Fields("words")) + .parallelismHint(config.spoutThreads) + .partitionBy(new Fields("words")) + .each(new Fields("words"), new WordCount(), new Fields("word", "count")) + .parallelismHint(config.workerCount) + ; } - private static class Count implements ReducerAggregator { - - private final StormBenchConfig config; - private LatencyReporter reporter = null; + public static class Split extends BaseFunction { + String separator; - Count(StormBenchConfig config) { - this.config = config; + public Split(String separator) { + this.separator = separator; } @Override - public State init() { - this.reporter = new KafkaReporter(config.reporterTopic, config.brokerList); - return new State(); + public void execute(TridentTuple tuple, TridentCollector collector) { + String sentence = tuple.getString(0); + for (String word : sentence.split(separator)) { + collector.emit(new Values(word)); + } } + } - @Override - public State reduce(State state, TridentTuple tridentTuple) { - state.ip = tridentTuple.getString(0); - state.count++; - reporter.report(tridentTuple.getLong(1), System.currentTimeMillis()); - return state; - } + public static class WordCount extends BaseFunction { + Map counts = new HashMap(); - static class State { - String ip; - long count = 0; + @Override + public void execute(TridentTuple tuple, TridentCollector collector) { + String word = tuple.getString(0); + Integer count = counts.get(word); + if (count == null) + count = 0; + count++; + counts.put(word, count); + BenchLogUtil.logMsg("Word:" + word + " count:" + count); + collector.emit(new Values(word, count)); } } + } diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/trident/functions/Identity.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/trident/functions/Identity.java deleted file mode 100644 index cf75a30bd..000000000 --- a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/trident/functions/Identity.java +++ /dev/null @@ -1,44 +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.intel.hibench.streambench.storm.trident.functions; - -import com.google.common.collect.ImmutableMap; -import com.intel.hibench.streambench.common.metrics.KafkaReporter; -import com.intel.hibench.streambench.common.metrics.LatencyReporter; -import com.intel.hibench.streambench.storm.util.StormBenchConfig; -import org.apache.storm.trident.operation.MapFunction; -import org.apache.storm.trident.tuple.TridentTuple; -import org.apache.storm.tuple.Values; - - -public class Identity implements MapFunction { - - private final StormBenchConfig config; - - public Identity(StormBenchConfig config) { - this.config = config; - } - - @Override - public Values execute(TridentTuple tridentTuple) { - ImmutableMap kv = (ImmutableMap) tridentTuple.getValue(0); - LatencyReporter reporter = new KafkaReporter(config.reporterTopic, config.brokerList); - reporter.report(Long.parseLong(kv.keySet().iterator().next()), - System.currentTimeMillis()); - return new Values(kv); - } -} diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/trident/functions/Parser.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/trident/functions/Parser.java deleted file mode 100644 index cb5bbe5ee..000000000 --- a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/trident/functions/Parser.java +++ /dev/null @@ -1,37 +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.intel.hibench.streambench.storm.trident.functions; - -import com.google.common.collect.ImmutableMap; -import com.intel.hibench.streambench.common.UserVisit; -import com.intel.hibench.streambench.common.UserVisitParser; -import org.apache.storm.trident.operation.BaseFunction; -import org.apache.storm.trident.operation.TridentCollector; -import org.apache.storm.trident.tuple.TridentTuple; -import org.apache.storm.tuple.Values; - -public class Parser extends BaseFunction { - - @Override - public void execute(TridentTuple tuple, TridentCollector collector) { - ImmutableMap kv = (ImmutableMap) tuple.getValue(0); - String key = kv.keySet().iterator().next(); - Long startTime = Long.parseLong(key); - UserVisit uv = UserVisitParser.parse(kv.get(key)); - collector.emit(new Values(uv.getIp(), startTime)); - } -} diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/util/BenchLogUtil.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/util/BenchLogUtil.java index d1daa58e1..91174eb45 100644 --- a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/util/BenchLogUtil.java +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/util/BenchLogUtil.java @@ -1,35 +1,34 @@ package com.intel.hibench.streambench.storm.util; -import java.io.File; -import java.io.PrintWriter; +import java.io.*; -public class BenchLogUtil { +public class BenchLogUtil{ private static PrintWriter out; - - public static void init() throws Exception { - File file = new File("/tmp/benchlog-storm.txt"); - out = new PrintWriter(file); + + public static void init() throws Exception{ + File file=new File("/tmp/benchlog-storm.txt"); + out=new PrintWriter(file); } - - public static void logMsg(String msg) { - try { - if (out == null) - init(); - } catch (Exception e) { - e.printStackTrace(); - } - + + public static void logMsg(String msg){ + try{ + if(out==null) + init(); + }catch(Exception e){ + e.printStackTrace(); + } + out.println(msg); - out.flush(); - System.out.println(msg); + out.flush(); + System.out.println(msg); } - - public static void close() { - if (out != null) out.close(); + + public static void close(){ + if(out!=null) out.close(); } - - public static void handleError(String msg) { + + public static void handleError(String msg){ System.err.println(msg); - System.exit(1); + System.exit(1); } } \ No newline at end of file diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/util/ConfigLoader.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/util/ConfigLoader.java new file mode 100644 index 000000000..57d663652 --- /dev/null +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/util/ConfigLoader.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.intel.hibench.streambench.storm.util; + + +import java.io.BufferedReader; +import java.io.FileNotFoundException; +import java.io.FileReader; +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +public class ConfigLoader { + private String ConfigFileName = null; + private Map store; + + public ConfigLoader(String filename){ + ConfigFileName = filename; + store = new HashMap(); + // Load and parse config + try { + BufferedReader br = new BufferedReader(new FileReader(filename)); + String line = br.readLine(); + while(line != null){ + if ((line.length()>0) && (line.charAt(0)!='#')) { + String[] words = line.split("\\s+"); + if (words.length == 2) { + String key = words[0]; + String value = words[1]; + store.put(key, value); + } else if (words.length == 1) { + String key = words[0]; + store.put(key, ""); + } else { + if (!line.startsWith("hibench")) + System.out.println("Warning: unknown config parsed, skip:" + line); + } + } + line = br.readLine(); + } + } catch (FileNotFoundException e) { + System.out.println("ERROR: Config file not found! Should not happen. Caused by:"); + } catch (IOException e) { + System.out.println("ERROR: IO exception during read file. Should not happen. Caused by:"); + e.printStackTrace(); + } + } + + public String getProperty(String key){ + if (store.containsKey(key)) + return (String) store.get(key); + else { + System.out.println("ERROR: Unknown config key:" + key); + return null; + } + } +} diff --git a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/util/StormBenchConfig.java b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/util/StormBenchConfig.java index a6b67fdff..179f2c2d1 100644 --- a/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/util/StormBenchConfig.java +++ b/src/streambench/stormbench/src/main/java/com/intel/hibench/streambench/storm/util/StormBenchConfig.java @@ -1,25 +1,23 @@ package com.intel.hibench.streambench.storm.util; -import java.io.Serializable; - -public class StormBenchConfig implements Serializable { +public class StormBenchConfig{ + public String nimbus; + public int nimbusAPIPort; public String zkHost; public int workerCount; public int spoutThreads; public int boltThreads; public String benchName; + public long recordCount; public String topic; public String consumerGroup; + public boolean readFromStart; public boolean ackon; - public boolean localShuffle; - public String brokerList; - public String reporterTopic; - - public long windowDuration; - public long windowSlideStep; - + public int nimbusContactInterval; + //Following are fields that are benchmark specific - public String separator = "\\s+"; - public int fieldIndex = 1; - public String pattern = " "; + public String separator; + public int fieldIndex; + public double prob; + public String pattern; } \ No newline at end of file diff --git a/src/streambench/stormbench/src/main/thrift/storm.thrift b/src/streambench/stormbench/src/main/thrift/storm.thrift new file mode 100644 index 000000000..48e3548bf --- /dev/null +++ b/src/streambench/stormbench/src/main/thrift/storm.thrift @@ -0,0 +1,255 @@ +#!/usr/local/bin/thrift --gen java:beans,nocamel,hashcode + +namespace java com.intel.hibench.streambench.storm.thrift.generated + +union JavaObjectArg { + 1: i32 int_arg; + 2: i64 long_arg; + 3: string string_arg; + 4: bool bool_arg; + 5: binary binary_arg; + 6: double double_arg; +} + +struct JavaObject { + 1: required string full_class_name; + 2: required list args_list; +} + +struct NullStruct { + +} + +struct GlobalStreamId { + 1: required string componentId; + 2: required string streamId; + #Going to need to add an enum for the stream type (NORMAL or FAILURE) +} + +union Grouping { + 1: list fields; //empty list means global grouping + 2: NullStruct shuffle; // tuple is sent to random task + 3: NullStruct all; // tuple is sent to every task + 4: NullStruct none; // tuple is sent to a single task (storm's choice) -> allows storm to optimize the topology by bundling tasks into a single process + 5: NullStruct direct; // this bolt expects the source bolt to send tuples directly to it + 6: JavaObject custom_object; + 7: binary custom_serialized; + 8: NullStruct local_or_shuffle; // prefer sending to tasks in the same worker process, otherwise shuffle +} + +struct StreamInfo { + 1: required list output_fields; + 2: required bool direct; +} + +struct ShellComponent { + // should change this to 1: required list execution_command; + 1: string execution_command; + 2: string script; +} + +union ComponentObject { + 1: binary serialized_java; + 2: ShellComponent shell; + 3: JavaObject java_object; +} + +struct ComponentCommon { + 1: required map inputs; + 2: required map streams; //key is stream id + 3: optional i32 parallelism_hint; //how many threads across the cluster should be dedicated to this component + + // component specific configuration respects: + // topology.debug: false + // topology.max.task.parallelism: null // can replace isDistributed with this + // topology.max.spout.pending: null + // topology.kryo.register // this is the only additive one + + // component specific configuration + 4: optional string json_conf; +} + +struct SpoutSpec { + 1: required ComponentObject spout_object; + 2: required ComponentCommon common; + // can force a spout to be non-distributed by overriding the component configuration + // and setting TOPOLOGY_MAX_TASK_PARALLELISM to 1 +} + +struct Bolt { + 1: required ComponentObject bolt_object; + 2: required ComponentCommon common; +} + +// not implemented yet +// this will eventually be the basis for subscription implementation in storm +struct StateSpoutSpec { + 1: required ComponentObject state_spout_object; + 2: required ComponentCommon common; +} + +struct StormTopology { + //ids must be unique across maps + // #workers to use is in conf + 1: required map spouts; + 2: required map bolts; + 3: required map state_spouts; +} + +exception AlreadyAliveException { + 1: required string msg; +} + +exception NotAliveException { + 1: required string msg; +} + +exception InvalidTopologyException { + 1: required string msg; +} + +struct TopologySummary { + 1: required string id; + 2: required string name; + 3: required i32 num_tasks; + 4: required i32 num_executors; + 5: required i32 num_workers; + 6: required i32 uptime_secs; + 7: required string status; +} + +struct SupervisorSummary { + 1: required string host; + 2: required i32 uptime_secs; + 3: required i32 num_workers; + 4: required i32 num_used_workers; + 5: required string supervisor_id; +} + +struct ClusterSummary { + 1: required list supervisors; + 2: required i32 nimbus_uptime_secs; + 3: required list topologies; +} + +struct ErrorInfo { + 1: required string error; + 2: required i32 error_time_secs; +} + +struct BoltStats { + 1: required map> acked; + 2: required map> failed; + 3: required map> process_ms_avg; + 4: required map> executed; + 5: required map> execute_ms_avg; +} + +struct SpoutStats { + 1: required map> acked; + 2: required map> failed; + 3: required map> complete_ms_avg; +} + +union ExecutorSpecificStats { + 1: BoltStats bolt; + 2: SpoutStats spout; +} + +// Stats are a map from the time window (all time or a number indicating number of seconds in the window) +// to the stats. Usually stats are a stream id to a count or average. +struct ExecutorStats { + 1: required map> emitted; + 2: required map> transferred; + 3: required ExecutorSpecificStats specific; +} + +struct ExecutorInfo { + 1: required i32 task_start; + 2: required i32 task_end; +} + +struct ExecutorSummary { + 1: required ExecutorInfo executor_info; + 2: required string component_id; + 3: required string host; + 4: required i32 port; + 5: required i32 uptime_secs; + 7: optional ExecutorStats stats; +} + +struct TopologyInfo { + 1: required string id; + 2: required string name; + 3: required i32 uptime_secs; + 4: required list executors; + 5: required string status; + 6: required map> errors; +} + +struct KillOptions { + 1: optional i32 wait_secs; +} + +struct RebalanceOptions { + 1: optional i32 wait_secs; + 2: optional i32 num_workers; + 3: optional map num_executors; +} + +enum TopologyInitialStatus { + ACTIVE = 1, + INACTIVE = 2 +} +struct SubmitOptions { + 1: required TopologyInitialStatus initial_status; +} + +service Nimbus { + void submitTopology(1: string name, 2: string uploadedJarLocation, 3: string jsonConf, 4: StormTopology topology) throws (1: AlreadyAliveException e, 2: InvalidTopologyException ite); + void submitTopologyWithOpts(1: string name, 2: string uploadedJarLocation, 3: string jsonConf, 4: StormTopology topology, 5: SubmitOptions options) throws (1: AlreadyAliveException e, 2: InvalidTopologyException ite); + void killTopology(1: string name) throws (1: NotAliveException e); + void killTopologyWithOpts(1: string name, 2: KillOptions options) throws (1: NotAliveException e); + void activate(1: string name) throws (1: NotAliveException e); + void deactivate(1: string name) throws (1: NotAliveException e); + void rebalance(1: string name, 2: RebalanceOptions options) throws (1: NotAliveException e, 2: InvalidTopologyException ite); + + // need to add functions for asking about status of storms, what nodes they're running on, looking at task logs + + string beginFileUpload(); + void uploadChunk(1: string location, 2: binary chunk); + void finishFileUpload(1: string location); + + string beginFileDownload(1: string file); + //can stop downloading chunks when receive 0-length byte array back + binary downloadChunk(1: string id); + + // returns json + string getNimbusConf(); + // stats functions + ClusterSummary getClusterInfo(); + TopologyInfo getTopologyInfo(1: string id) throws (1: NotAliveException e); + //returns json + string getTopologyConf(1: string id) throws (1: NotAliveException e); + StormTopology getTopology(1: string id) throws (1: NotAliveException e); + StormTopology getUserTopology(1: string id) throws (1: NotAliveException e); +} + +struct DRPCRequest { + 1: required string func_args; + 2: required string request_id; +} + +exception DRPCExecutionException { + 1: required string msg; +} + +service DistributedRPC { + string execute(1: string functionName, 2: string funcArgs) throws (1: DRPCExecutionException e); +} + +service DistributedRPCInvocations { + void result(1: string id, 2: string result); + DRPCRequest fetchRequest(1: string functionName); + void failRequest(1: string id); +} diff --git a/src/streambench/zkHelper/pom.xml b/src/streambench/zkHelper/pom.xml new file mode 100644 index 000000000..b7ddfb723 --- /dev/null +++ b/src/streambench/zkHelper/pom.xml @@ -0,0 +1,87 @@ + + 4.0.0 + + + com.intel.hibench + streambench + 5.0-SNAPSHOT + + + + 3.0.0 + + + com.intel.RPCcloud + streaming-bench-zkhelper + 0.1-SNAPSHOT + jar + Streaming Benchmark Zookeeper Helper Utils + + + + UTF-8 + + + + + org.apache.kafka + kafka_2.10 + 0.8.2.1 + + + org.slf4j + slf4j-api + ${slf4j.version} + + + org.slf4j + slf4j-log4j12 + ${slf4j.version} + + + + + + apache-releases + https://repository.apache.org/content/groups/public + + + + + src/main + + + org.apache.maven.plugins + maven-compiler-plugin + 3.1 + + 1.6 + 1.6 + + + + maven-assembly-plugin + + + jar-with-dependencies + + + + + + + + + + make-assembly + package + + single + + + + + + + diff --git a/src/streambench/common/src/main/java/com/intel/hibench/streambench/common/Platform.java b/src/streambench/zkHelper/src/main/com/intel/hibench/streambench/zkHelper/OffsetResetUpdater.java similarity index 73% rename from src/streambench/common/src/main/java/com/intel/hibench/streambench/common/Platform.java rename to src/streambench/zkHelper/src/main/com/intel/hibench/streambench/zkHelper/OffsetResetUpdater.java index dc3985d22..3f55455eb 100644 --- a/src/streambench/common/src/main/java/com/intel/hibench/streambench/common/Platform.java +++ b/src/streambench/zkHelper/src/main/com/intel/hibench/streambench/zkHelper/OffsetResetUpdater.java @@ -15,20 +15,20 @@ * limitations under the License. */ -package com.intel.hibench.streambench.common; +package com.intel.hibench.streambench.zkHelper; -public enum Platform { +import org.I0Itec.zkclient.DataUpdater; - SPARK("spark"), - FLINK("flink"), - STORM("storm"), - TRIDENT("trident"), - GEARPUMP("gearpump"), - SAMZA("samza"); +public class OffsetResetUpdater implements DataUpdater { - private final String platform; + T newOffset; + public OffsetResetUpdater(T offset){ + newOffset=offset; + } + + @Override + public T update(T l) { + return newOffset; + } - Platform(String platform) { - this.platform = platform; - } } diff --git a/src/streambench/flinkbench/src/main/java/com/intel/flinkbench/util/FlinkBenchConfig.java b/src/streambench/zkHelper/src/main/com/intel/hibench/streambench/zkHelper/ZKStringSerializer.java similarity index 52% rename from src/streambench/flinkbench/src/main/java/com/intel/flinkbench/util/FlinkBenchConfig.java rename to src/streambench/zkHelper/src/main/com/intel/hibench/streambench/zkHelper/ZKStringSerializer.java index 3bd69f8df..4639437fa 100644 --- a/src/streambench/flinkbench/src/main/java/com/intel/flinkbench/util/FlinkBenchConfig.java +++ b/src/streambench/zkHelper/src/main/com/intel/hibench/streambench/zkHelper/ZKStringSerializer.java @@ -15,30 +15,35 @@ * limitations under the License. */ -package com.intel.flinkbench.util; +package com.intel.hibench.streambench.zkHelper; -import java.io.Serializable; +import java.io.UnsupportedEncodingException; -public class FlinkBenchConfig implements Serializable{ - public String testCase; +import org.I0Itec.zkclient.exception.ZkMarshallingError; +import org.I0Itec.zkclient.serialize.ZkSerializer; - // Kafka related - public String zkHost; - public String brokerList; - public String topic; - public String consumerGroup; - public String offsetReset; - public String reportTopic; +public class ZKStringSerializer implements ZkSerializer { - public String separator; - public int fieldIndex; - public double prob; - public String pattern; - public String windowDuration; - public String windowSlideStep; + @Override + public Object deserialize(byte[] bytes) throws ZkMarshallingError { + if (bytes == null) + return null; + try { + return new String(bytes,"UTF-8"); + } catch (UnsupportedEncodingException e) { + e.printStackTrace(); + } + return null; + } - // Flink related - public long bufferTimeout; - public long checkpointDuration; + @Override + public byte[] serialize(Object data) throws ZkMarshallingError { + try { + return ((String)data).getBytes("UTF-8"); + } catch (UnsupportedEncodingException e) { + e.printStackTrace(); + } + return null; + } } diff --git a/src/streambench/zkHelper/src/main/com/intel/hibench/streambench/zkHelper/ZKUtil.java b/src/streambench/zkHelper/src/main/com/intel/hibench/streambench/zkHelper/ZKUtil.java new file mode 100644 index 000000000..4e9ad1173 --- /dev/null +++ b/src/streambench/zkHelper/src/main/com/intel/hibench/streambench/zkHelper/ZKUtil.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.intel.hibench.streambench.zkHelper; + +import org.I0Itec.zkclient.ZkClient; + +public class ZKUtil { + + ZkClient zkClient; + + public ZKUtil(String zkHost){ + zkClient=new ZkClient(zkHost,30*1000, 30*1000, new ZKStringSerializer()); + } + + public void resetOffsetToBegin(String path,int partitionCount,String newOffset){ + OffsetResetUpdater update=new OffsetResetUpdater(newOffset); + for(int i=0;i "); + return; + } + String op=args[0]; + String zkHost=args[1]; + String path=args[2]; + int partitionCount=Integer.parseInt(args[3]); + String offset="0"; + if(args.length>4){ + offset=args[4]; + } + + ZKUtil zk=new ZKUtil(zkHost); + if(op.equals("ls")){ + zk.lsOffsets(path, partitionCount); + }else if(op.equals("update")){ + zk.resetOffsetToBegin(path, partitionCount, offset); + }else{ + System.err.println("Unsupported operation!"); + } + } +} diff --git a/workloads/streambench/common/metrics_reader.sh b/workloads/streambench/common/metrics_reader.sh deleted file mode 100755 index de0ff6249..000000000 --- a/workloads/streambench/common/metrics_reader.sh +++ /dev/null @@ -1,36 +0,0 @@ -#!/bin/bash -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -workload_folder=`dirname "$0"` -workload_folder=`cd "${workload_folder}"; pwd` -root_folder=${workload_folder}/../../.. -root_folder=`cd "${root_folder}"; pwd` -common_folder=${root_folder}/src/streambench/common/target - - -KAFKA_FOLDER=/home/stream/mirror/kafka/kafka_2.11-0.8.2.2 -ZK_HOST="sr502:2181,sr503:2181,sr504:2181" -NUM_OF_RECORDS=5000000 -NUM_OF_THREADS=20 -REPORT_PATH=/home/stream/mirror/spark_metrics - -${KAFKA_FOLDER}/bin/kafka-topics.sh --zookeeper ${ZK_HOST} --list - -read -p "Please input the args: " topic - -java -cp ${common_folder}/streaming-bench-common-5.0-SNAPSHOT-jar-with-dependencies.jar \ - com.intel.hibench.streambench.common.metrics.MetricsReader \ - ${ZK_HOST} ${topic} ${REPORT_PATH} ${NUM_OF_RECORDS} ${NUM_OF_THREADS} \ No newline at end of file diff --git a/workloads/streambench/conf/00-streamingbench.conf b/workloads/streambench/conf/00-streamingbench.conf deleted file mode 100644 index ba408af3d..000000000 --- a/workloads/streambench/conf/00-streamingbench.conf +++ /dev/null @@ -1,32 +0,0 @@ -# default configurations here - -# export for shell script -# dummy input/output path -hibench.workload.input -hibench.workload.output - -# default scale factor -hibench.streambench.datagen.scale_factor 1 - -# default path setting for genearate data1 & data2 -hibench.streambench.datagen.data1.name Seed -hibench.streambench.datagen.data1.dir ${hibench.streambench.datagen.dir}/${hibench.streambench.datagen.data1.name} - -hibench.streambench.datagen.data2_cluster.dir ${hibench.streambench.datagen.dir}/Kmeans/Cluster -hibench.streambench.datagen.data2_samples.dir ${hibench.streambench.datagen.dir}/Kmeans/Samples - -# default data scale of streaming data -hibench.streambench.scale.profile large - -# data scale for data1 -hibench.workload.uservisits ${hibench.aggregation.${hibench.streambench.scale.profile}.uservisits} -hibench.workload.pages ${hibench.aggregation.${hibench.streambench.scale.profile}.pages} - -# data scale for data2 -hibench.kmeans.num_of_clusters ${hibench.kmeans.${hibench.streambench.scale.profile}.num_of_clusters} -hibench.kmeans.dimensions ${hibench.kmeans.${hibench.streambench.scale.profile}.dimensions} -hibench.kmeans.num_of_samples ${hibench.kmeans.${hibench.streambench.scale.profile}.num_of_samples} -hibench.kmeans.samples_per_inputfile ${hibench.kmeans.${hibench.streambench.scale.profile}.samples_per_inputfile} -hibench.kmeans.max_iteration ${hibench.kmeans.${hibench.streambench.scale.profile}.max_iteration} -hibench.kmeans.k ${hibench.kmeans.${hibench.streambench.scale.profile}.k} -hibench.kmeans.convergedist ${hibench.kmeans.${hibench.streambench.scale.profile}.convergedist} diff --git a/workloads/streambench/conf/10-streamingbench-userdefine.conf b/workloads/streambench/conf/10-streamingbench-userdefine.conf deleted file mode 100644 index 62c93f3c1..000000000 --- a/workloads/streambench/conf/10-streamingbench-userdefine.conf +++ /dev/null @@ -1,34 +0,0 @@ -# override configurations for streaming releated workload here - -# General -hibench.streambench.testCase identity -hibench.streambench.debugMode false - -hibench.streambench.kafka.consumerGroup HiBench -hibench.streambench.kafka.topicPartitions 140 - -# Data Generator -hibench.streambench.datagen.intervalSpan 5 -hibench.streambench.datagen.recordsPerInterval 1 -hibench.streambench.datagen.recordLength 200 -hibench.streambench.datagen.producerNumber 1 - -# Spark -hibench.streambench.spark.batchInterval 200 - -hibench.spark.master spark://10.1.2.87:7077 -spark.executor.memory 80g - -# Flink -hibench.streambench.flink.parallelism 140 -hibench.streambench.flink.bufferTimeout -1 - -# Storm -hibench.streambench.storm.ackon false -hibench.streambench.storm.worker_count 28 -hibench.streambench.storm.spout_threads 70 -hibench.streambench.storm.bolt_threads 70 - -# Gearpump -hibench.streambench.gearpump.parallelism 140 -hibench.streambench.gearpump.executors 28 diff --git a/workloads/streambench/flink/bin/run.sh b/workloads/streambench/flink/bin/run.sh deleted file mode 100755 index 776fb9de5..000000000 --- a/workloads/streambench/flink/bin/run.sh +++ /dev/null @@ -1,35 +0,0 @@ -#!/bin/bash -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -workload_folder=`dirname "$0"` -workload_folder=`cd "$workload_folder"; pwd` -workload_root=${workload_folder}/../.. -echo $workload_root -. "${workload_root}/../../bin/functions/load-bench-config.sh" - -enter_bench FlinkStreamingBench ${workload_root} ${workload_folder} -show_bannar start - -cd ${workload_folder} - -printFullLog - -START_TIME=`timestamp` -run-flink-job -c com.intel.flinkbench.RunBench -END_TIME=`timestamp` - -gen_report ${START_TIME} ${END_TIME} 0 # FIXME, size should be throughput -show_bannar finish1 \ No newline at end of file diff --git a/workloads/streambench/gearpump/bin/run.sh b/workloads/streambench/gearpump/bin/run.sh deleted file mode 100755 index 30bfb187e..000000000 --- a/workloads/streambench/gearpump/bin/run.sh +++ /dev/null @@ -1,35 +0,0 @@ -#!/bin/bash -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -workload_folder=`dirname "$0"` -workload_folder=`cd "$workload_folder"; pwd` -workload_root=${workload_folder}/../.. -echo $workload_root -. "${workload_root}/../../bin/functions/load-bench-config.sh" - -enter_bench GearpumpStreamingBench ${workload_root} ${workload_folder} -show_bannar start - -cd ${workload_folder} - -printFullLog - -START_TIME=`timestamp` -run-gearpump-app com.intel.hibench.streambench.gearpump.RunBench ${SPARKBENCH_PROPERTIES_FILES} -END_TIME=`timestamp` - -gen_report ${START_TIME} ${END_TIME} 0 # FIXME, size should be throughput -show_bannar finish1 \ No newline at end of file diff --git a/workloads/streamingbench/conf/00-streamingbench.conf b/workloads/streamingbench/conf/00-streamingbench.conf new file mode 100644 index 000000000..cba732dce --- /dev/null +++ b/workloads/streamingbench/conf/00-streamingbench.conf @@ -0,0 +1,35 @@ +# default configurations here + +# export for shell script +# dummy input/output path +hibench.workload.input +hibench.workload.output + +# default scale factor +hibench.streamingbench.datagen.scale_factor 1 + +# default path setting for genearate data1 & data2 +hibench.streamingbench.datagen.data1.name Seed +hibench.streamingbench.datagen.data1.dir ${hibench.streamingbench.datagen.dir}/${hibench.streamingbench.datagen.data1.name} + +# Length limitation suggestion: 60 ~ 180 +hibench.streamingbench.datagen.data1.length 60 + +hibench.streamingbench.datagen.data2_cluster.dir ${hibench.streamingbench.datagen.dir}/Kmeans/Cluster +hibench.streamingbench.datagen.data2_samples.dir ${hibench.streamingbench.datagen.dir}/Kmeans/Samples + +# default data scale of streaming data +hibench.streamingbench.scale.profile large + +# data scale for data1 +hibench.workload.uservisits ${hibench.aggregation.${hibench.streamingbench.scale.profile}.uservisits} +hibench.workload.pages ${hibench.aggregation.${hibench.streamingbench.scale.profile}.pages} + +# data scale for data2 +hibench.kmeans.num_of_clusters ${hibench.kmeans.${hibench.streamingbench.scale.profile}.num_of_clusters} +hibench.kmeans.dimensions ${hibench.kmeans.${hibench.streamingbench.scale.profile}.dimensions} +hibench.kmeans.num_of_samples ${hibench.kmeans.${hibench.streamingbench.scale.profile}.num_of_samples} +hibench.kmeans.samples_per_inputfile ${hibench.kmeans.${hibench.streamingbench.scale.profile}.samples_per_inputfile} +hibench.kmeans.max_iteration ${hibench.kmeans.${hibench.streamingbench.scale.profile}.max_iteration} +hibench.kmeans.k ${hibench.kmeans.${hibench.streamingbench.scale.profile}.k} +hibench.kmeans.convergedist ${hibench.kmeans.${hibench.streamingbench.scale.profile}.convergedist} diff --git a/workloads/streamingbench/conf/10-streamingbench-userdefine.conf b/workloads/streamingbench/conf/10-streamingbench-userdefine.conf new file mode 100644 index 000000000..56d7242e5 --- /dev/null +++ b/workloads/streamingbench/conf/10-streamingbench-userdefine.conf @@ -0,0 +1,3 @@ + +# override configurations for streaming releated workload here + diff --git a/workloads/streambench/prepare/genSeedDataset.sh b/workloads/streamingbench/prepare/genSeedDataset.sh similarity index 100% rename from workloads/streambench/prepare/genSeedDataset.sh rename to workloads/streamingbench/prepare/genSeedDataset.sh diff --git a/workloads/streamingbench/prepare/gendata.sh b/workloads/streamingbench/prepare/gendata.sh new file mode 100755 index 000000000..5294ff968 --- /dev/null +++ b/workloads/streamingbench/prepare/gendata.sh @@ -0,0 +1,47 @@ +#!/bin/bash +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +workload_folder=`dirname "$0"` +workload_folder=`cd "$workload_folder"; pwd` +workload_root=${workload_folder}/.. +. "${workload_root}/../../bin/functions/load-bench-config.sh" + +enter_bench StreamingBenchPrepare ${workload_root} ${workload_folder} +show_bannar start + +DATA_GEN_DIR=${workload_root}/../../src/streambench/datagen +DATA_FILE1=${STREAMING_DATA1_DIR}/uservisits +DATA_FILE2=${STREAMING_DATA2_SAMPLE_DIR} + +#echo "=========begin gen stream data=========" +#echo "Topic:$topicName dataset:$app records:$records kafkaBrokers:$brokerList mode:$mode data_dir:$data_dir" + + +JVM_OPTS="-Xmx256M -server -XX:+UseCompressedOops -XX:+UseParNewGC -XX:+UseConcMarkSweepGC -XX:+CMSClassUnloadingEnabled -XX:+CMSScavengeBeforeRemark -XX:+DisableExplicitGC -Djava.awt.headless=true -Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.authenticate=false -Dcom.sun.management.jmxremote.ssl=false -Dkafka.logs.dir=bin/../logs -cp ${DATA_GEN_DIR}/lib/kafka-clients-0.8.1.jar:${DATA_GEN_JAR}" + +printFullLog + +if [ "$STREAMING_DATAGEN_MODE" == "push" ]; then + CMD="$JAVA_BIN $JVM_OPTS com.intel.hibench.streambench.StartNew $SPARKBENCH_PROPERTIES_FILES $DATA_FILE1 0 $DATA_FILE2 0" + echo -e "${BGreen}Sending streaming data to kafka, concurrently: ${Green}$CMD${Color_Off}" + execute_withlog $CMD +else + CMD="$JAVA_BIN $JVM_OPTS com.intel.hibench.streambench.StartPeriodic $SPARKBENCH_PROPERTIES_FILES $DATA_FILE1 0 $DATA_FILE2 0" + echo -e "${BGreen}Sending streaming data to kafka, periodically: ${Green}$CMD${Color_Off}" + execute_withlog $CMD +fi + +show_bannar finish diff --git a/workloads/streambench/prepare/initTopic.sh b/workloads/streamingbench/prepare/initTopic.sh similarity index 100% rename from workloads/streambench/prepare/initTopic.sh rename to workloads/streamingbench/prepare/initTopic.sh diff --git a/workloads/streambench/prepare/dataGen.sh b/workloads/streamingbench/prepare/zkUtils.sh similarity index 56% rename from workloads/streambench/prepare/dataGen.sh rename to workloads/streamingbench/prepare/zkUtils.sh index 6a651e921..307e806fe 100755 --- a/workloads/streambench/prepare/dataGen.sh +++ b/workloads/streamingbench/prepare/zkUtils.sh @@ -19,19 +19,30 @@ workload_folder=`cd "$workload_folder"; pwd` workload_root=${workload_folder}/.. . "${workload_root}/../../bin/functions/load-bench-config.sh" -enter_bench StreamingBenchPrepare ${workload_root} ${workload_folder} +enter_bench StreamingBenchZkUtils ${workload_root} ${workload_folder} show_bannar start +printFullLog -DATA_GEN_DIR=${workload_root}/../../src/streambench/datagen -DATA_FILE1=${STREAMING_DATA1_DIR}/uservisits -DATA_FILE2=${STREAMING_DATA2_SAMPLE_DIR} +# operation type +op=${1:-ls} -JVM_OPTS="-Xmx1024M -server -XX:+UseCompressedOops -XX:+UseParNewGC -XX:+UseConcMarkSweepGC -XX:+CMSClassUnloadingEnabled -XX:+CMSScavengeBeforeRemark -XX:+DisableExplicitGC -Djava.awt.headless=true -Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.authenticate=false -Dcom.sun.management.jmxremote.ssl=false -Dkafka.logs.dir=bin/../logs -cp ${DATA_GEN_JAR}" +# number of partitions +partitions=${STREAMING_PARTITIONS} -printFullLog +# zkHost address:port +zkHost=$STREAMING_ZKADDR + +# topic +topic=$STREAMING_TOPIC_NAME + +# spark consumer name +consumer=$STREAMING_CONSUMER_GROUP + +path=/consumers/$consumer/offsets/$topic -CMD="$JAVA_BIN $JVM_OPTS com.intel.hibench.streambench.DataGenerator $SPARKBENCH_PROPERTIES_FILES $DATA_FILE1 0 $DATA_FILE2 0" -echo -e "${BGreen}Sending streaming data to kafka, periodically: ${Green}$CMD${Color_Off}" +CMD="$JAVA_BIN -cp $STREAMING_ZKHELPER_JAR com.intel.hibench.streambench.zkHelper.ZKUtil $op $zkHost $path $partitions" +echo -e "${BGreen}Query ZooKeeper for topic offsets, params: ${BCyan}operation:${Cyan}$op ${BCyan}partitions:${Cyan}$partitions ${BCyan}zkHost:${Cyan}$zkHost ${BCyan}topic:${Cyan}$topic ${BCyan}consumer:${Cyan}$consumer ${BCyan}path:${Cyan}$path${Color_Off}" +echo -e "${BGreen}Run:${Green}$CMD${Color_Off}" execute_withlog $CMD show_bannar finish \ No newline at end of file diff --git a/workloads/streambench/samza/bin/run-one-workload.sh b/workloads/streamingbench/samza/bin/run-one-workload.sh similarity index 96% rename from workloads/streambench/samza/bin/run-one-workload.sh rename to workloads/streamingbench/samza/bin/run-one-workload.sh index 1f998a505..5be79a2e0 100755 --- a/workloads/streambench/samza/bin/run-one-workload.sh +++ b/workloads/streamingbench/samza/bin/run-one-workload.sh @@ -23,7 +23,7 @@ workload_root=${workload_folder}/../.. workload_basename=`basename ${workload_folder}` workload_dirname=`dirname ${workload_folder}` workload_name=$1 -enter_bench SamzaStreaming-$workload_name ${workload_root} ${workload_dirname}_$1/${workload_basename} hibench.streambench.benchname=$workload_name +enter_bench SamzaStreaming-$workload_name ${workload_root} ${workload_dirname}_$1/${workload_basename} hibench.streamingbench.benchname=$workload_name show_bannar start SRC_DIR=${workload_root}/../../src/streambench/samzabench diff --git a/workloads/streambench/samza/bin/run.sh b/workloads/streamingbench/samza/bin/run.sh similarity index 96% rename from workloads/streambench/samza/bin/run.sh rename to workloads/streamingbench/samza/bin/run.sh index 8bb2758e7..e9f822665 100755 --- a/workloads/streambench/samza/bin/run.sh +++ b/workloads/streamingbench/samza/bin/run.sh @@ -32,7 +32,7 @@ function samza-submit() { } START_TIME=`timestamp` -. $SRC_DIR/scripts/$STREAMING_TESTCASE.sh +. $SRC_DIR/scripts/$STREAMING_BENCHNAME.sh END_TIME=`timestamp` gen_report ${START_TIME} ${END_TIME} 0 # FIXME, size should be throughput diff --git a/workloads/streambench/spark/bin/run.sh b/workloads/streamingbench/spark/bin/run.sh similarity index 97% rename from workloads/streambench/spark/bin/run.sh rename to workloads/streamingbench/spark/bin/run.sh index b95fc4dfd..172301de3 100755 --- a/workloads/streambench/spark/bin/run.sh +++ b/workloads/streamingbench/spark/bin/run.sh @@ -17,7 +17,6 @@ workload_folder=`dirname "$0"` workload_folder=`cd "$workload_folder"; pwd` workload_root=${workload_folder}/../.. -echo "workload root: " + $workload_root . "${workload_root}/../../bin/functions/load-bench-config.sh" enter_bench SparkStreamingBench ${workload_root} ${workload_folder} diff --git a/workloads/streambench/storm/bin/run.sh b/workloads/streamingbench/storm/bin/run.sh similarity index 97% rename from workloads/streambench/storm/bin/run.sh rename to workloads/streamingbench/storm/bin/run.sh index 67c1177c0..3ebb1a1cb 100755 --- a/workloads/streambench/storm/bin/run.sh +++ b/workloads/streamingbench/storm/bin/run.sh @@ -17,7 +17,7 @@ workload_folder=`dirname "$0"` workload_folder=`cd "$workload_folder"; pwd` workload_root=${workload_folder}/../.. -echo "workload root: " + $workload_root +echo $workload_root . "${workload_root}/../../bin/functions/load-bench-config.sh" enter_bench StormStreamingBench ${workload_root} ${workload_folder} diff --git a/workloads/streambench/storm/bin/stop.sh b/workloads/streamingbench/storm/bin/stop.sh similarity index 82% rename from workloads/streambench/storm/bin/stop.sh rename to workloads/streamingbench/storm/bin/stop.sh index 6d9887a80..f0db7f41d 100755 --- a/workloads/streambench/storm/bin/stop.sh +++ b/workloads/streamingbench/storm/bin/stop.sh @@ -21,11 +21,11 @@ echo $workload_root . "${workload_root}/../../bin/functions/load-bench-config.sh" enter_bench StormStreamingBenchOperator ${workload_root} ${workload_folder} -echo "=========stop storm benchmark $STREAMING_TESTCASE=========" +echo "=========stop storm benchmark $STREAMING_BENCHNAME=========" -${STORM_HOME}/bin/storm kill $STREAMING_TESTCASE +${STORM_BIN_HOME}/storm kill $STREAMING_BENCHNAME echo "Wait 30 seconds... and need double confirm!" sleep 30 echo "Double confirm the bench has been killed:" -${STORM_HOME}/bin/storm kill $STREAMING_TESTCASE -echo 'If got NotAliveException, the "$STREAMING_TESTCASE" has been killed successfully.' +${STORM_BIN_HOME}/storm kill $STREAMING_BENCHNAME +echo 'If got NotAliveException, the "$STREAMING_BENCHNAME" has been killed successfully.' diff --git a/workloads/streambench/trident/bin/run.sh b/workloads/streamingbench/trident/bin/run.sh similarity index 100% rename from workloads/streambench/trident/bin/run.sh rename to workloads/streamingbench/trident/bin/run.sh diff --git a/workloads/streambench/trident/bin/stop.sh b/workloads/streamingbench/trident/bin/stop.sh similarity index 81% rename from workloads/streambench/trident/bin/stop.sh rename to workloads/streamingbench/trident/bin/stop.sh index fb5ec54a4..a1d48eec2 100755 --- a/workloads/streambench/trident/bin/stop.sh +++ b/workloads/streamingbench/trident/bin/stop.sh @@ -21,11 +21,11 @@ echo $workload_root . "${workload_root}/../../bin/functions/load-bench-config.sh" enter_bench StormStreamingBenchOperator ${workload_root} ${workload_folder} -echo "=========stop trident benchmark $STREAMING_TESTCASE=========" +echo "=========stop trident benchmark $STREAMING_BENCHNAME=========" -${STORM_HOME}/bin/storm kill $STREAMING_TESTCASE +${STORM_BIN_HOME}/storm kill $STREAMING_BENCHNAME echo "Wait 30 seconds... and need double confirm!" sleep 30 echo "Double confirm the bench has been killed:" -${STORM_HOME}/bin/storm kill $STREAMING_TESTCASE -echo 'If got NotAliveException, the "$STREAMING_TESTCASE" has been killed successfully.' +${STORM_BIN_HOME}/storm kill $STREAMING_BENCHNAME +echo 'If got NotAliveException, the "$STREAMING_BENCHNAME" has been killed successfully.'