Skip to content

SAMOA-59: add an adapter for Apache Gearpump #54

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Open
wants to merge 1 commit into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
30 changes: 30 additions & 0 deletions bin/samoa
Original file line number Diff line number Diff line change
Expand Up @@ -205,6 +205,36 @@ elif [ $PLATFORM = 'STORM' ]; then
java -cp $CLASSPATH org.apache.samoa.LocalStormDoTask $COMPLETE_ARG $NUM_WORKER
fi

elif [ $PLATFORM = 'GEARPUMP' ]; then

echo "Deploying to $PLATFORM"
if [ -z $GEARPUMP_HOME ];then
echo "GEARPUMP_HOME is not set!"
echo "Please set GEARPUMP_HOME to point to your Gearpump installation"
exit -1
fi

if [ ! -f $2 ];then
echo "$2 does not exist!"
echo "Please use a valid jar file for Gearpump execution"
exit -1
fi

GEARPUMP_EXEC="sh $GEARPUMP_HOME/bin/gear"

COMPLETE_ARG=""
COUNTER=0
for var in "$@"
do
COUNTER=`expr $COUNTER + 1`
if [ $COUNTER -gt 2 ];then
COMPLETE_ARG="$COMPLETE_ARG $var"
fi
done

DEPLOYABLE=$JAR_PATH
$GEARPUMP_EXEC app -jar $DEPLOYABLE org.apache.samoa.topology.impl.gearpump.DoTask $COMPLETE_ARG

elif [ $PLATFORM = 'SAMZA' ]; then
echo "Deploying to SAMZA"

Expand Down
14 changes: 14 additions & 0 deletions pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -93,6 +93,15 @@
<module>samoa-test</module>
</modules>
</profile>
<profile>
<id>gearpump</id>
<modules>
<module>samoa-instances</module>
<module>samoa-api</module>
<module>samoa-gearpump</module>
<module>samoa-test</module>
</modules>
</profile>
<profile>
<id>all</id>
<modules>
Expand All @@ -103,6 +112,7 @@
<module>samoa-storm</module>
<module>samoa-flink</module>
<module>samoa-samza</module>
<module>samoa-gearpump</module>
<module>samoa-test</module>
</modules>
</profile>
Expand All @@ -127,6 +137,9 @@
<miniball.version>1.0.3</miniball.version>
<samza.version>0.7.0</samza.version>
<flink.version>0.10.1</flink.version>
<gearpump.version>0.8.1</gearpump.version>
<scala.binary.version>2.11</scala.binary.version>
<scala.version>2.11.5</scala.version>
<slf4j-log4j12.version>1.7.2</slf4j-log4j12.version>
<slf4j-simple.version>1.7.5</slf4j-simple.version>
<maven-surefire-plugin.version>2.18</maven-surefire-plugin.version>
Expand Down Expand Up @@ -210,6 +223,7 @@
<root>samoa-storm</root>
<root>samoa-flink</root>
<root>samoa-samza</root>
<root>samoa-gearpump</root>
<root>samoa-test</root>
<root>samoa-threads</root>
<root>bin</root>
Expand Down
23 changes: 23 additions & 0 deletions samoa-gearpump/README.md
Original file line number Diff line number Diff line change
@@ -0,0 +1,23 @@
# Executing Apache SAMOA with Apache Gearpump

