Skip to content
Closed
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
22 changes: 22 additions & 0 deletions runners/gearpump/README.md
Original file line number Diff line number Diff line change
@@ -0,0 +1,22 @@
<!--
Licensed to the Apache Software Foundation (ASF) under one
or more contributor license agreements. See the NOTICE file
distributed with this work for additional information
regarding copyright ownership. The ASF licenses this file
to you under the Apache License, Version 2.0 (the
"License"); you may not use this file except in compliance
with the License. You may obtain a copy of the License at

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

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

## Gearpump Beam Runner

The Gearpump Beam runner allows users to execute pipelines written using the Apache Beam programming API with Apache Gearpump (incubating) as an execution engine.
296 changes: 296 additions & 0 deletions runners/gearpump/pom.xml
Original file line number Diff line number Diff line change
@@ -0,0 +1,296 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--
Licensed to the Apache Software Foundation (ASF) under one
or more contributor license agreements. See the NOTICE file
distributed with this work for additional information
regarding copyright ownership. The ASF licenses this file
to you under the Apache License, Version 2.0 (the
"License"); you may not use this file except in compliance
with the License. You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing,
software distributed under the License is distributed on an
"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
KIND, either express or implied. See the License for the
specific language governing permissions and limitations
under the License.
-->
<project xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">

<modelVersion>4.0.0</modelVersion>

<parent>
<groupId>org.apache.beam</groupId>
<artifactId>beam-runners-parent</artifactId>
<version>0.2.0-incubating-SNAPSHOT</version>
<relativePath>../pom.xml</relativePath>
</parent>

<artifactId>beam-runners-gearpump</artifactId>

<name>Apache Beam :: Runners :: Gearpump</name>
<packaging>jar</packaging>

<properties>
<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
<project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
<gearpump.version>0.8.1-SNAPSHOT</gearpump.version>
</properties>

