Skip to content

Commit b27b901

Browse files
committed
add gearpump benchmark
1 parent a140e1c commit b27b901

File tree

5 files changed

+351
-0
lines changed

5 files changed

+351
-0
lines changed

gearpump-benchmarks/pom.xml

Lines changed: 116 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,116 @@
1+
<?xml version="1.0" encoding="UTF-8"?>
2+
<!--
3+
Copyright 2015, Yahoo Inc.
4+
Licensed under the terms of the Apache License 2.0. Please see LICENSE file in the project root for terms.
5+
-->
6+
<project xmlns="http://maven.apache.org/POM/4.0.0"
7+
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
8+
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
9+
<parent>
10+
<artifactId>yahoo-low-latency-bechmarks</artifactId>
11+
<groupId>com.yahoo.stream</groupId>
12+
<version>0.1.0</version>
13+
</parent>
14+
<modelVersion>4.0.0</modelVersion>
15+
<artifactId>gearpump-benchmarks</artifactId>
16+
17+
<dependencies>
18+
<dependency>
19+
<groupId>com.github.intel-hadoop</groupId>
20+
<artifactId>gearpump-streaming_${scala.binary.version}</artifactId>
21+
<scope>provided</scope>
22+
</dependency>
23+
<dependency>
24+
<groupId>com.github.intel-hadoop</groupId>
25+
<artifactId>gearpump-core_${scala.binary.version}</artifactId>
26+
<scope>provided</scope>
27+
</dependency>
28+
<dependency>
29+
<groupId>com.github.intel-hadoop</groupId>
30+
<artifactId>gearpump-external-kafka_${scala.binary.version}</artifactId>
31+
</dependency>
32+
<dependency>
33+
<groupId>org.json</groupId>
34+
<artifactId>json</artifactId>
35+
</dependency>
36+
<dependency>
37+
<groupId>com.yahoo.stream</groupId>
38+
<artifactId>streaming-benchmark-common</artifactId>
39+
</dependency>
40+
</dependencies>
41+
42+
<build>
43+
<plugins>
44+
<plugin>
45+
<groupId>net.alchim31.maven</groupId>
46+
<artifactId>scala-maven-plugin</artifactId>
47+
<version>3.2.2</version>
48+
<executions>
49+
<execution>
50+
<id>eclipse-add-source</id>
51+
<goals>
52+
<goal>add-source</goal>
53+
</goals>
54+
</execution>
55+
<execution>
56+
<id>scala-compile-first</id>
57+
<phase>process-resources</phase>
58+
<goals>
59+
<goal>compile</goal>
60+
</goals>
61+
</execution>
62+
<execution>
63+
<id>scala-test-compile-first</id>
64+
<phase>process-test-resources</phase>
65+
<goals>
66+
<goal>testCompile</goal>
67+
</goals>
68+
</execution>
69+
</executions>
70+
<configuration>
71+
<scalaVersion>${scala.version}</scalaVersion>
72+
<recompileMode>incremental</recompileMode>
73+
<useZincServer>true</useZincServer>
74+
<args>
75+
<arg>-unchecked</arg>
76+
<arg>-deprecation</arg>
77+
<arg>-feature</arg>
78+
</args>
79+
<jvmArgs>
80+
<jvmArg>-Xms1024m</jvmArg>
81+
<jvmArg>-Xmx1024m</jvmArg>
82+
</jvmArgs>
83+
<javacArgs>
84+
<javacArg>-source</javacArg>
85+
<javacArg>${java.version}</javacArg>
86+
<javacArg>-target</javacArg>
87+
<javacArg>${java.version}</javacArg>
88+
<javacArg>-Xlint:all,-serial,-path</javacArg>
89+
</javacArgs>
90+
</configuration>
91+
</plugin>
92+
<plugin>
93+
<groupId>org.apache.maven.plugins</groupId>
94+
<artifactId>maven-shade-plugin</artifactId>
95+
<configuration>
96+
<createDependencyReducedPom>true</createDependencyReducedPom>
97+
<minimizeJar>false</minimizeJar>
98+
<transformers>
99+
<transformer
100+
implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer"/>
101+
<transformer
102+
implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer"/>
103+
</transformers>
104+
</configuration>
105+
<executions>
106+
<execution>
107+
<phase>package</phase>
108+
<goals>
109+
<goal>shade</goal>
110+
</goals>
111+
</execution>
112+
</executions>
113+
</plugin>
114+
</plugins>
115+
</build>
116+
</project>
Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,5 @@
1+
gearpump {
2+
serializers {
3+
"scala.Tuple7" = ""
4+
}
5+
}
Lines changed: 138 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,138 @@
1+
package gearpump.benchmark
2+
3+
import akka.actor.ActorSystem
4+
import benchmark.common.Utils
5+
import benchmark.common.advertising.{CampaignProcessorCommon, RedisAdCampaignCache}
6+
import io.gearpump.Message
7+
import io.gearpump.partitioner.{Partitioner, UnicastPartitioner, HashPartitioner}
8+
import io.gearpump.cluster.UserConfig
9+
import io.gearpump.cluster.client.ClientContext
10+
import io.gearpump.streaming.kafka.lib.StringMessageDecoder
11+
import io.gearpump.streaming.{StreamApplication, Processor}
12+
import io.gearpump.streaming.kafka.{KafkaSource, KafkaStorageFactory}
13+
import io.gearpump.streaming.source.{DefaultTimeStampFilter, DataSourceProcessor}
14+
import io.gearpump.streaming.task.{StartTime, Task, TaskContext}
15+
import io.gearpump.util.{AkkaApp, Graph}
16+
import org.json.JSONObject
17+
import io.gearpump.util.Graph.Node
18+
import scala.collection.JavaConverters._
19+
20+
object Advertising extends AkkaApp{
21+
22+
def application(args: Array[String], system: ActorSystem) : StreamApplication = {
23+
implicit val actorSystem = system
24+
val commonConfig = Utils.findAndReadConfigFile(args(0), true).asInstanceOf[java.util.Map[String, Any]]
25+
26+
val cores = commonConfig.get("process.cores").asInstanceOf[Int]
27+
val topic = commonConfig.get("kafka.topic").asInstanceOf[String]
28+
val partitions = commonConfig.get("kafka.partitions").asInstanceOf[Int]
29+
val redisHost = commonConfig.get("redis.host").asInstanceOf[String]
30+
31+
val zookeeperHosts = commonConfig.get("zookeeper.servers").asInstanceOf[java.util.List[String]] match {
32+
case l: java.util.List[String] => l.asScala.toSeq
33+
case other => throw new ClassCastException(other + " not a List[String]")
34+
}
35+
val zookeeperPort = commonConfig.get("zookeeper.port").asInstanceOf[Int]
36+
val zookeeperConnect = zookeeperHosts.map(_ + ":" + zookeeperPort).mkString(",")
37+
38+
val kafkaHosts = commonConfig.get("kafka.brokers").asInstanceOf[java.util.List[String]] match {
39+
case l: java.util.List[String] => l.asScala.toSeq
40+
case other => throw new ClassCastException(other + " not a List[String]")
41+
}
42+
val kafkaPort = commonConfig.get("kafka.port").asInstanceOf[Int]
43+
val brokerList = kafkaHosts.map(_ + ":" + kafkaPort).mkString(",")
44+
45+
val parallel = Math.max(1, cores / 7)
46+
val gearConfig = UserConfig.empty.withString("redis.host", redisHost)
47+
val offsetStorageFactory = new KafkaStorageFactory(zookeeperConnect, brokerList)
48+
val source = new KafkaSource(topic, zookeeperConnect, offsetStorageFactory, new StringMessageDecoder, new DefaultTimeStampFilter)
49+
val sourceProcessor = DataSourceProcessor(source, partitions)
50+
val deserializer = Processor[DeserializeTask](parallel)
51+
val filter = Processor[EventFilterTask](parallel)
52+
val projection = Processor[EventProjectionTask](parallel)
53+
val join = Processor[RedisJoinTask](parallel)
54+
val campaign = Processor[CampaignProcessorTask](parallel * 2)
55+
val partitioner = new AdPartitioner
56+
57+
val graph = Graph(sourceProcessor ~ new HashPartitioner ~> deserializer ~> filter ~> projection ~> join ~ partitioner ~> campaign)
58+
StreamApplication("Advertising", graph, gearConfig)
59+
}
60+
61+
override def main(akkaConf: Advertising.Config, args: Array[String]): Unit = {
62+
val context = ClientContext(akkaConf)
63+
context.submit(application(args, context.system))
64+
context.close()
65+
}
66+
67+
override def help: Unit = {}
68+
}
69+
70+
class AdPartitioner extends UnicastPartitioner {
71+
override def getPartition(msg: Message, partitionNum: Int, currentPartitionId: Int): Int = {
72+
(msg.msg.asInstanceOf[(String, String, String)]._1.hashCode & Integer.MAX_VALUE) % partitionNum
73+
}
74+
}
75+
76+
class DeserializeTask(taskContext : TaskContext, conf: UserConfig) extends Task(taskContext, conf) {
77+
override def onNext(msg : Message) : Unit = {
78+
val jsonObj = new JSONObject(msg.msg.asInstanceOf[String])
79+
val tuple = (
80+
jsonObj.getString("user_id"),
81+
jsonObj.getString("page_id"),
82+
jsonObj.getString("ad_id"),
83+
jsonObj.getString("ad_type"),
84+
jsonObj.getString("event_type"),
85+
jsonObj.getString("event_time"),
86+
jsonObj.getString("ip_address")
87+
)
88+
taskContext.output(Message(tuple, msg.timestamp))
89+
}
90+
}
91+
92+
class EventFilterTask(taskContext : TaskContext, conf: UserConfig) extends Task(taskContext, conf) {
93+
override def onNext(msg: Message): Unit = {
94+
val tuple = msg.msg.asInstanceOf[(String, String, String, String, String, String, String)]
95+
if(tuple._5 == "view") {
96+
taskContext.output(msg)
97+
}
98+
}
99+
}
100+
101+
class EventProjectionTask(taskContext : TaskContext, conf: UserConfig) extends Task(taskContext, conf) {
102+
override def onNext(msg: Message): Unit = {
103+
val tuple = msg.msg.asInstanceOf[(String, String, String, String, String, String, String)]
104+
taskContext.output(Message((tuple._3, tuple._6), msg.timestamp))
105+
}
106+
}
107+
108+
class RedisJoinTask(taskContext : TaskContext, conf: UserConfig) extends Task(taskContext, conf) {
109+
private val redisHost = conf.getString("redis.host").get
110+
private val redisAdCampaignCache = new RedisAdCampaignCache(redisHost)
111+
112+
override def onStart(startTime : StartTime) : Unit = {
113+
redisAdCampaignCache.prepare()
114+
}
115+
116+
override def onNext(msg: Message): Unit = {
117+
val (ad_id, event_time) = msg.msg.asInstanceOf[(String, String)]
118+
val campaign_id = redisAdCampaignCache.execute(ad_id)
119+
if(campaign_id != null) {
120+
val result = (campaign_id, ad_id, event_time)
121+
taskContext.output(Message(result, msg.timestamp))
122+
}
123+
}
124+
}
125+
126+
class CampaignProcessorTask(taskContext : TaskContext, conf: UserConfig) extends Task(taskContext, conf) {
127+
private val redisHost = conf.getString("redis.host").get
128+
private val campaignProcessorCommon = new CampaignProcessorCommon(redisHost)
129+
130+
override def onStart(startTime : StartTime) : Unit = {
131+
campaignProcessorCommon.prepare()
132+
}
133+
134+
override def onNext(msg: Message): Unit = {
135+
val (campaign_id, _, event_time) = msg.msg.asInstanceOf[(String, String, String)]
136+
campaignProcessorCommon.execute(campaign_id, event_time)
137+
}
138+
}