In this tutorial README we describe how to execute Apache SAMOA on top of [Apache Gearpump(incubating)](http://gearpump.apache.org/).

## Build

Simply clone the repository and install SAMOA.

```
git clone http://git.apache.org/incubator-samoa.git
cd incubator-samoa
mvn -Pgearpump package
```

The deployable jar for SAMOA will be in `target/SAMOA-gearpump-0.4.0-incubating-SNAPSHOT.jar`.

## Executing SAMOA with Gearpump step-by-step

1. Ensure that you already have Gearpump running. You can follow this [tutorial](http://gearpump.apache.org/releases/latest/deployment-local.html) to deploy Gearpump in local mode.
2. Set `GEARPUMP_HOME` to point to your Gearpump installation path.
3. In the SAMOA path, you can input command to execute SAMOA tasks. For example, `bin/samoa gearpump target/SAMOA-gearpump-0.4.0-incubating-SNAPSHOT.jar "PrequentialEvaluation -d /tmp/dump.csv -i 1000000 -f 100000 -l (classifiers.trees.VerticalHoeffdingTree -p 4) -s (generators.RandomTreeGenerator -c 2 -o 10 -u 10)"`


163 changes: 163 additions & 0 deletions samoa-gearpump/pom.xml
Original file line number Diff line number Diff line change
@@ -0,0 +1,163 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--
#%L
SAMOA
%%
Copyright (C) 2014 - 2015 Apache Software Foundation
%%
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
#L%
-->

<project xmlns="http://maven.apache.org/POM/4.0.0" 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">
<modelVersion>4.0.0</modelVersion>
<properties>
<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
</properties>

<name>samoa-gearpump</name>
<description>gearpump bindings for SAMOA</description>

<artifactId>samoa-gearpump</artifactId>
<parent>
<groupId>org.apache.samoa</groupId>
<artifactId>samoa</artifactId>
<version>0.4.0-incubating-SNAPSHOT</version>
</parent>

<repositories>
<repository>
<id>apache.snapshots</id>
<name>Apache Development Snapshot Repository</name>
<url>https://repository.apache.org/content/repositories/snapshots/</url>
<releases>
<enabled>false</enabled>
</releases>
<snapshots>
<enabled>true</enabled>
</snapshots>
</repository>
<repository>
<id>gearpump-shaded-repo</id>
<name>Vincent at Bintray</name>
<url>http://dl.bintray.com/fvunicorn/maven</url>
</repository>
</repositories>

<dependencies>
<dependency>
<groupId>org.apache.samoa</groupId>
<artifactId>samoa-api</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.samoa</groupId>
<artifactId>samoa-test</artifactId>
<type>test-jar</type>
<classifier>test-jar-with-dependencies</classifier>
<version>${project.version}</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.gearpump</groupId>
<artifactId>gearpump-streaming_${scala.binary.version}</artifactId>
<version>${gearpump.version}</version>
</dependency>
<dependency>
<groupId>org.apache.gearpump</groupId>
<artifactId>gearpump-daemon_${scala.binary.version}</artifactId>
<version>${gearpump.version}</version>
<exclusions>
<exclusion>
<groupId>org.apache.gearpump</groupId>
<artifactId>gearpump-experimental-cgroup_${scala.binary.version}</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-log4j12</artifactId>
<version>${slf4j-log4j12.version}</version>
<scope>test</scope>
</dependency>
</dependencies>

<build>
<plugins>
<plugin>
<groupId>net.alchim31.maven</groupId>
<artifactId>scala-maven-plugin</artifactId>
<version>3.2.2</version>
<executions>
<execution>
<id>scala-compile-first</id>
<phase>process-resources</phase>
<goals>
<goal>add-source</goal>
<goal>compile</goal>
</goals>
</execution>
<execution>
<id>scala-test-compile-first</id>
<phase>process-test-resources</phase>
<goals>
<goal>testCompile</goal>
</goals>
</execution>
</executions>
</plugin>
<!-- SAMOA assembly -->
<plugin>
<artifactId>maven-assembly-plugin</artifactId>
<version>${maven-assembly-plugin.version}</version>
<configuration>
<finalName>SAMOA-gearpump-${project.version}</finalName>
<appendAssemblyId>false</appendAssemblyId>
<attach>false</attach>
<outputDirectory>../target</outputDirectory>
<descriptorRefs>
<descriptorRef>jar-with-dependencies</descriptorRef>
</descriptorRefs>
<archive>
<manifestEntries>
<Bundle-Version>${parsedVersion.osgiVersion}</Bundle-Version>
<Bundle-Description>${project.description}</Bundle-Description>
<Implementation-Version>${project.version}</Implementation-Version>
<Implementation-Vendor>Yahoo Labs</Implementation-Vendor>
<Implementation-Vendor-Id>SAMOA</Implementation-Vendor-Id>
</manifestEntries>
</archive>
</configuration>
<executions>
<execution>
<id>make-assembly</id> <!-- this is used for inheritance merges -->
<phase>package</phase>
<goals>
<goal>single</goal>
</goals>
</execution>
</executions>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-surefire-plugin</artifactId>
<version>${maven-surefire-plugin.version}</version>
<configuration>
<argLine>-Xmx1G</argLine>
<redirectTestOutputToFile>false</redirectTestOutputToFile>
</configuration>
</plugin>
</plugins>
</build>
</project>
Original file line number Diff line number Diff line change
@@ -0,0 +1,58 @@
package org.apache.samoa.topology.impl.gearpump;

/*
* #%L
* SAMOA
* %%
* Copyright (C) 2014 - 2015 Apache Software Foundation
* %%
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
* #L%
*/

import org.apache.samoa.core.EntranceProcessor;
import org.apache.samoa.core.Processor;
import org.apache.samoa.topology.EntranceProcessingItem;
import org.apache.samoa.topology.IProcessingItem;
import org.apache.samoa.topology.ProcessingItem;
import org.apache.samoa.topology.Stream;
import org.apache.samoa.topology.Topology;

public class ComponentFactory implements org.apache.samoa.topology.ComponentFactory {
@Override
public ProcessingItem createPi(Processor processor) {
return createPi(processor, 1);
}

@Override
public ProcessingItem createPi(Processor processor, int parallelism) {
return new GearpumpProcessingItem(processor, parallelism);
}

@Override
public EntranceProcessingItem createEntrancePi(EntranceProcessor
entranceProcessor) {
return new GearpumpEntranceProcessingItem(entranceProcessor);
}

@Override
public Stream createStream(IProcessingItem sourcePi) {
TopologyNode topologyNode = (TopologyNode) sourcePi;
return topologyNode.createStream();
}

@Override
public Topology createTopology(String topoName) {
return new GearpumpTopology(topoName);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,42 @@
package org.apache.samoa.topology.impl.gearpump;

/*
* #%L
* SAMOA
* %%
* Copyright (C) 2014 - 2015 Apache Software Foundation
* %%
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
* #L%
*/

import org.apache.gearpump.cluster.UserConfig;
import org.apache.gearpump.cluster.client.ClientContext;
import org.apache.gearpump.streaming.StreamApplication;
import org.apache.gearpump.util.Graph;

public class DoTask {

public static void main(String[] args) {

GearpumpTopology topology = Utils.argsToTopology(args);
String topologyName = topology.getTopologyName();
Graph graph = topology.getGraph();
StreamApplication app = StreamApplication.apply(topologyName, graph, UserConfig.empty());
ClientContext context = ClientContext.apply();
context.submit(app);
context.close();

}

}
Loading