<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.gearpump</groupId>
<artifactId>gearpump-streaming_2.11</artifactId>
<version>${gearpump.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.gearpump</groupId>
<artifactId>gearpump-core_2.11</artifactId>
<version>${gearpump.version}</version>
<scope>provided</scope>
<exclusions>
<exclusion>
<groupId>com.google.code.findbugs</groupId>
<artifactId>jsr305</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>org.apache.gearpump</groupId>
<artifactId>gearpump-daemon_2.11</artifactId>
<version>${gearpump.version}</version>
<scope>provided</scope>
<exclusions>
<exclusion>
<groupId>org.apache.gearpump</groupId>
<artifactId>gearpump-experimental-cgroup_2.11</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>com.typesafe</groupId>
<artifactId>config</artifactId>
<scope>provided</scope>
<version>1.3.0</version>
</dependency>
<dependency>
<groupId>org.scala-lang</groupId>
<artifactId>scala-library</artifactId>
<version>2.11.8</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.beam</groupId>
<artifactId>beam-sdks-java-core</artifactId>
<exclusions>
<exclusion>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-jdk14</artifactId>
</exclusion>
<exclusion>
<groupId>com.google.collections</groupId>
<artifactId>google-collections</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>org.apache.beam</groupId>
<artifactId>beam-runners-core-java</artifactId>
</dependency>
<dependency>
<groupId>com.google.code.findbugs</groupId>
<artifactId>annotations</artifactId>
<version>3.0.1</version>
</dependency>
<dependency>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-api</artifactId>
</dependency>
<dependency>
<groupId>joda-time</groupId>
<artifactId>joda-time</artifactId>
</dependency>
<dependency>
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-core</artifactId>
</dependency>
<dependency>
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-databind</artifactId>
</dependency>
<dependency>
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-annotations</artifactId>
</dependency>
<dependency>
<groupId>com.google.http-client</groupId>
<artifactId>google-http-client</artifactId>
</dependency>
<dependency>
<groupId>com.google.guava</groupId>
<artifactId>guava</artifactId>
</dependency>
<dependency>
<groupId>junit</groupId>
<artifactId>junit</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.hamcrest</groupId>
<artifactId>hamcrest-all</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.beam</groupId>
<artifactId>beam-sdks-java-core</artifactId>
<type>test-jar</type>
<scope>test</scope>
<exclusions>
<exclusion>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-jdk14</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>org.mockito</groupId>
<artifactId>mockito-all</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>com.google.auto.service</groupId>
<artifactId>auto-service</artifactId>
<version>1.0-rc2</version>
</dependency>
</dependencies>

<build>
<plugins>
<!-- JAR Packaging -->
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-jar-plugin</artifactId>
<configuration>
<archive>
<manifest>
<addDefaultImplementationEntries>true</addDefaultImplementationEntries>
<addDefaultSpecificationEntries>true</addDefaultSpecificationEntries>
</manifest>
</archive>
</configuration>
</plugin>

<!-- Java compiler -->
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-compiler-plugin</artifactId>
</plugin>

<!-- Integration Tests -->
<plugin>
<artifactId>maven-failsafe-plugin</artifactId>
<executions>
<execution>
<goals>
<goal>integration-test</goal>
<goal>verify</goal>
</goals>
</execution>
</executions>
<configuration>
<forkCount>1</forkCount>
<argLine>-Dlog4j.configuration=log4j-test.properties -XX:-UseGCOverheadLimit</argLine>
</configuration>
</plugin>

<!-- Unit Tests -->
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-surefire-plugin</artifactId>
<version>2.19.1</version>
<executions>
<execution>
<id>runnable-on-service-tests</id>
<configuration>
<groups>org.apache.beam.sdk.testing.RunnableOnService</groups>
<parallel>none</parallel>
<failIfNoTests>true</failIfNoTests>
<dependenciesToScan>
<dependency>org.apache.beam:beam-sdks-java-core</dependency>
<dependency>org.apache.beam:beam-runners-java-core</dependency>
</dependenciesToScan>
<excludes>
<!-- side input is not supported in Gearpump -->
<exclude>
org.apache.beam.sdk.io.BigQueryIOTest,
org.apache.beam.sdk.io.CountingInputTest,
org.apache.beam.sdk.io.CountingSourceTest,
org.apache.beam.sdk.testing.PAssertTest,
org.apache.beam.sdk.transforms.ApproximateUniqueTest,
org.apache.beam.sdk.transforms.CombineTest,
org.apache.beam.sdk.transforms.CombineFnsTest,
org.apache.beam.sdk.transforms.CountTest,
org.apache.beam.sdk.transforms.FlattenTest,
org.apache.beam.sdk.transforms.ParDoTest,
org.apache.beam.sdk.transforms.SampleTest,
org.apache.beam.sdk.transforms.ViewTest,
org.apache.beam.sdk.transforms.join.CoGroupByKeyTest
</exclude>
<!-- merging windows is not supported in Gearpump -->
<exclude>
org.apache.beam.sdk.transforms.windowing.WindowingTest,
org.apache.beam.sdk.util.ReshuffleTest
</exclude>
</excludes>
<systemPropertyVariables>
<beamTestPipelineOptions>
[
"--runner=TestGearpumpRunner",
"--streaming=true"
]
</beamTestPipelineOptions>
</systemPropertyVariables>
</configuration>
</execution>
</executions>
</plugin>

<!-- uber jar -->
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-assembly-plugin</artifactId>
<configuration>
<descriptorRefs>
<descriptorRef>jar-with-dependencies</descriptorRef>
</descriptorRefs>
</configuration>
</plugin>

<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-checkstyle-plugin</artifactId>
</plugin>
</plugins>
</build>
</project>
Original file line number Diff line number Diff line change
@@ -0,0 +1,67 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.beam.runners.gearpump;

import org.apache.beam.sdk.options.Default;
import org.apache.beam.sdk.options.Description;
import org.apache.beam.sdk.options.PipelineOptions;

import com.fasterxml.jackson.annotation.JsonIgnore;

import org.apache.gearpump.cluster.client.ClientContext;
import org.apache.gearpump.cluster.embedded.EmbeddedCluster;

import java.util.Map;

/**
* Options that configure the Gearpump pipeline.
*/
public interface GearpumpPipelineOptions extends PipelineOptions {

@Description("set unique application name for Gearpump runner")
void setApplicationName(String name);

String getApplicationName();

@Description("set parallelism for Gearpump processor")
void setParallelism(int parallelism);

@Default.Integer(1)
int getParallelism();

@Description("register Kryo serializers")
void setSerializers(Map<String, String> serializers);

@JsonIgnore
Map<String, String> getSerializers();

@Description("set EmbeddedCluster for tests")
void setEmbeddedCluster(EmbeddedCluster cluster);

@JsonIgnore
EmbeddedCluster getEmbeddedCluster();

void setClientContext(ClientContext clientContext);

@JsonIgnore
@Description("get client context to query application status")
ClientContext getClientContext();

}

Loading