pom.xml

Lines changed: 31 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -12,13 +12,28 @@ Licensed under the terms of the Apache License 2.0. Please see LICENSE file in t
1212
<name>yahoo-low-latency-bechmarks</name>
1313
<version>0.1.0</version>
1414

15+
<repositories>
16+
<repository>
17+
<id>releases-oss.sonatype.org</id>
18+
<name>Sonatype Releases Repository</name>
19+
<url>http://oss.sonatype.org/content/repositories/releases/</url>
20+
</repository>
21+
22+
<repository>
23+
<id>gearpump-shaded-repo</id>
24+
<name>Vincent at Bintray</name>
25+
<url>http://dl.bintray.com/fvunicorn/maven</url>
26+
</repository>
27+
</repositories>
28+
1529
<properties>
1630
<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
1731
<project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
1832
<spark.version>1.5.1</spark.version>
1933
<kafka.version>0.8.2.1</kafka.version>
2034
<flink.version>0.10.1</flink.version>
2135
<storm.version>0.10.0</storm.version>
36+
<gearpump.version>0.7.5</gearpump.version>
2237
<scala.binary.version>2.10</scala.binary.version>
2338
<scala.version>2.10.4</scala.version>
2439
<json.version>20141113</json.version>
@@ -65,6 +80,21 @@ Licensed under the terms of the Apache License 2.0. Please see LICENSE file in t
6580
<artifactId>flink-connector-kafka</artifactId>
6681
<version>${flink.version}</version>
6782
</dependency>
83+
<dependency>
84+
<groupId>com.github.intel-hadoop</groupId>
85+
<artifactId>gearpump-core_${scala.binary.version}</artifactId>
86+
<version>${gearpump.version}</version>
87+
</dependency>
88+
<dependency>
89+
<groupId>com.github.intel-hadoop</groupId>
90+
<artifactId>gearpump-streaming_${scala.binary.version}</artifactId>
91+
<version>${gearpump.version}</version>
92+
</dependency>
93+
<dependency>
94+
<groupId>com.github.intel-hadoop</groupId>
95+
<artifactId>gearpump-external-kafka_${scala.binary.version}</artifactId>
96+
<version>${gearpump.version}</version>
97+
</dependency>
6898
<dependency>
6999
<groupId>com.yahoo.stream</groupId>
70100
<artifactId>streaming-benchmark-common</artifactId>
@@ -128,5 +158,6 @@ Licensed under the terms of the Apache License 2.0. Please see LICENSE file in t
128158
<module>storm-benchmarks</module>
129159
<module>flink-benchmarks</module>
130160
<module>spark-benchmarks</module>
161+
<module>gearpump-benchmarks</module>
131162
</modules>
132163
</project>

0 commit comments

Comments
 (0)