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
+
+
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
-
-
-
-
-
-
-
-
+ 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
+
+
+
+
+
+
+
+
+
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
+
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