You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@samza.apache.org by xi...@apache.org on 2019/03/12 20:42:54 UTC

[samza] branch 1.1.0 updated: SAMZA-2124: Add Beam API doc to the website (#948)

This is an automated email from the ASF dual-hosted git repository.

xinyu pushed a commit to branch 1.1.0
in repository https://gitbox.apache.org/repos/asf/samza.git


The following commit(s) were added to refs/heads/1.1.0 by this push:
     new 3470591  SAMZA-2124: Add Beam API doc to the website (#948)
3470591 is described below

commit 34705916215fd4478e2ed4bec6e4a2c092897990
Author: xinyuiscool <xi...@gmail.com>
AuthorDate: Tue Mar 12 13:10:01 2019 -0700

    SAMZA-2124: Add Beam API doc to the website (#948)
    
    * SAMZA-2124: Add Beam API doc to the website
    
    * Address pr feedback
---
 docs/learn/documentation/versioned/api/beam-api.md | 111 +++++++++++++++
 .../versioned/core-concepts/core-concepts.md       |   3 +-
 docs/learn/documentation/versioned/index.html      |   2 +-
 docs/startup/code-examples/versioned/beam.md       | 152 +++++++++++++++++++++
 docs/startup/code-examples/versioned/index.md      | 105 +-------------
 .../code-examples/versioned/{index.md => samza.md} |  68 +--------
 docs/startup/quick-start/versioned/beam.md         | 104 ++++++++++++++
 docs/startup/quick-start/versioned/index.md        |   2 +
 8 files changed, 377 insertions(+), 170 deletions(-)

diff --git a/docs/learn/documentation/versioned/api/beam-api.md b/docs/learn/documentation/versioned/api/beam-api.md
new file mode 100644
index 0000000..f65a9b9
--- /dev/null
+++ b/docs/learn/documentation/versioned/api/beam-api.md
@@ -0,0 +1,111 @@
+---
+layout: page
+title: Apache Beam API
+---
+<!--
+   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.
+-->
+
+### Table Of Contents
+- [Introduction](#introduction)
+- [Basic Concepts](#basic-concepts)
+- [Apache Beam - A Samza’s Perspective](#apache-beam---a-samza’s-perspective)
+
+### Introduction
+
+Apache Beam brings an easy-to-usen but powerful API and model for state-of-art stream and batch data processing with portability across a variety of languages. The Beam API and model has the following characteristics:
+
+- *Simple constructs, powerful semantics*: the whole beam API can be simply described by a `Pipeline` object, which captures all your data processing steps from input to output. Beam SDK supports over [20 data IOs](https://beam.apache.org/documentation/io/built-in/), and data transformations from simple [Map](https://beam.apache.org/releases/javadoc/2.11.0/org/apache/beam/sdk/transforms/MapElements.html) to complex [Combines and Joins](https://beam.apache.org/releases/javadoc/2.11.0/inde [...]
+
+- *Strong consistency via event-time*: Beam provides advanced [event-time support](https://beam.apache.org/documentation/programming-guide/#watermarks-and-late-data) so you can perform windowing and aggregations based on when the events happen, instead of when they arrive. The event-time mechanism improves the accuracy of processing results, and guarantees repeatability in results when reprocessing the same data set.
+
+- *Comprehensive stream processing semantics*: Beam supports an up-to-date stream processing model, including [tumbling/sliding/session windows](https://beam.apache.org/documentation/programming-guide/#windowing), joins and aggregations. It provides [triggers](https://beam.apache.org/documentation/programming-guide/#triggers) based on conditions of early and late firings, and late arrival handling with accumulation mode and allowed lateness.
+
+- *Portability with multiple programming languages*: Beam supports a consistent API in multiple languages, including [Java, Python and Go](https://beam.apache.org/roadmap/portability/). This allows you to leverage the rich ecosystem built for different languages, e.g. ML libs for Python.
+
+### Basic Concepts
+
+Let's walk through the WordCount example to illustrate the Beam basic concepts. A Beam program often starts by creating a [Pipeline](https://beam.apache.org/releases/javadoc/2.11.0/org/apache/beam/sdk/Pipeline.html) object in your `main()` function.
+
+{% highlight java %}
+
+// Start by defining the options for the pipeline.
+PipelineOptions options = PipelineOptionsFactory.create();
+
+// Then create the pipeline.
+Pipeline p = Pipeline.create(options);
+
+{% endhighlight %}
+
+The `PipelineOptions` supported by SamzaRunner is documented in detail [here](https://beam.apache.org/documentation/runners/samza/).
+
+Let's apply the first data transform to read from a text file using [TextIO.read()](https://beam.apache.org/releases/javadoc/2.11.0/org/apache/beam/sdk/io/TextIO.html):
+
+{% highlight java %}
+
+PCollection<String> lines = p.apply(
+  "ReadLines", TextIO.read().from("/path/to/inputData"));
+
+{% endhighlight %}
+
+To break down each line into words, you can use a [FlatMap](https://beam.apache.org/releases/javadoc/2.11.0/org/apache/beam/sdk/transforms/FlatMapElements.html):
+
+{% highlight java %}
+
+PCollection<String> words = lines.apply(
+    FlatMapElements.into(TypeDescriptors.strings())
+        .via((String word) -> Arrays.asList(word.split("\\W+"))));
+
+{% endhighlight %}
+
+Beam provides a build-in transform [Count.perElement](https://beam.apache.org/releases/javadoc/2.11.0/org/apache/beam/sdk/transforms/Count.html) to count the number of elements based on each value. Let's use it here to count the words:
+
+{% highlight java %}
+
+PCollection<KV<String, Long>> counts = pipeline.apply(Count.perElement());
+
+{% endhighlight %}
+
+Finally we format the counts into strings and write to a file using `TextIO.write()`:
+
+{% highlight java %}
+
+counts.apply(ToString.kvs())
+      .apply(TextIO.write().to("/path/to/output").withoutSharding());
+
+{% endhighlight %}
+
+To run your pipeline and wait for the results, just do:
+
+{% highlight java %}
+
+pipeline.run().waitUntilFinish();
+
+{% endhighlight %}
+
+Or you can run your pipeline asynchronously, e.g. when you submit it to a remote cluster:
+
+{% highlight java %}
+
+pipeline.run();
+
+{% endhighlight %}
+
+To run this Beam program with Samza, you can simply provide "--runner=SamzaRunner" as a program argument. You can follow our [quick start](/startup/quick-start/{{site.version}}/beam.html) to set up your project and run different examples. For more details on writing the Beam program, please refer the [Beam programming guide](https://beam.apache.org/documentation/programming-guide/).
+
+### Apache Beam - A Samza’s Perspective
+
+The goal of Samza is to provide large-scale streaming processing capabilities with first-class state support. This does not contradict with Beam. In fact, while Samza lays out a solid foundation for large-scale stateful stream processing, Beam adds the cutting-edge stream processing API and model on top of it. The Beam API and model allows further optimization in the Samza platform, including multi-stage distributed computation and parallel processing on the per-key basis. The performanc [...]
\ No newline at end of file
diff --git a/docs/learn/documentation/versioned/core-concepts/core-concepts.md b/docs/learn/documentation/versioned/core-concepts/core-concepts.md
index ffe39a5..265c072 100644
--- a/docs/learn/documentation/versioned/core-concepts/core-concepts.md
+++ b/docs/learn/documentation/versioned/core-concepts/core-concepts.md
@@ -62,10 +62,11 @@ A _stream application_ processes messages from input streams, transforms them an
 
 ![diagram-medium](/img/{{site.version}}/learn/documentation/core-concepts/stream-application.png)
 
-Samza offers three top-level APIs to help you build your stream applications: <br/>
+Samza offers foure top-level APIs to help you build your stream applications: <br/>
 1. The [High Level Streams API](/learn/documentation/{{site.version}}/api/high-level-api.html),  which offers several built-in operators like map, filter, etc. This is the recommended API for most use-cases. <br/>
 2. The [Low Level Task API](/learn/documentation/{{site.version}}/api/low-level-api.html), which allows greater flexibility to define your processing-logic and offers greater control <br/>
 3. [Samza SQL](/learn/documentation/{{site.version}}/api/samza-sql.html), which offers a declarative SQL interface to create your applications <br/>
+4. [Apache Beam API](/learn/documentation/{{site.version}}/api/beam-api.html), which offers the full Java API from [Apache beam](https://beam.apache.org/) while Python and Go are work-in-progress.
 
 ### State
 Samza supports both stateless and stateful stream processing. _Stateless processing_, as the name implies, does not retain any state associated with the current message after it has been processed. A good example of this is filtering an incoming stream of user-records by a field (eg:userId) and writing the filtered messages to their own stream. 
diff --git a/docs/learn/documentation/versioned/index.html b/docs/learn/documentation/versioned/index.html
index b565b42..3f76b9e 100644
--- a/docs/learn/documentation/versioned/index.html
+++ b/docs/learn/documentation/versioned/index.html
@@ -32,7 +32,7 @@ title: Documentation
   <li><a href="api/table-api.html">Table API</a></li>
   <li><a href="api/test-framework.html">Testing Samza</a></li>
   <li><a href="api/samza-sql.html">Samza SQL</a></li>
-  <li><a href="https://beam.apache.org/documentation/runners/samza/">Apache BEAM</a></li>
+  <li><a href="api/beam-api.html">Apache BEAM</a></li>
 </ul>
 
 <h4>DEPLOYMENT</h4>
diff --git a/docs/startup/code-examples/versioned/beam.md b/docs/startup/code-examples/versioned/beam.md
new file mode 100644
index 0000000..10c1ac0
--- /dev/null
+++ b/docs/startup/code-examples/versioned/beam.md
@@ -0,0 +1,152 @@
+---
+layout: page
+title: Beam Code Examples
+---
+<!--
+   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.
+-->
+
+The [samza-beam-examples](https://github.com/apache/samza-beam-examples) project contains examples to demonstrate running Beam pipelines with SamzaRunner locally, in Yarn cluster, or in standalone cluster with Zookeeper. More complex pipelines can be built from this project and run in similar manner.  
+
+### Example Pipelines
+The following examples are included:
+
+1. [`WordCount`](https://github.com/apache/samza-beam-examples/blob/master/src/main/java/org/apache/beam/examples/WordCount.java) reads a file as input (bounded data source), and computes word frequencies. 
+
+2. [`KafkaWordCount`](https://github.com/apache/samza-beam-examples/blob/master/src/main/java/org/apache/beam/examples/KafkaWordCount.java) does the same word-count computation but reading from a Kafka stream (unbounded data source). It uses a fixed 10-sec window to aggregate the counts.
+
+### Run the Examples
+
+Each example can be run locally, in Yarn cluster or in standalone cluster. Here we use KafkaWordCount as an example.
+
+#### Set Up
+1. Download and install [JDK version 8](https://www.oracle.com/technetwork/java/javase/downloads/jdk8-downloads-2133151.html). Verify that the JAVA_HOME environment variable is set and points to your JDK installation.
+
+2. Download and install [Apache Maven](http://maven.apache.org/download.cgi) by following Maven’s [installation guide](http://maven.apache.org/install.html) for your specific operating system.
+
+Check out the `samza-beam-examples` repo:
+
+```
+$ git clone https://github.com/apache/samza-beam-examples.git
+$ cd samza-beam-examples
+```
+
+A script named "grid" is included in this project which allows you to easily download and install Zookeeper, Kafka, and Yarn.
+You can run the following to bring them all up running in your local machine:
+
+```
+$ scripts/grid bootstrap
+```
+
+All the downloaded package files will be put under `deploy` folder. Once the grid command completes, 
+you can verify that Yarn is up and running by going to http://localhost:8088. You can also choose to
+bring them up separately, e.g.:
+
+```
+$ scripts/grid install zookeeper
+$ scripts/grid start zookeeper
+```
+Now let's create a Kafka topic named "input-text" for this example:
+
+```
+$ ./deploy/kafka/bin/kafka-topics.sh  --zookeeper localhost:2181 --create --topic input-text --partitions 10 --replication-factor 1
+```
+   
+#### Run Locally
+You can run directly within the project using maven:
+
+```
+$ mvn compile exec:java -Dexec.mainClass=org.apache.beam.examples.KafkaWordCount \
+    -Dexec.args="--runner=SamzaRunner" -P samza-runner
+```
+
+#### Packaging Your Application
+To execute the example in either Yarn or standalone, you need to package it first.
+After packaging, we deploy and explode the tgz in the deploy folder:
+
+```
+ $ mkdir -p deploy/examples
+ $ mvn package && tar -xvf target/samza-beam-examples-0.1-dist.tar.gz -C deploy/examples/
+```
+
+#### Run in Standalone Cluster with Zookeeper
+You can use the `run-beam-standalone.sh` script included in this repo to run an example
+in standalone mode. The config file is provided as `config/standalone.properties`. Note by
+default we create one single split for the whole input (--maxSourceParallelism=1). To 
+set each Kafka partition in a split, we can set a large "maxSourceParallelism" value which 
+is the upper bound of the number of splits.
+
+```
+$ deploy/examples/bin/run-beam-standalone.sh org.apache.beam.examples.KafkaWordCount \
+    --configFilePath=$PWD/deploy/examples/config/standalone.properties --maxSourceParallelism=1024
+```
+
+#### Run Yarn Cluster
+Similar to running standalone, we can use the `run-beam-yarn.sh` to run the examples
+in Yarn cluster. The config file is provided as `config/yarn.properties`. To run the 
+KafkaWordCount example in yarn:
+
+```
+$ deploy/examples/bin/run-beam-yarn.sh org.apache.beam.examples.KafkaWordCount \
+    --configFilePath=$PWD/deploy/examples/config/yarn.properties --maxSourceParallelism=1024
+```
+
+#### Validate the Pipeline Results
+Now the pipeline is deployed to either locally, standalone or Yarn. Let's check out the results. First we start a kakfa consumer to listen to the output:
+
+```
+$ ./deploy/kafka/bin/kafka-console-consumer.sh --bootstrap-server localhost:9092 --topic word-count --property print.key=true
+```
+
+Then let's publish a few lines to the input Kafka topic:
+
+```
+$ ./deploy/kafka/bin/kafka-console-producer.sh --topic input-text --broker-list localhost:9092
+Nory was a Catholic because her mother was a Catholic, and Nory’s mother was a Catholic because her father was a Catholic, and her father was a Catholic because his mother was a Catholic, or had been.
+```
+
+You should see the word count shows up in the consumer console in about 10 secs:
+
+```
+a       6
+br      1
+mother  3
+was     6
+Catholic        6
+his     1
+Nory    2
+s       1
+father  2
+had     1
+been    1
+and     2
+her     3
+or      1
+because 3
+```
+
+### Beyond Examples
+Feel free to build more complex pipelines based on the examples above, and reach out to us:
+
+* Subscribe and mail to [user@beam.apache.org](mailto:user@beam.apache.org) for any Beam questions.
+
+* Subscribe and mail to [user@samza.apache.org](mailto:user@samza.apache.org) for any Samza questions.
+
+### More Information
+
+* [Apache Beam](http://beam.apache.org)
+* [Apache Samza](https://samza.apache.org/)
+* Quickstart: [Java](https://beam.apache.org/get-started/quickstart-java), [Python](https://beam.apache.org/get-started/quickstart-py), [Go](https://beam.apache.org/get-started/quickstart-go)
\ No newline at end of file
diff --git a/docs/startup/code-examples/versioned/index.md b/docs/startup/code-examples/versioned/index.md
index 80e2ed7..21a7f84 100644
--- a/docs/startup/code-examples/versioned/index.md
+++ b/docs/startup/code-examples/versioned/index.md
@@ -1,6 +1,6 @@
 ---
 layout: page
-title:
+title: Code Examples
 ---
 <!--
    Licensed to the Apache Software Foundation (ASF) under one or more
@@ -19,105 +19,6 @@ title:
    limitations under the License.
 -->
 
+[Samza Code Examples](samza.html)
 
-### Checking out our examples
-
-The [hello-samza](https://github.com/apache/samza-hello-samza) project contains several examples to help you create your Samza applications. To checkout the hello-samza project:
-
-{% highlight bash %}
-> git clone https://git.apache.org/samza-hello-samza.git hello-samza
-{% endhighlight %}
-
-#### High-level API examples
-[The Samza Cookbook](https://github.com/apache/samza-hello-samza/tree/master/src/main/java/samza/examples/cookbook) contains various recipes using the Samza high-level API.
-These include:
-
-- The [Filter example](https://github.com/apache/samza-hello-samza/blob/latest/src/main/java/samza/examples/cookbook/FilterExample.java) demonstrates how to perform stateless operations on a stream. 
-
-- The [Join example](https://github.com/apache/samza-hello-samza/blob/latest/src/main/java/samza/examples/cookbook/JoinExample.java) demonstrates how you can join a Kafka stream of page-views with a stream of ad-clicks
-
-- The [Stream-Table Join example](https://github.com/apache/samza-hello-samza/blob/latest/src/main/java/samza/examples/cookbook/RemoteTableJoinExample.java) demonstrates how to use the Samza Table API. It joins a Kafka stream with a remote dataset accessed through a REST service.
-
-- The [SessionWindow](https://github.com/apache/samza-hello-samza/blob/latest/src/main/java/samza/examples/cookbook/SessionWindowExample.java) and [TumblingWindow](https://github.com/apache/samza-hello-samza/blob/latest/src/main/java/samza/examples/cookbook/TumblingWindowExample.java) examples illustrate Samza's rich windowing and triggering capabilities.
-
-
-In addition to the cookbook, you can also consult these:
-
-- [Wikipedia Parser](https://github.com/apache/samza-hello-samza/tree/master/src/main/java/samza/examples/wikipedia): An advanced example that builds a streaming pipeline consuming a live-feed of wikipedia edits, parsing each message and generating statistics from them.
-
-
-- [Amazon Kinesis](https://github.com/apache/samza-hello-samza/tree/master/src/main/java/samza/examples/kinesis) and [Azure Eventhubs](https://github.com/apache/samza-hello-samza/tree/latest/src/main/java/samza/examples/azure) examples that cover how to consume input data from the respective systems.
-
-#### Low-level API examples
-The [Wikipedia Parser (low-level API)](https://github.com/apache/samza-hello-samza/tree/latest/src/main/java/samza/examples/wikipedia/task/application): 
-Same example that builds a streaming pipeline consuming a live-feed of 
-wikipedia edits, parsing each message and generating statistics from them, but
-using low-level APIs. 
-
-#### Samza SQL API examples
-You can easily create a Samza job declaratively using 
-[Samza SQL](https://samza.apache.org/learn/tutorials/0.14/samza-sql.html).
-
-#### Apache Beam API examples
-
-The easiest way to get a copy of the WordCount examples in Beam API is to use [Apache Maven](http://maven.apache.org/download.cgi). After installing Maven, please run the following command:
-
-{% highlight bash %}
-> mvn archetype:generate \
-      -DarchetypeGroupId=org.apache.beam \
-      -DarchetypeArtifactId=beam-sdks-java-maven-archetypes-examples \
-      -DarchetypeVersion=2.9.0 \
-      -DgroupId=org.example \
-      -DartifactId=word-count-beam \
-      -Dversion="0.1" \
-      -Dpackage=org.apache.beam.examples \
-      -DinteractiveMode=false
-{% endhighlight %}
-
-This command creates a maven project `word-count-beam` which contains a series of example pipelines that count words in text files:
-
-{% highlight bash %}
-> cd word-count-beam/
-
-> ls src/main/java/org/apache/beam/examples/
-DebuggingWordCount.java	WindowedWordCount.java	common
-MinimalWordCount.java	WordCount.java
-{% endhighlight %}
-
-To use SamzaRunner, please add the following `samza-runner` profile to `pom.xml` under the "profiles" section, same as in [here](https://github.com/apache/beam/blob/master/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml).
-
-{% highlight xml %}
-    ...
-    <profile>
-      <id>samza-runner</id>
-      <dependencies>
-        <dependency>
-          <groupId>org.apache.beam</groupId>
-          <artifactId>beam-runners-samza</artifactId>
-          <version>${beam.version}</version>
-          <scope>runtime</scope>
-        </dependency>
-      </dependencies>
-    </profile>
-    ....
-{% endhighlight %}
-
-Now we can run the wordcount example with Samza using the following command:
-
-{% highlight bash %}
->mvn compile exec:java -Dexec.mainClass=org.apache.beam.examples.WordCount \
-     -Dexec.args="--inputFile=pom.xml --output=/tmp/counts --runner=SamzaRunner" -Psamza-runner
-{% endhighlight %}
-
-After the pipeline finishes, you can check out the output counts files in /tmp folder. Note Beam generates multiple output files for parallel processing. If you prefer a single output, please update the code to use TextIO.write().withoutSharding().
-
-{% highlight bash %}
->more /tmp/counts*
-AS: 1
-IO: 2
-IS: 1
-OF: 1
-...
-{% endhighlight %}
-
-A walkthrough of the example code can be found [here](https://beam.apache.org/get-started/wordcount-example/). Feel free to play with other examples in the project or write your own. Please don't hesitate to [reach out](https://samza.apache.org/community/contact-us.html) if you encounter any issues.
+[Beam Code Examples](beam.html)
\ No newline at end of file
diff --git a/docs/startup/code-examples/versioned/index.md b/docs/startup/code-examples/versioned/samza.md
similarity index 57%
copy from docs/startup/code-examples/versioned/index.md
copy to docs/startup/code-examples/versioned/samza.md
index 80e2ed7..710099a 100644
--- a/docs/startup/code-examples/versioned/index.md
+++ b/docs/startup/code-examples/versioned/samza.md
@@ -1,6 +1,6 @@
 ---
 layout: page
-title:
+title: Samza Code Examples
 ---
 <!--
    Licensed to the Apache Software Foundation (ASF) under one or more
@@ -56,68 +56,4 @@ using low-level APIs.
 
 #### Samza SQL API examples
 You can easily create a Samza job declaratively using 
-[Samza SQL](https://samza.apache.org/learn/tutorials/0.14/samza-sql.html).
-
-#### Apache Beam API examples
-
-The easiest way to get a copy of the WordCount examples in Beam API is to use [Apache Maven](http://maven.apache.org/download.cgi). After installing Maven, please run the following command:
-
-{% highlight bash %}
-> mvn archetype:generate \
-      -DarchetypeGroupId=org.apache.beam \
-      -DarchetypeArtifactId=beam-sdks-java-maven-archetypes-examples \
-      -DarchetypeVersion=2.9.0 \
-      -DgroupId=org.example \
-      -DartifactId=word-count-beam \
-      -Dversion="0.1" \
-      -Dpackage=org.apache.beam.examples \
-      -DinteractiveMode=false
-{% endhighlight %}
-
-This command creates a maven project `word-count-beam` which contains a series of example pipelines that count words in text files:
-
-{% highlight bash %}
-> cd word-count-beam/
-
-> ls src/main/java/org/apache/beam/examples/
-DebuggingWordCount.java	WindowedWordCount.java	common
-MinimalWordCount.java	WordCount.java
-{% endhighlight %}
-
-To use SamzaRunner, please add the following `samza-runner` profile to `pom.xml` under the "profiles" section, same as in [here](https://github.com/apache/beam/blob/master/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/pom.xml).
-
-{% highlight xml %}
-    ...
-    <profile>
-      <id>samza-runner</id>
-      <dependencies>
-        <dependency>
-          <groupId>org.apache.beam</groupId>
-          <artifactId>beam-runners-samza</artifactId>
-          <version>${beam.version}</version>
-          <scope>runtime</scope>
-        </dependency>
-      </dependencies>
-    </profile>
-    ....
-{% endhighlight %}
-
-Now we can run the wordcount example with Samza using the following command:
-
-{% highlight bash %}
->mvn compile exec:java -Dexec.mainClass=org.apache.beam.examples.WordCount \
-     -Dexec.args="--inputFile=pom.xml --output=/tmp/counts --runner=SamzaRunner" -Psamza-runner
-{% endhighlight %}
-
-After the pipeline finishes, you can check out the output counts files in /tmp folder. Note Beam generates multiple output files for parallel processing. If you prefer a single output, please update the code to use TextIO.write().withoutSharding().
-
-{% highlight bash %}
->more /tmp/counts*
-AS: 1
-IO: 2
-IS: 1
-OF: 1
-...
-{% endhighlight %}
-
-A walkthrough of the example code can be found [here](https://beam.apache.org/get-started/wordcount-example/). Feel free to play with other examples in the project or write your own. Please don't hesitate to [reach out](https://samza.apache.org/community/contact-us.html) if you encounter any issues.
+[Samza SQL](https://samza.apache.org/learn/tutorials/latest/samza-sql.html).
diff --git a/docs/startup/quick-start/versioned/beam.md b/docs/startup/quick-start/versioned/beam.md
new file mode 100644
index 0000000..2b53141
--- /dev/null
+++ b/docs/startup/quick-start/versioned/beam.md
@@ -0,0 +1,104 @@
+---
+layout: page
+title: Beam on Samza Quick Start
+---
+<!--
+   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.
+-->
+
+[Apache Beam](https://beam.apache.org/) is an open-source SDK which provides state-of-the-art data processing API and model for both batch and streaming processing pipelines across multiple languages, i.e. Java, Python and Go. By collaborating with Beam, Samza offers the capability of executing Beam API on Samza's large-scale and stateful streaming engine. Current Samza supports the full Beam [Java API](https://beam.apache.org/documentation/runners/capability-matrix/), and the support of [...]
+
+### Setting up the Word-Count Project
+
+To get started, you need to install [Java 8 SDK]() as well as [Apache Maven](http://maven.apache.org/download.cgi). After that, the easiest way to get a copy of the WordCount examples in Beam API is to use the following command to generate a simple Maven project:
+
+{% highlight bash %}
+> mvn archetype:generate \
+      -DarchetypeGroupId=org.apache.beam \
+      -DarchetypeArtifactId=beam-sdks-java-maven-archetypes-examples \
+      -DarchetypeVersion=2.11.0 \
+      -DgroupId=org.example \
+      -DartifactId=word-count-beam \
+      -Dversion="0.1" \
+      -Dpackage=org.apache.beam.examples \
+      -DinteractiveMode=false
+{% endhighlight %}
+
+This command creates a maven project `word-count-beam` which contains a series of example pipelines that count words in text files:
+
+{% highlight bash %}
+> cd word-count-beam/
+
+> ls src/main/java/org/apache/beam/examples/
+DebuggingWordCount.java	WindowedWordCount.java	common
+MinimalWordCount.java	WordCount.java
+{% endhighlight %}
+
+Let's use the MinimalWordCount example to demonstrate how to create a simple Beam pipeline:
+
+{% highlight java %}
+public class MinimalWordCount {
+
+  public static void main(String[] args) {
+    // Create the Pipeline object with the options we defined above
+    PipelineOptions options = PipelineOptionsFactory.create();
+    Pipeline p = Pipeline.create(options);
+
+    // This example reads a public data set consisting of the complete works of Shakespeare.
+    p.apply(TextIO.read().from("gs://apache-beam-samples/shakespeare/*"))
+        .apply(
+            FlatMapElements.into(TypeDescriptors.strings())
+                .via((String word) -> Arrays.asList(word.split("[^\\p{L}]+"))))
+        .apply(Filter.by((String word) -> !word.isEmpty()))
+        .apply(Count.perElement())
+        .apply(
+            MapElements.into(TypeDescriptors.strings())
+                .via(
+                    (KV<String, Long> wordCount) ->
+                        wordCount.getKey() + ": " + wordCount.getValue()))
+        .apply(TextIO.write().to("wordcounts"));
+
+    p.run().waitUntilFinish();
+  }
+}
+{% endhighlight %}
+
+In this example, we first create a Beam `Pipeline` object to build the graph of transformations to be executed. Then we first use the Read transform to consume a public data set, and split into words. Then we use Beam build-in `Count` transform and returns the key/value pairs where each key represents a unique element from the input collection, and each value represents the number of times that key appeared in the input collection. Finally we format the results and write them to a file.  [...]
+
+Let's run the WordCount example with Samza using the following command:
+
+{% highlight bash %}
+>mvn compile exec:java -Dexec.mainClass=org.apache.beam.examples.WordCount \
+     -Dexec.args="--inputFile=pom.xml --output=/tmp/counts --runner=SamzaRunner" -Psamza-runner
+{% endhighlight %}
+
+After the pipeline finishes, you can check out the output counts files in /tmp folder. Note Beam generates multiple output files for parallel processing. If you prefer a single output, please update the code to use TextIO.write().withoutSharding().
+
+{% highlight bash %}
+>more /tmp/counts*
+This: 1
+When: 1
+YARN: 1
+apex: 2
+apis: 2
+beam: 43
+beta: 1
+code: 2
+copy: 1
+...
+{% endhighlight %}
+
+For more examples and how to deploy your job in local, standalone and Yarn cluster, you can look at the [code examples](/startup/code-examples/{{site.version}}/beam.html). Please don't hesitate to [reach out](https://samza.apache.org/community/contact-us.html) if you encounter any issues.
\ No newline at end of file
diff --git a/docs/startup/quick-start/versioned/index.md b/docs/startup/quick-start/versioned/index.md
index 3cc0c1f..0554c27 100644
--- a/docs/startup/quick-start/versioned/index.md
+++ b/docs/startup/quick-start/versioned/index.md
@@ -22,3 +22,5 @@ title: Quick Start
 [Samza Tutorial](samza.html)
 
 [Samza SQL Tutorial](samza-sql.html)
+
+[Beam on Samza Tutorial](beam.html)