You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by da...@apache.org on 2017/01/31 01:37:21 UTC

[1/5] beam-site git commit: WIP: Add Python to WordCount documentation

Repository: beam-site
Updated Branches:
  refs/heads/asf-site 2f4d86d31 -> b81afa390


WIP: Add Python to WordCount documentation


Project: http://git-wip-us.apache.org/repos/asf/beam-site/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam-site/commit/fb8666ef
Tree: http://git-wip-us.apache.org/repos/asf/beam-site/tree/fb8666ef
Diff: http://git-wip-us.apache.org/repos/asf/beam-site/diff/fb8666ef

Branch: refs/heads/asf-site
Commit: fb8666ef0b6c262a3705329fcd026d32c1a931a7
Parents: 2f4d86d
Author: Hadar Hod <ha...@google.com>
Authored: Tue Jan 24 19:37:16 2017 -0800
Committer: Davor Bonaci <da...@google.com>
Committed: Mon Jan 30 17:35:50 2017 -0800

----------------------------------------------------------------------
 src/get-started/wordcount-example.md | 222 +++++++++++++++++++++++++-----
 1 file changed, 184 insertions(+), 38 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam-site/blob/fb8666ef/src/get-started/wordcount-example.md
----------------------------------------------------------------------
diff --git a/src/get-started/wordcount-example.md b/src/get-started/wordcount-example.md
index b8776d6..a02b327 100644
--- a/src/get-started/wordcount-example.md
+++ b/src/get-started/wordcount-example.md
@@ -10,6 +10,14 @@ redirect_from: /use/wordcount-example/
 * TOC
 {:toc}
 
+<nav class="language-switcher">
+  <strong>Adapt for:</strong> 
+  <ul>
+    <li data-type="language-java">Java SDK</li>
+    <li data-type="language-py">Python SDK</li>
+  </ul>
+</nav>
+
 The WordCount examples demonstrate how to set up a processing pipeline that can read text, tokenize the text lines into individual words, and perform a frequency count on each of those words. The Beam SDKs contain a series of these four successively more detailed WordCount examples that build on each other. The input text for all the examples is a set of Shakespeare's texts.
 
 Each WordCount example introduces different concepts in the Beam programming model. Begin by understanding Minimal WordCount, the simplest of the examples. Once you feel comfortable with the basic principles in building a pipeline, continue on to learn more concepts in the other examples.
@@ -60,12 +68,26 @@ You can specify a runner for executing your pipeline, such as the `DataflowRunne
     //   options.setRunner(FlinkRunner.class);
 ```
 
+```py
+options = PipelineOptions()
+google_cloud_options = options.view_as(GoogleCloudOptions)
+google_cloud_options.project = 'my-project-id'
+google_cloud_options.job_name = 'myjob'
+google_cloud_options.staging_location = 'gs://your-bucket-name-here/staging'
+google_cloud_options.temp_location = 'gs://your-bucket-name-here/temp'
+options.view_as(StandardOptions).runner = 'BlockingDataflowPipelineRunner'
+```
+
 The next step is to create a Pipeline object with the options we've just constructed. The Pipeline object builds up the graph of transformations to be executed, associated with that particular pipeline.
 
 ```java
 Pipeline p = Pipeline.create(options);
 ```
 
+```py
+p = beam.Pipeline(options=options)
+```
+
 ### Applying Pipeline Transforms
 
 The Minimal WordCount pipeline contains several transforms to read data into the pipeline, manipulate or otherwise transform the data, and write out the results. Each transform represents an operation in the pipeline.
@@ -79,51 +101,72 @@ The Minimal WordCount pipeline contains five transforms:
 
 1.  A text file `Read` transform is applied to the Pipeline object itself, and produces a `PCollection` as output. Each element in the output PCollection represents one line of text from the input file. This example happens to use input data stored in a publicly accessible Google Cloud Storage bucket ("gs://").
 
-	```java
-	        p.apply(TextIO.Read.from("gs://apache-beam-samples/shakespeare/*"))
-	```
+    ```java
+    p.apply(TextIO.Read.from("gs://apache-beam-samples/shakespeare/*"))
+    ```
+
+    ```py
+    p | beam.io.Read(beam.io.TextFileSource('gs://dataflow-samples/shakespeare/kinglear.txt'))
+    ```
 
 2.  A [ParDo]({{ site.baseurl }}/documentation/programming-guide/#transforms-pardo) transform that invokes a `DoFn` (defined in-line as an anonymous class) on each element that tokenizes the text lines into individual words. The input for this transform is the `PCollection` of text lines generated by the previous `TextIO.Read` transform. The `ParDo` transform outputs a new `PCollection`, where each element represents an individual word in the text.
 
-	```java
-	        .apply("ExtractWords", ParDo.of(new DoFn<String, String>() {
-	            @ProcessElement
-	            public void processElement(ProcessContext c) {
-	                for (String word : c.element().split("[^a-zA-Z']+")) {
-	                    if (!word.isEmpty()) {
-	                        c.output(word);
-	                    }
-	                }
-	            }
-	        }))
-	```
+    ```java
+    .apply("ExtractWords", ParDo.of(new DoFn<String, String>() {
+        @ProcessElement
+        public void processElement(ProcessContext c) {
+            for (String word : c.element().split("[^a-zA-Z']+")) {
+                if (!word.isEmpty()) {
+                    c.output(word);
+                }
+            }
+        }
+    }))
+    ```
+
+    ```py
+    # The Flatmap transform is a simplified version of ParDo.
+    | 'ExtractWords' >> beam.FlatMap(lambda x: re.findall(r'[A-Za-z\']+', x))
+    ```
 
 3.  The SDK-provided `Count` transform is a generic transform that takes a `PCollection` of any type, and returns a `PCollection` of key/value pairs. 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.
 
 	In this pipeline, the input for `Count` is the `PCollection` of individual words generated by the previous `ParDo`, and the output is a `PCollection` of key/value pairs where each key represents a unique word in the text and the associated value is the occurrence count for each.
 
-	```java
-	        .apply(Count.<String>perElement())
-	```
+    ```java
+    .apply(Count.<String>perElement())
+    ```
+
+    ```py
+    | beam.combiners.Count.PerElement()
+    ```
 
 4.  The next transform formats each of the key/value pairs of unique words and occurrence counts into a printable string suitable for writing to an output file.
 
 	`MapElements` is a higher-level composite transform that encapsulates a simple `ParDo`; for each element in the input `PCollection`, `MapElements` applies a function that produces exactly one output element. In this example, `MapElements` invokes a `SimpleFunction` (defined in-line as an anonymous class) that does the formatting. As input, `MapElements` takes a `PCollection` of key/value pairs generated by `Count`, and produces a new `PCollection` of printable strings.
 
-	```java
-	        .apply("FormatResults", MapElements.via(new SimpleFunction<KV<String, Long>, String>() {
-	            @Override
-	            public String apply(KV<String, Long> input) {
-	                return input.getKey() + ": " + input.getValue();
-	            }
-	        }))
-	```
+    ```java
+    .apply("FormatResults", MapElements.via(new SimpleFunction<KV<String, Long>, String>() {
+        @Override
+        public String apply(KV<String, Long> input) {
+            return input.getKey() + ": " + input.getValue();
+        }
+    }))
+    ```
+
+    ```py
+    | beam.Map(lambda (word, count): '%s: %s' % (word, count))
+    ```
 
 5.  A text file `Write`. This transform takes the final `PCollection` of formatted Strings as input and writes each element to an output text file. Each element in the input `PCollection` represents one line of text in the resulting output file.
 
-	```java
-	        .apply(TextIO.Write.to("wordcounts"));
-	```
+    ```java
+    .apply(TextIO.Write.to("wordcounts"));
+    ```
+
+    ```py
+    | beam.io.Write(beam.io.TextFileSink('gs://my-bucket/counts.txt'))
+    ```
 
 Note that the `Write` transform produces a trivial result value of type `PDone`, which in this case is ignored.
 
@@ -135,6 +178,10 @@ Run the pipeline by calling the `run` method, which sends your pipeline to be ex
 p.run().waitUntilFinish();
 ```
 
+```py
+p.run()
+```
+
 Note that the `run` method is asynchronous. For a blocking execution instead, run your pipeline appending the `waitUntilFinish` method.
 
 ## WordCount Example
@@ -157,9 +204,9 @@ The following sections explain these key concepts in detail, and break down the
 
 When using `ParDo` transforms, you need to specify the processing operation that gets applied to each element in the input `PCollection`. This processing operation is a subclass of the SDK class `DoFn`. You can create the `DoFn` subclasses for each `ParDo` inline, as an anonymous inner class instance, as is done in the previous example (Minimal WordCount). However, it's often a good idea to define the `DoFn` at the global level, which makes it easier to unit test and can make the `ParDo` code more readable.
 
-In this example, `ExtractWordsFn` is a `DoFn` that is defined as a static class:
-
 ```java
+// In this example, ExtractWordsFn is a DoFn that is defined as a static class:
+
 static class ExtractWordsFn extends DoFn<String, String> {
     ...
 
@@ -170,10 +217,16 @@ static class ExtractWordsFn extends DoFn<String, String> {
 }
 ```
 
-This `DoFn` (`ExtractWordsFn`) is the processing operation that `ParDo` applies to the `PCollection` of words:
+```py
+# In this example, the DoFns are defined as classes:
 
-```java
-PCollection<String> words = lines.apply(ParDo.of(new ExtractWordsFn()));
+class FormatAsTextFn(beam.DoFn):
+
+  def process(self, context):
+    word, count = context.element
+    yield '%s: %s' % (word, count)
+
+formatted = counts | beam.ParDo(FormatAsTextFn())
 ```
 
 ### Creating Composite Transforms
@@ -211,13 +264,28 @@ public static void main(String[] args) throws IOException {
 }
 ```
 
+```py
+class CountWords(beam.PTransform):
+
+  def apply(self, pcoll):
+    return (pcoll
+            # Convert lines of text into individual words.
+            | beam.FlatMap(
+                'ExtractWords', lambda x: re.findall(r'[A-Za-z\']+', x))
+
+            # Count the number of times each word occurs.
+            | beam.combiners.Count.PerElement())
+
+counts = lines | CountWords()
+```
+
 ### Using Parameterizable PipelineOptions
 
 You can hard-code various execution options when you run your pipeline. However, the more common way is to define your own configuration options via command-line argument parsing. Defining your configuration options via the command-line makes the code more easily portable across different runners.
 
 Add arguments to be processed by the command-line parser, and specify default values for them. You can then access the options values in your pipeline code. 
 
-``` java
+```java
 public static interface WordCountOptions extends PipelineOptions {
   @Description("Path of the file to read from")
   @Default.String("gs://dataflow-samples/shakespeare/kinglear.txt")
@@ -234,6 +302,19 @@ public static void main(String[] args) {
 }
 ```
 
+```py
+class WordCountOptions(PipelineOptions):
+
+  @classmethod
+  def _add_argparse_args(cls, parser):
+    parser.add_argument('--input',
+                        help='Input for the dataflow pipeline',
+                        default='gs://my-bucket/input')
+
+options = PipelineOptions(argv)
+p = beam.Pipeline(options=options)
+```
+
 ## Debugging WordCount Example
 
 The Debugging WordCount example demonstrates some best practices for instrumenting your pipeline code.
@@ -259,9 +340,9 @@ If you execute your pipeline using `DirectRunner`, it will print the log message
 
 If you execute your pipeline using `DataflowRunner`, you can use Google Cloud Logging. Google Cloud Logging (currently in beta) aggregates the logs from all of your Dataflow job's workers to a single location in the Google Cloud Platform Console. You can use Cloud Logging to search and access the logs from all of the Compute Engine instances that Dataflow has spun up to complete your Dataflow job. You can add logging statements into your pipeline's `DoFn` instances that will appear in Cloud Logging as your pipeline runs.
 
-In this example, we use `.trace` and `.debug`:
-
 ```java
+// This example uses .trace and .debug:
+
 public class DebuggingWordCount {
 
   public static class FilterTextFn extends DoFn<KV<String, Long>, KV<String, Long>> {
@@ -280,6 +361,43 @@ public class DebuggingWordCount {
 }
 ```
 
+```py
+import logging
+
+class FilterTextFn(beam.DoFn):
+  """A DoFn that filters for a specific key based on a regular expression."""
+
+  # A custom aggregator can track values in your pipeline as it runs. Create
+  # custom aggregators matched_word and unmatched_words.
+  matched_words = beam.Aggregator('matched_words')
+  umatched_words = beam.Aggregator('umatched_words')
+
+  def __init__(self, pattern):
+    self.pattern = pattern
+
+  def process(self, context):
+    word, _ = context.element
+    if re.match(self.pattern, word):
+      # Log at INFO level each element we match. When executing this pipeline
+      # using the Dataflow service, these log lines will appear in the Cloud
+      # Logging UI.
+      logging.info('Matched %s', word)
+
+      # Add 1 to the custom aggregator matched_words
+      context.aggregate_to(self.matched_words, 1)
+      yield context.element
+    else:
+      # Log at the "DEBUG" level each element that is not matched. Different
+      # log levels can be used to control the verbosity of logging providing
+      # an effective mechanism to filter less important information. Note
+      # currently only "INFO" and higher level logs are emitted to the Cloud
+      # Logger. This log message will not be visible in the Cloud Logger.
+      logging.debug('Did not match %s', word)
+
+      # Add 1 to the custom aggregator umatched_words
+      context.aggregate_to(self.umatched_words, 1)
+```
+
 If you execute your pipeline using `DataflowRunner`, you can control the worker log levels. Dataflow workers that execute user code are configured to log to Cloud Logging by default at "INFO" log level and higher. You can override log levels for specific logging namespaces by specifying: `--workerLogLevelOverrides={"Name1":"Level1","Name2":"Level2",...}`. For example, by specifying `--workerLogLevelOverrides={"org.apache.beam.examples":"DEBUG"}` when executing this pipeline using the Dataflow service, Cloud Logging would contain only "DEBUG" or higher level logs for the package in addition to the default "INFO" or higher level logs. 
 
 The default Dataflow worker logging configuration can be overridden by specifying `--defaultWorkerLogLevel=<one of TRACE, DEBUG, INFO, WARN, ERROR>`. For example, by specifying `--defaultWorkerLogLevel=DEBUG` when executing this pipeline with the Dataflow service, Cloud Logging would contain all "DEBUG" or higher level logs. Note that changing the default worker log level to TRACE or DEBUG will significantly increase the amount of logs output.
@@ -309,6 +427,10 @@ public static void main(String[] args) {
 }
 ```
 
+```py
+This feature is not yet available in the Beam SDK for Python.
+```
+
 ## WindowedWordCount
 
 This example, `WindowedWordCount`, counts words in text just as the previous examples did, but introduces several advanced concepts.
@@ -338,6 +460,10 @@ public static void main(String[] args) throws IOException {
 
 ```
 
+```py
+This feature is not yet available in the Beam SDK for Python.
+```
+
 ### Adding Timestamps to Data
 
 Each element in a `PCollection` has an associated **timestamp**. The timestamp for each element is initially assigned by the source that creates the `PCollection` and can be adjusted by a `DoFn`. In this example the input is bounded. For the purpose of the example, the `DoFn` method named `AddTimestampsFn` (invoked by `ParDo`) will set a timestamp for each element in the `PCollection`.
@@ -346,6 +472,10 @@ Each element in a `PCollection` has an associated **timestamp**. The timestamp f
 .apply(ParDo.of(new AddTimestampFn()));
 ```
 
+```py
+This feature is not yet available in the Beam SDK for Python.
+```
+
 Below is the code for `AddTimestampFn`, a `DoFn` invoked by `ParDo`, that sets the data element of the timestamp given the element itself. For example, if the elements were log lines, this `ParDo` could parse the time out of the log string and set it as the element's timestamp. There are no timestamps inherent in the works of Shakespeare, so in this case we've made up random timestamps just to illustrate the concept. Each line of the input text will get a random associated timestamp sometime in a 2-hour period.
 
 ```java
@@ -369,6 +499,10 @@ static class AddTimestampFn extends DoFn<String, String> {
 }
 ```
 
+```py
+This feature is not yet available in the Beam SDK for Python.
+```
+
 ### Windowing
 
 Beam uses a concept called **Windowing** to subdivide a `PCollection` according to the timestamps of its individual elements. `PTransforms` that aggregate multiple elements, process each `PCollection` as a succession of multiple, finite windows, even though the entire collection itself may be of infinite size (unbounded).
@@ -381,14 +515,22 @@ PCollection<String> windowedWords = input
     FixedWindows.of(Duration.standardMinutes(options.getWindowSize()))));
 ```
 
+```py
+This feature is not yet available in the Beam SDK for Python.
+```
+
 ### Reusing PTransforms over windowed PCollections
 
 You can reuse existing `PTransform`s, that were created for manipulating simple `PCollection`s, over windowed `PCollection`s as well.
 
-```
+```java
 PCollection<KV<String, Long>> wordCounts = windowedWords.apply(new WordCount.CountWords());
 ```
 
+```py
+This feature is not yet available in the Beam SDK for Python.
+```
+
 ### Write Results to an Unbounded Sink
 
 Since our input is unbounded, the same is true of our output `PCollection`. We need to make sure that we choose an appropriate, unbounded sink. Some output sinks support only bounded output, such as a text file. Google Cloud BigQuery is an output source that supports both bounded and unbounded input.
@@ -404,3 +546,7 @@ wordCounts.apply(ParDo.of(new FormatAsTableRowFn()))
       .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_APPEND));
 ```
 
+```py
+This feature is not yet available in the Beam SDK for Python.
+```
+


[2/5] beam-site git commit: Added Python Quickstart

Posted by da...@apache.org.
Added Python Quickstart


Project: http://git-wip-us.apache.org/repos/asf/beam-site/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam-site/commit/852da20d
Tree: http://git-wip-us.apache.org/repos/asf/beam-site/tree/852da20d
Diff: http://git-wip-us.apache.org/repos/asf/beam-site/diff/852da20d

Branch: refs/heads/asf-site
Commit: 852da20d32f25fb89b55e74f5f1f86040d353cbd
Parents: fb8666e
Author: Hadar Hod <ha...@google.com>
Authored: Wed Jan 25 01:46:37 2017 -0800
Committer: Davor Bonaci <da...@google.com>
Committed: Mon Jan 30 17:35:51 2017 -0800

----------------------------------------------------------------------
 src/_includes/header.html            |   3 +-
 src/get-started/beam-overview.md     |   6 +-
 src/get-started/quickstart-java.md   | 236 ++++++++++++++++++++++++++++++
 src/get-started/quickstart-py.md     |  97 ++++++++++++
 src/get-started/quickstart.md        | 235 -----------------------------
 src/get-started/wordcount-example.md | 119 ++++-----------
 6 files changed, 367 insertions(+), 329 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam-site/blob/852da20d/src/_includes/header.html
----------------------------------------------------------------------
diff --git a/src/_includes/header.html b/src/_includes/header.html
index 6ffd1fe..d44e0dc 100644
--- a/src/_includes/header.html
+++ b/src/_includes/header.html
@@ -17,7 +17,8 @@
 		  <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Get Started <span class="caret"></span></a>
 		  <ul class="dropdown-menu">
 			  <li><a href="{{ site.baseurl }}/get-started/beam-overview/">Beam Overview</a></li>
-              <li><a href="{{ site.baseurl }}/get-started/quickstart/">Quickstart</a></li>
+        <li><a href="{{ site.baseurl }}/get-started/quickstart-java/">Quickstart - Java</a></li>
+        <li><a href="{{ site.baseurl }}/get-started/quickstart-py/">Quickstart - Python</a></li>
 			  <li role="separator" class="divider"></li>
 			  <li class="dropdown-header">Example Walkthroughs</li>
 			  <li><a href="{{ site.baseurl }}/get-started/wordcount-example/">WordCount</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/852da20d/src/get-started/beam-overview.md
----------------------------------------------------------------------
diff --git a/src/get-started/beam-overview.md b/src/get-started/beam-overview.md
index 79716a9..6796c2e 100644
--- a/src/get-started/beam-overview.md
+++ b/src/get-started/beam-overview.md
@@ -71,4 +71,8 @@ Beam currently supports Runners that work with the following distributed process
 
 ## Getting Started with Apache Beam
 
-Get started using Beam for your data processing tasks by following the [Quickstart]({{ site.baseurl }}/get-started/quickstart) and the [WordCount Examples Walkthrough]({{ site.baseurl }}/get-started/wordcount-example).
+Get started using Beam for your data processing tasks. 
+
+1. Follow the Quickstart for the [Java SDK]({{ site.baseurl }}/get-started/quickstart-java) or the [Python SDK]({{ site.baseurl }}/get-started/quickstart-py).
+
+2. See the [WordCount Examples Walkthrough]({{ site.baseurl }}/get-started/wordcount-example) for examples that introduce various features of the SDKs.

http://git-wip-us.apache.org/repos/asf/beam-site/blob/852da20d/src/get-started/quickstart-java.md
----------------------------------------------------------------------
diff --git a/src/get-started/quickstart-java.md b/src/get-started/quickstart-java.md
new file mode 100644
index 0000000..a97a61f
--- /dev/null
+++ b/src/get-started/quickstart-java.md
@@ -0,0 +1,236 @@
+---
+layout: default
+title: "Beam Quickstart for Java"
+permalink: /get-started/quickstart-java/
+redirect_from:
+  - /get-started/quickstart/
+  - /use/quickstart/
+  - /getting-started/
+---
+
+# Apache Beam Java SDK Quickstart
+
+This Quickstart will walk you through executing your first Beam pipeline to run [WordCount]({{ site.baseurl }}/get-started/wordcount-example), written using Beam's [Java SDK]({{ site.baseurl }}/documentation/sdks/java), on a [runner]({{ site.baseurl }}/documentation#runners) of your choice.
+
+* TOC
+{:toc}
+
+
+## Set up your Development Environment
+
+1. Download and install the [Java Development Kit (JDK)](http://www.oracle.com/technetwork/java/javase/downloads/index.html) version 1.7 or later. Verify that the [JAVA_HOME](https://docs.oracle.com/javase/8/docs/technotes/guides/troubleshoot/envvars001.html) environment variable is set and points to your JDK installation.
+
+1. 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.
+
+
+## Get the WordCount Code
+
+The easiest way to get a copy of the WordCount pipeline is to use the following command to generate a simple Maven project that contains Beam's WordCount examples and builds against the most recent Beam release:
+
+```
+$ mvn archetype:generate \
+      -DarchetypeRepository=https://repository.apache.org/content/groups/snapshots \
+      -DarchetypeGroupId=org.apache.beam \
+      -DarchetypeArtifactId=beam-sdks-java-maven-archetypes-examples \
+      -DarchetypeVersion=LATEST \
+      -DgroupId=org.example \
+      -DartifactId=word-count-beam \
+      -Dversion="0.1" \
+      -Dpackage=org.apache.beam.examples \
+      -DinteractiveMode=false
+```
+
+This will create a directory `word-count-beam` that contains a simple `pom.xml` and a series of example pipelines that count words in text files.
+
+```
+$ cd word-count-beam/
+
+$ ls
+pom.xml	src
+
+$ ls src/main/java/org/apache/beam/examples/
+DebuggingWordCount.java	WindowedWordCount.java	common
+MinimalWordCount.java	WordCount.java
+```
+
+For a detailed introduction to the Beam concepts used in these examples, see the [WordCount Example Walkthrough]({{ site.baseurl }}/get-started/wordcount-example). Here, we'll just focus on executing `WordCount.java`.
+
+
+## Run WordCount
+
+A single Beam pipeline can run on multiple Beam [runners]({{ site.baseurl }}/documentation#runners), including the [ApexRunner]({{ site.baseurl }}/documentation/runners/apex), [FlinkRunner]({{ site.baseurl }}/documentation/runners/flink), [SparkRunner]({{ site.baseurl }}/documentation/runners/spark) or [DataflowRunner]({{ site.baseurl }}/documentation/runners/dataflow). The [DirectRunner]({{ site.baseurl }}/documentation/runners/direct) is a common runner for getting started, as it runs locally on your machine and requires no specific setup.
+
+After you've chosen which runner you'd like to use:
+
+1.  Ensure you've done any runner-specific setup.
+1.  Build your commandline by:
+    1. Specifying a specific runner with `--runner=<runner>` (defaults to the [DirectRunner]({{ site.baseurl }}/documentation/runners/direct))
+    1. Adding any runner-specific required options
+    1. Choosing input files and an output location are accessible on the chosen runner. (For example, you can't access a local file if you are running the pipeline on an external cluster.)
+1.  Run your first WordCount pipeline.
+
+{:.runner-direct}
+```
+$ mvn compile exec:java -Dexec.mainClass=org.apache.beam.examples.WordCount \
+     -Dexec.args="--inputFile=pom.xml --output=counts" -Pdirect-runner
+```
+
+{:.runner-apex}
+```
+$ mvn compile exec:java -Dexec.mainClass=org.apache.beam.examples.WordCount \
+     -Dexec.args="--inputFile=pom.xml --output=counts --runner=ApexRunner" -Papex-runner
+```
+
+{:.runner-flink-local}
+```
+$ mvn compile exec:java -Dexec.mainClass=org.apache.beam.examples.WordCount \
+     -Dexec.args="--runner=FlinkRunner --inputFile=pom.xml --output=counts" -Pflink-runner
+```
+
+{:.runner-flink-cluster}
+```
+$ mvn package exec:java -Dexec.mainClass=org.apache.beam.examples.WordCount \
+     -Dexec.args="--runner=FlinkRunner --flinkMaster=<flink master> --filesToStage=target/word-count-beam-bundled-0.1.jar \
+                  --inputFile=/path/to/quickstart/pom.xml --output=/tmp/counts" -Pflink-runner
+
+You can monitor the running job by visiting the Flink dashboard at http://<flink master>:8081
+```
+
+{:.runner-spark}
+```
+$ mvn compile exec:java -Dexec.mainClass=org.apache.beam.examples.WordCount \
+     -Dexec.args="--runner=SparkRunner --inputFile=pom.xml --output=counts" -Pspark-runner
+```
+
+{:.runner-dataflow}
+```
+$ mvn compile exec:java -Dexec.mainClass=org.apache.beam.examples.WordCount \
+	 -Dexec.args="--runner=DataflowRunner --gcpTempLocation=gs://<your-gcs-bucket>/tmp \
+	              --inputFile=gs://apache-beam-samples/shakespeare/* --output=gs://<your-gcs-bucket>/counts" \
+     -Pdataflow-runner
+```
+
+
+## Inspect the results
+
+Once the pipeline has completed, you can view the output. You'll notice that there may be multiple output files prefixed by `count`. The exact number of these files is decided by the runner, giving it the flexibility to do efficient, distributed execution.
+
+{:.runner-direct}
+```
+$ ls counts*
+```
+
+{:.runner-apex}
+```
+$ ls counts*
+```
+
+{:.runner-flink-local}
+```
+$ ls counts*
+```
+
+{:.runner-flink-cluster}
+```
+$ ls /tmp/counts*
+```
+
+{:.runner-spark}
+```
+$ ls counts*
+```
+
+
+{:.runner-dataflow}
+```
+$ gsutil ls gs://<your-gcs-bucket>/counts*
+```
+
+When you look into the contents of the file, you'll see that they contain unique words and the number of occurrences of each word. The order of elements within the file may differ because the Beam model does not generally guarantee ordering, again to allow runners to optimize for efficiency.
+
+{:.runner-direct}
+```
+$ more counts*
+api: 9
+bundled: 1
+old: 4
+Apache: 2
+The: 1
+limitations: 1
+Foundation: 1
+...
+```
+
+{:.runner-apex}
+```
+$ cat counts*
+BEAM: 1
+have: 1
+simple: 1
+skip: 4
+PAssert: 1
+...
+```
+
+{:.runner-flink-local}
+```
+$ more counts*
+The: 1
+api: 9
+old: 4
+Apache: 2
+limitations: 1
+bundled: 1
+Foundation: 1
+...
+```
+
+{:.runner-flink-cluster}
+```
+$ more /tmp/counts*
+The: 1
+api: 9
+old: 4
+Apache: 2
+limitations: 1
+bundled: 1
+Foundation: 1
+...
+```
+
+{:.runner-spark}
+```
+$ more counts*
+beam: 27
+SF: 1
+fat: 1
+job: 1
+limitations: 1
+require: 1
+of: 11
+profile: 10
+...
+```
+
+{:.runner-dataflow}
+```
+$ gsutil cat gs://<your-gcs-bucket>/counts*
+feature: 15
+smother'st: 1
+revelry: 1
+bashfulness: 1
+Bashful: 1
+Below: 2
+deserves: 32
+barrenly: 1
+...
+```
+
+## Next Steps
+
+* Learn more about these WordCount examples in the [WordCount Example Walkthrough]({{ site.baseurl }}/get-started/wordcount-example).
+* Dive in to some of our favorite [articles and presentations]({{ site.baseurl }}/documentation/resources).
+* Join the Beam [users@]({{ site.baseurl }}/get-started/support#mailing-lists) mailing list.
+
+Please don't hesitate to [reach out]({{ site.baseurl }}/get-started/support) if you encounter any issues!
+

http://git-wip-us.apache.org/repos/asf/beam-site/blob/852da20d/src/get-started/quickstart-py.md
----------------------------------------------------------------------
diff --git a/src/get-started/quickstart-py.md b/src/get-started/quickstart-py.md
new file mode 100644
index 0000000..a198eba
--- /dev/null
+++ b/src/get-started/quickstart-py.md
@@ -0,0 +1,97 @@
+---
+layout: default
+title: "Beam Quickstart for Python"
+permalink: /get-started/quickstart-py/
+---
+
+# Apache Beam Python SDK Quickstart
+
+This guide shows you how to set up your Python development environment, get the Apache Beam SDK for Python, and run an example pipeline.
+
+* TOC
+{:toc}
+
+## Set up your environment
+
+### Install pip
+
+Install [pip](https://pip.pypa.io/en/stable/installing/), Python's package manager. Check that you have version 7.0.0 or newer, by running: 
+
+```
+pip --version
+```
+
+### Install Python virtual environment 
+
+It is recommended that you install a [Python virtual environment](http://docs.python-guide.org/en/latest/dev/virtualenvs/)
+for initial experiments. If you do not have `virtualenv` version 13.1.0 or newer, install it by running:
+
+```
+pip install --upgrade virtualenv
+```
+
+If you do not want to use a Python virtual environment (not recommended), ensure `setuptools` is installed on your machine. If you do not have `setuptools` version 17.1 or newer, install it by running:
+
+```
+pip install --upgrade setuptools
+```
+
+## Get Apache Beam
+
+### Create and activate a virtual environment
+
+A virtual environment is a directory tree containing its own Python distribution. To create a virtual environment, create a directory and run:
+
+```
+virtualenv /path/to/directory
+```
+
+A virtual environment needs to be activated for each shell that is to use it.
+Activating it sets some environment variables that point to the virtual
+environment's directories. 
+
+To activate a virtual environment in Bash, run:
+
+```
+. /path/to/directory/bin/activate
+```
+
+That is, source the script `bin/activate` under the virtual environment directory you created.
+
+For instructions using other shells, see the [virtualenv documentation](https://virtualenv.pypa.io/en/stable/userguide/#activate-script).
+
+### Download and install
+
+1. Clone the Apache Beam repo from GitHub: 
+  `git clone https://github.com/apache/beam.git --branch python-sdk`
+
+2. Navigate to the `python` directory: 
+  `cd beam/sdks/python/`
+
+3. Create the Apache Beam Python SDK installation package: 
+  `python setup.py sdist`
+
+4. Navigate to the `dist` directory:
+  `cd dist/`
+
+5. Install the Apache Beam SDK
+  `pip install apache-beam-sdk-*.tar.gz`
+
+## Execute a pipeline locally
+
+The Apache Beam [examples](https://github.com/apache/beam/tree/python-sdk/sdks/python/apache_beam/examples) directory has many examples. All examples can be run locally by passing the required arguments described in the example script.
+
+For example, to run `wordcount.py`, run:
+
+```
+python -m apache_beam.examples.wordcount --input gs://dataflow-samples/shakespeare/kinglear.txt --output output.txt
+```
+
+## Next Steps
+
+* Learn more about these WordCount examples in the [WordCount Example Walkthrough]({{ site.baseurl }}/get-started/wordcount-example).
+* Dive in to some of our favorite [articles and presentations]({{ site.baseurl }}/documentation/resources).
+* Join the Beam [users@]({{ site.baseurl }}/get-started/support#mailing-lists) mailing list.
+
+Please don't hesitate to [reach out]({{ site.baseurl }}/get-started/support) if you encounter any issues!
+

http://git-wip-us.apache.org/repos/asf/beam-site/blob/852da20d/src/get-started/quickstart.md
----------------------------------------------------------------------
diff --git a/src/get-started/quickstart.md b/src/get-started/quickstart.md
deleted file mode 100644
index e10f9f3..0000000
--- a/src/get-started/quickstart.md
+++ /dev/null
@@ -1,235 +0,0 @@
----
-layout: default
-title: "Beam Quickstart"
-permalink: /get-started/quickstart/
-redirect_from:
-  - /use/quickstart/
-  - /getting-started/
----
-
-# Apache Beam Java SDK Quickstart
-
-This Quickstart will walk you through executing your first Beam pipeline to run [WordCount]({{ site.baseurl }}/get-started/wordcount-example), written using Beam's [Java SDK]({{ site.baseurl }}/documentation/sdks/java), on a [runner]({{ site.baseurl }}/documentation#runners) of your choice.
-
-* TOC
-{:toc}
-
-
-## Set up your Development Environment
-
-1. Download and install the [Java Development Kit (JDK)](http://www.oracle.com/technetwork/java/javase/downloads/index.html) version 1.7 or later. Verify that the [JAVA_HOME](https://docs.oracle.com/javase/8/docs/technotes/guides/troubleshoot/envvars001.html) environment variable is set and points to your JDK installation.
-
-1. 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.
-
-
-## Get the WordCount Code
-
-The easiest way to get a copy of the WordCount pipeline is to use the following command to generate a simple Maven project that contains Beam's WordCount examples and builds against the most recent Beam release:
-
-```
-$ mvn archetype:generate \
-      -DarchetypeRepository=https://repository.apache.org/content/groups/snapshots \
-      -DarchetypeGroupId=org.apache.beam \
-      -DarchetypeArtifactId=beam-sdks-java-maven-archetypes-examples \
-      -DarchetypeVersion=LATEST \
-      -DgroupId=org.example \
-      -DartifactId=word-count-beam \
-      -Dversion="0.1" \
-      -Dpackage=org.apache.beam.examples \
-      -DinteractiveMode=false
-```
-
-This will create a directory `word-count-beam` that contains a simple `pom.xml` and a series of example pipelines that count words in text files.
-
-```
-$ cd word-count-beam/
-
-$ ls
-pom.xml	src
-
-$ ls src/main/java/org/apache/beam/examples/
-DebuggingWordCount.java	WindowedWordCount.java	common
-MinimalWordCount.java	WordCount.java
-```
-
-For a detailed introduction to the Beam concepts used in these examples, see the [WordCount Example Walkthrough]({{ site.baseurl }}/get-started/wordcount-example). Here, we'll just focus on executing `WordCount.java`.
-
-
-## Run WordCount
-
-A single Beam pipeline can run on multiple Beam [runners]({{ site.baseurl }}/documentation#runners), including the [ApexRunner]({{ site.baseurl }}/documentation/runners/apex), [FlinkRunner]({{ site.baseurl }}/documentation/runners/flink), [SparkRunner]({{ site.baseurl }}/documentation/runners/spark) or [DataflowRunner]({{ site.baseurl }}/documentation/runners/dataflow). The [DirectRunner]({{ site.baseurl }}/documentation/runners/direct) is a common runner for getting started, as it runs locally on your machine and requires no specific setup.
-
-After you've chosen which runner you'd like to use:
-
-1.  Ensure you've done any runner-specific setup.
-1.  Build your commandline by:
-    1. Specifying a specific runner with `--runner=<runner>` (defaults to the [DirectRunner]({{ site.baseurl }}/documentation/runners/direct))
-    1. Adding any runner-specific required options
-    1. Choosing input files and an output location are accessible on the chosen runner. (For example, you can't access a local file if you are running the pipeline on an external cluster.)
-1.  Run your first WordCount pipeline.
-
-{:.runner-direct}
-```
-$ mvn compile exec:java -Dexec.mainClass=org.apache.beam.examples.WordCount \
-     -Dexec.args="--inputFile=pom.xml --output=counts" -Pdirect-runner
-```
-
-{:.runner-apex}
-```
-$ mvn compile exec:java -Dexec.mainClass=org.apache.beam.examples.WordCount \
-     -Dexec.args="--inputFile=pom.xml --output=counts --runner=ApexRunner" -Papex-runner
-```
-
-{:.runner-flink-local}
-```
-$ mvn compile exec:java -Dexec.mainClass=org.apache.beam.examples.WordCount \
-     -Dexec.args="--runner=FlinkRunner --inputFile=pom.xml --output=counts" -Pflink-runner
-```
-
-{:.runner-flink-cluster}
-```
-$ mvn package exec:java -Dexec.mainClass=org.apache.beam.examples.WordCount \
-     -Dexec.args="--runner=FlinkRunner --flinkMaster=<flink master> --filesToStage=target/word-count-beam-bundled-0.1.jar \
-                  --inputFile=/path/to/quickstart/pom.xml --output=/tmp/counts" -Pflink-runner
-
-You can monitor the running job by visiting the Flink dashboard at http://<flink master>:8081
-```
-
-{:.runner-spark}
-```
-$ mvn compile exec:java -Dexec.mainClass=org.apache.beam.examples.WordCount \
-     -Dexec.args="--runner=SparkRunner --inputFile=pom.xml --output=counts" -Pspark-runner
-```
-
-{:.runner-dataflow}
-```
-$ mvn compile exec:java -Dexec.mainClass=org.apache.beam.examples.WordCount \
-	 -Dexec.args="--runner=DataflowRunner --gcpTempLocation=gs://<your-gcs-bucket>/tmp \
-	              --inputFile=gs://apache-beam-samples/shakespeare/* --output=gs://<your-gcs-bucket>/counts" \
-     -Pdataflow-runner
-```
-
-
-## Inspect the results
-
-Once the pipeline has completed, you can view the output. You'll notice that there may be multiple output files prefixed by `count`. The exact number of these files is decided by the runner, giving it the flexibility to do efficient, distributed execution.
-
-{:.runner-direct}
-```
-$ ls counts*
-```
-
-{:.runner-apex}
-```
-$ ls counts*
-```
-
-{:.runner-flink-local}
-```
-$ ls counts*
-```
-
-{:.runner-flink-cluster}
-```
-$ ls /tmp/counts*
-```
-
-{:.runner-spark}
-```
-$ ls counts*
-```
-
-
-{:.runner-dataflow}
-```
-$ gsutil ls gs://<your-gcs-bucket>/counts*
-```
-
-When you look into the contents of the file, you'll see that they contain unique words and the number of occurrences of each word. The order of elements within the file may differ because the Beam model does not generally guarantee ordering, again to allow runners to optimize for efficiency.
-
-{:.runner-direct}
-```
-$ more counts*
-api: 9
-bundled: 1
-old: 4
-Apache: 2
-The: 1
-limitations: 1
-Foundation: 1
-...
-```
-
-{:.runner-apex}
-```
-$ cat counts*
-BEAM: 1
-have: 1
-simple: 1
-skip: 4
-PAssert: 1
-...
-```
-
-{:.runner-flink-local}
-```
-$ more counts*
-The: 1
-api: 9
-old: 4
-Apache: 2
-limitations: 1
-bundled: 1
-Foundation: 1
-...
-```
-
-{:.runner-flink-cluster}
-```
-$ more /tmp/counts*
-The: 1
-api: 9
-old: 4
-Apache: 2
-limitations: 1
-bundled: 1
-Foundation: 1
-...
-```
-
-{:.runner-spark}
-```
-$ more counts*
-beam: 27
-SF: 1
-fat: 1
-job: 1
-limitations: 1
-require: 1
-of: 11
-profile: 10
-...
-```
-
-{:.runner-dataflow}
-```
-$ gsutil cat gs://<your-gcs-bucket>/counts*
-feature: 15
-smother'st: 1
-revelry: 1
-bashfulness: 1
-Bashful: 1
-Below: 2
-deserves: 32
-barrenly: 1
-...
-```
-
-## Next Steps
-
-* Learn more about these WordCount examples in the [WordCount Example Walkthrough]({{ site.baseurl }}/get-started/wordcount-example).
-* Dive in to some of our favorite [articles and presentations]({{ site.baseurl }}/documentation/resources).
-* Join the Beam [users@]({{ site.baseurl }}/get-started/support#mailing-lists) mailing list.
-
-Please don't hesitate to [reach out]({{ site.baseurl }}/get-started/support) if you encounter any issues!
-

http://git-wip-us.apache.org/repos/asf/beam-site/blob/852da20d/src/get-started/wordcount-example.md
----------------------------------------------------------------------
diff --git a/src/get-started/wordcount-example.md b/src/get-started/wordcount-example.md
index a02b327..bf484b2 100644
--- a/src/get-started/wordcount-example.md
+++ b/src/get-started/wordcount-example.md
@@ -69,14 +69,8 @@ You can specify a runner for executing your pipeline, such as the `DataflowRunne
 ```
 
 ```py
-options = PipelineOptions()
-google_cloud_options = options.view_as(GoogleCloudOptions)
-google_cloud_options.project = 'my-project-id'
-google_cloud_options.job_name = 'myjob'
-google_cloud_options.staging_location = 'gs://your-bucket-name-here/staging'
-google_cloud_options.temp_location = 'gs://your-bucket-name-here/temp'
-options.view_as(StandardOptions).runner = 'BlockingDataflowPipelineRunner'
-```
+{% github_sample /apache/beam/blob/python-sdk/sdks/python/apache_beam/examples/snippets/snippets.py tag:examples_wordcount_minimal_options
+%}```
 
 The next step is to create a Pipeline object with the options we've just constructed. The Pipeline object builds up the graph of transformations to be executed, associated with that particular pipeline.
 
@@ -85,8 +79,8 @@ Pipeline p = Pipeline.create(options);
 ```
 
 ```py
-p = beam.Pipeline(options=options)
-```
+{% github_sample /apache/beam/blob/python-sdk/sdks/python/apache_beam/examples/snippets/snippets.py tag:examples_wordcount_minimal_create
+%}```
 
 ### Applying Pipeline Transforms
 
@@ -106,8 +100,8 @@ The Minimal WordCount pipeline contains five transforms:
     ```
 
     ```py
-    p | beam.io.Read(beam.io.TextFileSource('gs://dataflow-samples/shakespeare/kinglear.txt'))
-    ```
+    {% github_sample /apache/beam/blob/python-sdk/sdks/python/apache_beam/examples/snippets/snippets.py tag:examples_wordcount_minimal_read
+    %}```
 
 2.  A [ParDo]({{ site.baseurl }}/documentation/programming-guide/#transforms-pardo) transform that invokes a `DoFn` (defined in-line as an anonymous class) on each element that tokenizes the text lines into individual words. The input for this transform is the `PCollection` of text lines generated by the previous `TextIO.Read` transform. The `ParDo` transform outputs a new `PCollection`, where each element represents an individual word in the text.
 
@@ -126,8 +120,8 @@ The Minimal WordCount pipeline contains five transforms:
 
     ```py
     # The Flatmap transform is a simplified version of ParDo.
-    | 'ExtractWords' >> beam.FlatMap(lambda x: re.findall(r'[A-Za-z\']+', x))
-    ```
+    {% github_sample /apache/beam/blob/python-sdk/sdks/python/apache_beam/examples/snippets/snippets.py tag:examples_wordcount_minimal_pardo
+    %}```
 
 3.  The SDK-provided `Count` transform is a generic transform that takes a `PCollection` of any type, and returns a `PCollection` of key/value pairs. 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.
 
@@ -138,12 +132,12 @@ The Minimal WordCount pipeline contains five transforms:
     ```
 
     ```py
-    | beam.combiners.Count.PerElement()
-    ```
+    {% github_sample /apache/beam/blob/python-sdk/sdks/python/apache_beam/examples/snippets/snippets.py tag:examples_wordcount_minimal_count
+    %}```
 
 4.  The next transform formats each of the key/value pairs of unique words and occurrence counts into a printable string suitable for writing to an output file.
 
-	`MapElements` is a higher-level composite transform that encapsulates a simple `ParDo`; for each element in the input `PCollection`, `MapElements` applies a function that produces exactly one output element. In this example, `MapElements` invokes a `SimpleFunction` (defined in-line as an anonymous class) that does the formatting. As input, `MapElements` takes a `PCollection` of key/value pairs generated by `Count`, and produces a new `PCollection` of printable strings.
+	The map transform is a higher-level composite transform that encapsulates a simple `ParDo`; for each element in the input `PCollection`, the map transform applies a function that produces exactly one output element.
 
     ```java
     .apply("FormatResults", MapElements.via(new SimpleFunction<KV<String, Long>, String>() {
@@ -155,19 +149,19 @@ The Minimal WordCount pipeline contains five transforms:
     ```
 
     ```py
-    | beam.Map(lambda (word, count): '%s: %s' % (word, count))
-    ```
+    {% github_sample /apache/beam/blob/python-sdk/sdks/python/apache_beam/examples/snippets/snippets.py tag:examples_wordcount_minimal_map
+    %}```
 
-5.  A text file `Write`. This transform takes the final `PCollection` of formatted Strings as input and writes each element to an output text file. Each element in the input `PCollection` represents one line of text in the resulting output file.
+5.  A text file write transform. This transform takes the final `PCollection` of formatted Strings as input and writes each element to an output text file. Each element in the input `PCollection` represents one line of text in the resulting output file.
 
     ```java
     .apply(TextIO.Write.to("wordcounts"));
     ```
 
     ```py
-    | beam.io.Write(beam.io.TextFileSink('gs://my-bucket/counts.txt'))
-    ```
-
+    {% github_sample /apache/beam/blob/python-sdk/sdks/python/apache_beam/examples/snippets/snippets.py tag:examples_wordcount_minimal_write
+    %}```
+    
 Note that the `Write` transform produces a trivial result value of type `PDone`, which in this case is ignored.
 
 ### Running the Pipeline
@@ -179,8 +173,8 @@ p.run().waitUntilFinish();
 ```
 
 ```py
-p.run()
-```
+{% github_sample /apache/beam/blob/python-sdk/sdks/python/apache_beam/examples/snippets/snippets.py tag:examples_wordcount_minimal_run
+%}```
 
 Note that the `run` method is asynchronous. For a blocking execution instead, run your pipeline appending the `waitUntilFinish` method.
 
@@ -220,14 +214,8 @@ static class ExtractWordsFn extends DoFn<String, String> {
 ```py
 # In this example, the DoFns are defined as classes:
 
-class FormatAsTextFn(beam.DoFn):
-
-  def process(self, context):
-    word, count = context.element
-    yield '%s: %s' % (word, count)
-
-formatted = counts | beam.ParDo(FormatAsTextFn())
-```
+{% github_sample /apache/beam/blob/python-sdk/sdks/python/apache_beam/examples/snippets/snippets.py tag:examples_wordcount_wordcount_dofn
+%}```
 
 ### Creating Composite Transforms
 
@@ -265,19 +253,8 @@ public static void main(String[] args) throws IOException {
 ```
 
 ```py
-class CountWords(beam.PTransform):
-
-  def apply(self, pcoll):
-    return (pcoll
-            # Convert lines of text into individual words.
-            | beam.FlatMap(
-                'ExtractWords', lambda x: re.findall(r'[A-Za-z\']+', x))
-
-            # Count the number of times each word occurs.
-            | beam.combiners.Count.PerElement())
-
-counts = lines | CountWords()
-```
+{% github_sample /apache/beam/blob/python-sdk/sdks/python/apache_beam/examples/snippets/snippets.py tag:examples_wordcount_wordcount_composite
+%}```
 
 ### Using Parameterizable PipelineOptions
 
@@ -303,17 +280,8 @@ public static void main(String[] args) {
 ```
 
 ```py
-class WordCountOptions(PipelineOptions):
-
-  @classmethod
-  def _add_argparse_args(cls, parser):
-    parser.add_argument('--input',
-                        help='Input for the dataflow pipeline',
-                        default='gs://my-bucket/input')
-
-options = PipelineOptions(argv)
-p = beam.Pipeline(options=options)
-```
+{% github_sample /apache/beam/blob/python-sdk/sdks/python/apache_beam/examples/snippets/snippets.py tag:examples_wordcount_wordcount_options
+%}```
 
 ## Debugging WordCount Example
 
@@ -362,41 +330,8 @@ public class DebuggingWordCount {
 ```
 
 ```py
-import logging
-
-class FilterTextFn(beam.DoFn):
-  """A DoFn that filters for a specific key based on a regular expression."""
-
-  # A custom aggregator can track values in your pipeline as it runs. Create
-  # custom aggregators matched_word and unmatched_words.
-  matched_words = beam.Aggregator('matched_words')
-  umatched_words = beam.Aggregator('umatched_words')
-
-  def __init__(self, pattern):
-    self.pattern = pattern
-
-  def process(self, context):
-    word, _ = context.element
-    if re.match(self.pattern, word):
-      # Log at INFO level each element we match. When executing this pipeline
-      # using the Dataflow service, these log lines will appear in the Cloud
-      # Logging UI.
-      logging.info('Matched %s', word)
-
-      # Add 1 to the custom aggregator matched_words
-      context.aggregate_to(self.matched_words, 1)
-      yield context.element
-    else:
-      # Log at the "DEBUG" level each element that is not matched. Different
-      # log levels can be used to control the verbosity of logging providing
-      # an effective mechanism to filter less important information. Note
-      # currently only "INFO" and higher level logs are emitted to the Cloud
-      # Logger. This log message will not be visible in the Cloud Logger.
-      logging.debug('Did not match %s', word)
-
-      # Add 1 to the custom aggregator umatched_words
-      context.aggregate_to(self.umatched_words, 1)
-```
+{% github_sample /apache/beam/blob/python-sdk/sdks/python/apache_beam/examples/snippets/snippets.py tag:example_wordcount_debugging_logging
+%}```
 
 If you execute your pipeline using `DataflowRunner`, you can control the worker log levels. Dataflow workers that execute user code are configured to log to Cloud Logging by default at "INFO" log level and higher. You can override log levels for specific logging namespaces by specifying: `--workerLogLevelOverrides={"Name1":"Level1","Name2":"Level2",...}`. For example, by specifying `--workerLogLevelOverrides={"org.apache.beam.examples":"DEBUG"}` when executing this pipeline using the Dataflow service, Cloud Logging would contain only "DEBUG" or higher level logs for the package in addition to the default "INFO" or higher level logs. 
 


[3/5] beam-site git commit: Regenerate website

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam-site/blob/9727633e/content/get-started/quickstart/index.html
----------------------------------------------------------------------
diff --git a/content/get-started/quickstart/index.html b/content/get-started/quickstart/index.html
index 53e0147..87e96c0 100644
--- a/content/get-started/quickstart/index.html
+++ b/content/get-started/quickstart/index.html
@@ -1,394 +1,10 @@
 <!DOCTYPE html>
-<html lang="en">
-
-  <head>
-  <meta charset="utf-8">
-  <meta http-equiv="X-UA-Compatible" content="IE=edge">
-  <meta name="viewport" content="width=device-width, initial-scale=1">
-
-  <title>Beam Quickstart</title>
-  <meta name="description" content="Apache Beam is an open source, unified model and set of language-specific SDKs for defining and executing data processing workflows, and also data ingestion and integration flows, supporting Enterprise Integration Patterns (EIPs) and Domain Specific Languages (DSLs). Dataflow pipelines simplify the mechanics of large-scale batch and streaming data processing and can run on a number of runtimes like Apache Flink, Apache Spark, and Google Cloud Dataflow (a cloud service). Beam also brings DSL in different languages, allowing users to easily implement their data integration processes.
-">
-
-  <link rel="stylesheet" href="/styles/site.css">
-  <link rel="stylesheet" href="/css/theme.css">
-  <script src="https://ajax.googleapis.com/ajax/libs/jquery/2.2.0/jquery.min.js"></script>
-  <script src="/js/bootstrap.min.js"></script>
-  <script src="/js/language-switch.js"></script>
-  <link rel="canonical" href="http://beam.apache.org/get-started/quickstart/" data-proofer-ignore>
-  <link rel="alternate" type="application/rss+xml" title="Apache Beam" href="http://beam.apache.org/feed.xml">
-  <script>
-    (function(i,s,o,g,r,a,m){i['GoogleAnalyticsObject']=r;i[r]=i[r]||function(){
-    (i[r].q=i[r].q||[]).push(arguments)},i[r].l=1*new Date();a=s.createElement(o),
-    m=s.getElementsByTagName(o)[0];a.async=1;a.src=g;m.parentNode.insertBefore(a,m)
-    })(window,document,'script','//www.google-analytics.com/analytics.js','ga');
-
-    ga('create', 'UA-73650088-1', 'auto');
-    ga('send', 'pageview');
-
-  </script>
-  <link rel="shortcut icon" type="image/x-icon" href="/images/favicon.ico">
-</head>
-
-
-  <body role="document">
-
-    <nav class="navbar navbar-default navbar-fixed-top">
-  <div class="container">
-    <div class="navbar-header">
-      <a href="/" class="navbar-brand" >
-        <img alt="Brand" style="height: 25px" src="/images/beam_logo_navbar.png">
-      </a>
-      <button type="button" class="navbar-toggle collapsed" data-toggle="collapse" data-target="#navbar" aria-expanded="false" aria-controls="navbar">
-        <span class="sr-only">Toggle navigation</span>
-        <span class="icon-bar"></span>
-        <span class="icon-bar"></span>
-        <span class="icon-bar"></span>
-      </button>
-    </div>
-    <div id="navbar" class="navbar-collapse collapse">
-      <ul class="nav navbar-nav">
-        <li class="dropdown">
-		  <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Get Started <span class="caret"></span></a>
-		  <ul class="dropdown-menu">
-			  <li><a href="/get-started/beam-overview/">Beam Overview</a></li>
-              <li><a href="/get-started/quickstart/">Quickstart</a></li>
-			  <li role="separator" class="divider"></li>
-			  <li class="dropdown-header">Example Walkthroughs</li>
-			  <li><a href="/get-started/wordcount-example/">WordCount</a></li>
-			  <li><a href="/get-started/mobile-gaming-example/">Mobile Gaming</a></li>
-              <li role="separator" class="divider"></li>
-              <li class="dropdown-header">Resources</li>
-              <li><a href="/get-started/downloads">Downloads</a></li>
-              <li><a href="/get-started/support">Support</a></li>
-		  </ul>
-	    </li>
-        <li class="dropdown">
-		  <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Documentation <span class="caret"></span></a>
-		  <ul class="dropdown-menu">
-			  <li><a href="/documentation">Using the Documentation</a></li>
-			  <li role="separator" class="divider"></li>
-			  <li class="dropdown-header">Beam Concepts</li>
-			  <li><a href="/documentation/programming-guide/">Programming Guide</a></li>
-			  <li><a href="/documentation/resources/">Additional Resources</a></li>
-			  <li role="separator" class="divider"></li>
-              <li class="dropdown-header">Pipeline Fundamentals</li>
-              <li><a href="/documentation/pipelines/design-your-pipeline/">Design Your Pipeline</a></li>
-              <li><a href="/documentation/pipelines/create-your-pipeline/">Create Your Pipeline</a></li>
-              <li><a href="/documentation/pipelines/test-your-pipeline/">Test Your Pipeline</a></li>
-              <li role="separator" class="divider"></li>
-			  <li class="dropdown-header">SDKs</li>
-			  <li><a href="/documentation/sdks/java/">Java SDK</a></li>
-			  <li><a href="/documentation/sdks/javadoc/0.4.0/" target="_blank">Java SDK API Reference <img src="/images/external-link-icon.png"
-                 width="14" height="14"
-                 alt="External link."></a>
-        </li>
-        <li><a href="/documentation/sdks/python/">Python SDK</a></li>
-			  <li role="separator" class="divider"></li>
-			  <li class="dropdown-header">Runners</li>
-			  <li><a href="/documentation/runners/capability-matrix/">Capability Matrix</a></li>
-			  <li><a href="/documentation/runners/direct/">Direct Runner</a></li>
-			  <li><a href="/documentation/runners/apex/">Apache Apex Runner</a></li>
-			  <li><a href="/documentation/runners/flink/">Apache Flink Runner</a></li>
-			  <li><a href="/documentation/runners/spark/">Apache Spark Runner</a></li>
-			  <li><a href="/documentation/runners/dataflow/">Cloud Dataflow Runner</a></li>
-		  </ul>
-	    </li>
-        <li class="dropdown">
-		  <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Contribute <span class="caret"></span></a>
-		  <ul class="dropdown-menu">
-			  <li><a href="/contribute">Get Started Contributing</a></li>
-        <li role="separator" class="divider"></li>
-        <li class="dropdown-header">Guides</li>
-			  <li><a href="/contribute/contribution-guide/">Contribution Guide</a></li>
-        <li><a href="/contribute/testing/">Testing Guide</a></li>
-        <li><a href="/contribute/release-guide/">Release Guide</a></li>
-        <li><a href="/contribute/ptransform-style-guide/">PTransform Style Guide</a></li>
-        <li role="separator" class="divider"></li>
-        <li class="dropdown-header">Technical References</li>
-        <li><a href="/contribute/design-principles/">Design Principles</a></li>
-			  <li><a href="/contribute/work-in-progress/">Ongoing Projects</a></li>
-        <li><a href="/contribute/source-repository/">Source Repository</a></li>      
-        <li role="separator" class="divider"></li>
-			  <li class="dropdown-header">Promotion</li>
-        <li><a href="/contribute/presentation-materials/">Presentation Materials</a></li>
-        <li><a href="/contribute/logos/">Logos and Design</a></li>
-        <li role="separator" class="divider"></li>
-        <li><a href="/contribute/maturity-model/">Maturity Model</a></li>
-        <li><a href="/contribute/team/">Team</a></li>
-		  </ul>
-	    </li>
-
-        <li><a href="/blog">Blog</a></li>
-      </ul>
-      <ul class="nav navbar-nav navbar-right">
-        <li class="dropdown">
-          <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false"><img src="https://www.apache.org/foundation/press/kit/feather_small.png" alt="Apache Logo" style="height:24px;">Apache Software Foundation<span class="caret"></span></a>
-          <ul class="dropdown-menu dropdown-menu-right">
-            <li><a href="http://www.apache.org/">ASF Homepage</a></li>
-            <li><a href="http://www.apache.org/licenses/">License</a></li>
-            <li><a href="http://www.apache.org/security/">Security</a></li>
-            <li><a href="http://www.apache.org/foundation/thanks.html">Thanks</a></li>
-            <li><a href="http://www.apache.org/foundation/sponsorship.html">Sponsorship</a></li>
-            <li><a href="https://www.apache.org/foundation/policies/conduct">Code of Conduct</a></li>
-          </ul>
-        </li>
-      </ul>
-    </div><!--/.nav-collapse -->
-  </div>
-</nav>
-
-
-<link rel="stylesheet" href="">
-
-
-    <div class="container" role="main">
-
-      <div class="row">
-        <h1 id="apache-beam-java-sdk-quickstart">Apache Beam Java SDK Quickstart</h1>
-
-<p>This Quickstart will walk you through executing your first Beam pipeline to run <a href="/get-started/wordcount-example">WordCount</a>, written using Beam\u2019s <a href="/documentation/sdks/java">Java SDK</a>, on a <a href="/documentation#runners">runner</a> of your choice.</p>
-
-<ul id="markdown-toc">
-  <li><a href="#set-up-your-development-environment" id="markdown-toc-set-up-your-development-environment">Set up your Development Environment</a></li>
-  <li><a href="#get-the-wordcount-code" id="markdown-toc-get-the-wordcount-code">Get the WordCount Code</a></li>
-  <li><a href="#run-wordcount" id="markdown-toc-run-wordcount">Run WordCount</a></li>
-  <li><a href="#inspect-the-results" id="markdown-toc-inspect-the-results">Inspect the results</a></li>
-  <li><a href="#next-steps" id="markdown-toc-next-steps">Next Steps</a></li>
-</ul>
-
-<h2 id="set-up-your-development-environment">Set up your Development Environment</h2>
-
-<ol>
-  <li>
-    <p>Download and install the <a href="http://www.oracle.com/technetwork/java/javase/downloads/index.html">Java Development Kit (JDK)</a> version 1.7 or later. Verify that the <a href="https://docs.oracle.com/javase/8/docs/technotes/guides/troubleshoot/envvars001.html">JAVA_HOME</a> environment variable is set and points to your JDK installation.</p>
-  </li>
-  <li>
-    <p>Download and install <a href="http://maven.apache.org/download.cgi">Apache Maven</a> by following Maven\u2019s <a href="http://maven.apache.org/install.html">installation guide</a> for your specific operating system.</p>
-  </li>
-</ol>
-
-<h2 id="get-the-wordcount-code">Get the WordCount Code</h2>
-
-<p>The easiest way to get a copy of the WordCount pipeline is to use the following command to generate a simple Maven project that contains Beam\u2019s WordCount examples and builds against the most recent Beam release:</p>
-
-<div class="highlighter-rouge"><pre class="highlight"><code>$ mvn archetype:generate \
-      -DarchetypeRepository=https://repository.apache.org/content/groups/snapshots \
-      -DarchetypeGroupId=org.apache.beam \
-      -DarchetypeArtifactId=beam-sdks-java-maven-archetypes-examples \
-      -DarchetypeVersion=LATEST \
-      -DgroupId=org.example \
-      -DartifactId=word-count-beam \
-      -Dversion="0.1" \
-      -Dpackage=org.apache.beam.examples \
-      -DinteractiveMode=false
-</code></pre>
-</div>
-
-<p>This will create a directory <code class="highlighter-rouge">word-count-beam</code> that contains a simple <code class="highlighter-rouge">pom.xml</code> and a series of example pipelines that count words in text files.</p>
-
-<div class="highlighter-rouge"><pre class="highlight"><code>$ cd word-count-beam/
-
-$ ls
-pom.xml	src
-
-$ ls src/main/java/org/apache/beam/examples/
-DebuggingWordCount.java	WindowedWordCount.java	common
-MinimalWordCount.java	WordCount.java
-</code></pre>
-</div>
-
-<p>For a detailed introduction to the Beam concepts used in these examples, see the <a href="/get-started/wordcount-example">WordCount Example Walkthrough</a>. Here, we\u2019ll just focus on executing <code class="highlighter-rouge">WordCount.java</code>.</p>
-
-<h2 id="run-wordcount">Run WordCount</h2>
-
-<p>A single Beam pipeline can run on multiple Beam <a href="/documentation#runners">runners</a>, including the <a href="/documentation/runners/apex">ApexRunner</a>, <a href="/documentation/runners/flink">FlinkRunner</a>, <a href="/documentation/runners/spark">SparkRunner</a> or <a href="/documentation/runners/dataflow">DataflowRunner</a>. The <a href="/documentation/runners/direct">DirectRunner</a> is a common runner for getting started, as it runs locally on your machine and requires no specific setup.</p>
-
-<p>After you\u2019ve chosen which runner you\u2019d like to use:</p>
-
-<ol>
-  <li>Ensure you\u2019ve done any runner-specific setup.</li>
-  <li>Build your commandline by:
-    <ol>
-      <li>Specifying a specific runner with <code class="highlighter-rouge">--runner=&lt;runner&gt;</code> (defaults to the <a href="/documentation/runners/direct">DirectRunner</a>)</li>
-      <li>Adding any runner-specific required options</li>
-      <li>Choosing input files and an output location are accessible on the chosen runner. (For example, you can\u2019t access a local file if you are running the pipeline on an external cluster.)</li>
-    </ol>
-  </li>
-  <li>Run your first WordCount pipeline.</li>
-</ol>
-
-<div class="runner-direct highlighter-rouge"><pre class="highlight"><code>$ mvn compile exec:java -Dexec.mainClass=org.apache.beam.examples.WordCount \
-     -Dexec.args="--inputFile=pom.xml --output=counts" -Pdirect-runner
-</code></pre>
-</div>
-
-<div class="runner-apex highlighter-rouge"><pre class="highlight"><code>$ mvn compile exec:java -Dexec.mainClass=org.apache.beam.examples.WordCount \
-     -Dexec.args="--inputFile=pom.xml --output=counts --runner=ApexRunner" -Papex-runner
-</code></pre>
-</div>
-
-<div class="runner-flink-local highlighter-rouge"><pre class="highlight"><code>$ mvn compile exec:java -Dexec.mainClass=org.apache.beam.examples.WordCount \
-     -Dexec.args="--runner=FlinkRunner --inputFile=pom.xml --output=counts" -Pflink-runner
-</code></pre>
-</div>
-
-<div class="runner-flink-cluster highlighter-rouge"><pre class="highlight"><code>$ mvn package exec:java -Dexec.mainClass=org.apache.beam.examples.WordCount \
-     -Dexec.args="--runner=FlinkRunner --flinkMaster=&lt;flink master&gt; --filesToStage=target/word-count-beam-bundled-0.1.jar \
-                  --inputFile=/path/to/quickstart/pom.xml --output=/tmp/counts" -Pflink-runner
-
-You can monitor the running job by visiting the Flink dashboard at http://&lt;flink master&gt;:8081
-</code></pre>
-</div>
-
-<div class="runner-spark highlighter-rouge"><pre class="highlight"><code>$ mvn compile exec:java -Dexec.mainClass=org.apache.beam.examples.WordCount \
-     -Dexec.args="--runner=SparkRunner --inputFile=pom.xml --output=counts" -Pspark-runner
-</code></pre>
-</div>
-
-<div class="runner-dataflow highlighter-rouge"><pre class="highlight"><code>$ mvn compile exec:java -Dexec.mainClass=org.apache.beam.examples.WordCount \
-	 -Dexec.args="--runner=DataflowRunner --gcpTempLocation=gs://&lt;your-gcs-bucket&gt;/tmp \
-	              --inputFile=gs://apache-beam-samples/shakespeare/* --output=gs://&lt;your-gcs-bucket&gt;/counts" \
-     -Pdataflow-runner
-</code></pre>
-</div>
-
-<h2 id="inspect-the-results">Inspect the results</h2>
-
-<p>Once the pipeline has completed, you can view the output. You\u2019ll notice that there may be multiple output files prefixed by <code class="highlighter-rouge">count</code>. The exact number of these files is decided by the runner, giving it the flexibility to do efficient, distributed execution.</p>
-
-<div class="runner-direct highlighter-rouge"><pre class="highlight"><code>$ ls counts*
-</code></pre>
-</div>
-
-<div class="runner-apex highlighter-rouge"><pre class="highlight"><code>$ ls counts*
-</code></pre>
-</div>
-
-<div class="runner-flink-local highlighter-rouge"><pre class="highlight"><code>$ ls counts*
-</code></pre>
-</div>
-
-<div class="runner-flink-cluster highlighter-rouge"><pre class="highlight"><code>$ ls /tmp/counts*
-</code></pre>
-</div>
-
-<div class="runner-spark highlighter-rouge"><pre class="highlight"><code>$ ls counts*
-</code></pre>
-</div>
-
-<div class="runner-dataflow highlighter-rouge"><pre class="highlight"><code>$ gsutil ls gs://&lt;your-gcs-bucket&gt;/counts*
-</code></pre>
-</div>
-
-<p>When you look into the contents of the file, you\u2019ll see that they contain unique words and the number of occurrences of each word. The order of elements within the file may differ because the Beam model does not generally guarantee ordering, again to allow runners to optimize for efficiency.</p>
-
-<div class="runner-direct highlighter-rouge"><pre class="highlight"><code>$ more counts*
-api: 9
-bundled: 1
-old: 4
-Apache: 2
-The: 1
-limitations: 1
-Foundation: 1
-...
-</code></pre>
-</div>
-
-<div class="runner-apex highlighter-rouge"><pre class="highlight"><code>$ cat counts*
-BEAM: 1
-have: 1
-simple: 1
-skip: 4
-PAssert: 1
-...
-</code></pre>
-</div>
-
-<div class="runner-flink-local highlighter-rouge"><pre class="highlight"><code>$ more counts*
-The: 1
-api: 9
-old: 4
-Apache: 2
-limitations: 1
-bundled: 1
-Foundation: 1
-...
-</code></pre>
-</div>
-
-<div class="runner-flink-cluster highlighter-rouge"><pre class="highlight"><code>$ more /tmp/counts*
-The: 1
-api: 9
-old: 4
-Apache: 2
-limitations: 1
-bundled: 1
-Foundation: 1
-...
-</code></pre>
-</div>
-
-<div class="runner-spark highlighter-rouge"><pre class="highlight"><code>$ more counts*
-beam: 27
-SF: 1
-fat: 1
-job: 1
-limitations: 1
-require: 1
-of: 11
-profile: 10
-...
-</code></pre>
-</div>
-
-<div class="runner-dataflow highlighter-rouge"><pre class="highlight"><code>$ gsutil cat gs://&lt;your-gcs-bucket&gt;/counts*
-feature: 15
-smother'st: 1
-revelry: 1
-bashfulness: 1
-Bashful: 1
-Below: 2
-deserves: 32
-barrenly: 1
-...
-</code></pre>
-</div>
-
-<h2 id="next-steps">Next Steps</h2>
-
-<ul>
-  <li>Learn more about these WordCount examples in the <a href="/get-started/wordcount-example">WordCount Example Walkthrough</a>.</li>
-  <li>Dive in to some of our favorite <a href="/documentation/resources">articles and presentations</a>.</li>
-  <li>Join the Beam <a href="/get-started/support#mailing-lists">users@</a> mailing list.</li>
-</ul>
-
-<p>Please don\u2019t hesitate to <a href="/get-started/support">reach out</a> if you encounter any issues!</p>
-
-
-      </div>
-
-
-    <hr>
-  <div class="row">
-      <div class="col-xs-12">
-          <footer>
-              <p class="text-center">
-                &copy; Copyright
-                <a href="http://www.apache.org">The Apache Software Foundation</a>,
-                2017. All Rights Reserved.
-              </p>
-              <p class="text-center">
-                <a href="/privacy_policy">Privacy Policy</a> |
-                <a href="/feed.xml">RSS Feed</a>
-              </p>
-          </footer>
-      </div>
-  </div>
-  <!-- container div end -->
-</div>
-
-
-  </body>
-
+<html lang="en-US">
+<meta charset="utf-8">
+<title>Redirecting\u2026</title>
+<link rel="canonical" href="http://beam.apache.org/get-started/quickstart-java/">
+<meta http-equiv="refresh" content="0; url=http://beam.apache.org/get-started/quickstart-java/">
+<h1>Redirecting\u2026</h1>
+<a href="http://beam.apache.org/get-started/quickstart-java/">Click here if you are not redirected.</a>
+<script>location="http://beam.apache.org/get-started/quickstart-java/"</script>
 </html>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/9727633e/content/get-started/support/index.html
----------------------------------------------------------------------
diff --git a/content/get-started/support/index.html b/content/get-started/support/index.html
index d5c9a09..53cba70 100644
--- a/content/get-started/support/index.html
+++ b/content/get-started/support/index.html
@@ -52,7 +52,8 @@
 		  <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Get Started <span class="caret"></span></a>
 		  <ul class="dropdown-menu">
 			  <li><a href="/get-started/beam-overview/">Beam Overview</a></li>
-              <li><a href="/get-started/quickstart/">Quickstart</a></li>
+        <li><a href="/get-started/quickstart-java/">Quickstart - Java</a></li>
+        <li><a href="/get-started/quickstart-py/">Quickstart - Python</a></li>
 			  <li role="separator" class="divider"></li>
 			  <li class="dropdown-header">Example Walkthroughs</li>
 			  <li><a href="/get-started/wordcount-example/">WordCount</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/9727633e/content/get-started/wordcount-example/index.html
----------------------------------------------------------------------
diff --git a/content/get-started/wordcount-example/index.html b/content/get-started/wordcount-example/index.html
index 1c9820b..3809656 100644
--- a/content/get-started/wordcount-example/index.html
+++ b/content/get-started/wordcount-example/index.html
@@ -52,7 +52,8 @@
 		  <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Get Started <span class="caret"></span></a>
 		  <ul class="dropdown-menu">
 			  <li><a href="/get-started/beam-overview/">Beam Overview</a></li>
-              <li><a href="/get-started/quickstart/">Quickstart</a></li>
+        <li><a href="/get-started/quickstart-java/">Quickstart - Java</a></li>
+        <li><a href="/get-started/quickstart-py/">Quickstart - Python</a></li>
 			  <li role="separator" class="divider"></li>
 			  <li class="dropdown-header">Example Walkthroughs</li>
 			  <li><a href="/get-started/wordcount-example/">WordCount</a></li>
@@ -181,6 +182,14 @@
   </li>
 </ul>
 
+<nav class="language-switcher">
+  <strong>Adapt for:</strong> 
+  <ul>
+    <li data-type="language-java">Java SDK</li>
+    <li data-type="language-py">Python SDK</li>
+  </ul>
+</nav>
+
 <p>The WordCount examples demonstrate how to set up a processing pipeline that can read text, tokenize the text lines into individual words, and perform a frequency count on each of those words. The Beam SDKs contain a series of these four successively more detailed WordCount examples that build on each other. The input text for all the examples is a set of Shakespeare\u2019s texts.</p>
 
 <p>Each WordCount example introduces different concepts in the Beam programming model. Begin by understanding Minimal WordCount, the simplest of the examples. Once you feel comfortable with the basic principles in building a pipeline, continue on to learn more concepts in the other examples.</p>
@@ -235,12 +244,26 @@
 </code></pre>
 </div>
 
+<div class="language-py highlighter-rouge"><pre class="highlight"><code><span class="n">options</span> <span class="o">=</span> <span class="n">PipelineOptions</span><span class="p">()</span>
+<span class="n">google_cloud_options</span> <span class="o">=</span> <span class="n">options</span><span class="o">.</span><span class="n">view_as</span><span class="p">(</span><span class="n">GoogleCloudOptions</span><span class="p">)</span>
+<span class="n">google_cloud_options</span><span class="o">.</span><span class="n">project</span> <span class="o">=</span> <span class="s">'my-project-id'</span>
+<span class="n">google_cloud_options</span><span class="o">.</span><span class="n">job_name</span> <span class="o">=</span> <span class="s">'myjob'</span>
+<span class="n">google_cloud_options</span><span class="o">.</span><span class="n">staging_location</span> <span class="o">=</span> <span class="s">'gs://your-bucket-name-here/staging'</span>
+<span class="n">google_cloud_options</span><span class="o">.</span><span class="n">temp_location</span> <span class="o">=</span> <span class="s">'gs://your-bucket-name-here/temp'</span>
+<span class="n">options</span><span class="o">.</span><span class="n">view_as</span><span class="p">(</span><span class="n">StandardOptions</span><span class="p">)</span><span class="o">.</span><span class="n">runner</span> <span class="o">=</span> <span class="s">'DataflowRunner'</span>
+</code></pre>
+</div>
+
 <p>The next step is to create a Pipeline object with the options we\u2019ve just constructed. The Pipeline object builds up the graph of transformations to be executed, associated with that particular pipeline.</p>
 
 <div class="language-java highlighter-rouge"><pre class="highlight"><code><span class="n">Pipeline</span> <span class="n">p</span> <span class="o">=</span> <span class="n">Pipeline</span><span class="o">.</span><span class="na">create</span><span class="o">(</span><span class="n">options</span><span class="o">);</span>
 </code></pre>
 </div>
 
+<div class="language-py highlighter-rouge"><pre class="highlight"><code><span class="n">p</span> <span class="o">=</span> <span class="n">beam</span><span class="o">.</span><span class="n">Pipeline</span><span class="p">(</span><span class="n">options</span><span class="o">=</span><span class="n">options</span><span class="p">)</span>
+</code></pre>
+</div>
+
 <h3 id="applying-pipeline-transforms">Applying Pipeline Transforms</h3>
 
 <p>The Minimal WordCount pipeline contains several transforms to read data into the pipeline, manipulate or otherwise transform the data, and write out the results. Each transform represents an operation in the pipeline.</p>
@@ -256,23 +279,33 @@ Figure 1: The pipeline data flow.</p>
   <li>
     <p>A text file <code class="highlighter-rouge">Read</code> transform is applied to the Pipeline object itself, and produces a <code class="highlighter-rouge">PCollection</code> as output. Each element in the output PCollection represents one line of text from the input file. This example happens to use input data stored in a publicly accessible Google Cloud Storage bucket (\u201cgs://\u201d).</p>
 
-    <div class="language-java highlighter-rouge"><pre class="highlight"><code>        <span class="n">p</span><span class="o">.</span><span class="na">apply</span><span class="o">(</span><span class="n">TextIO</span><span class="o">.</span><span class="na">Read</span><span class="o">.</span><span class="na">from</span><span class="o">(</span><span class="s">"gs://apache-beam-samples/shakespeare/*"</span><span class="o">))</span>
+    <div class="language-java highlighter-rouge"><pre class="highlight"><code><span class="n">p</span><span class="o">.</span><span class="na">apply</span><span class="o">(</span><span class="n">TextIO</span><span class="o">.</span><span class="na">Read</span><span class="o">.</span><span class="na">from</span><span class="o">(</span><span class="s">"gs://apache-beam-samples/shakespeare/*"</span><span class="o">))</span>
+</code></pre>
+    </div>
+
+    <div class="language-py highlighter-rouge"><pre class="highlight"><code><span class="n">p</span> <span class="o">|</span> <span class="n">beam</span><span class="o">.</span><span class="n">io</span><span class="o">.</span><span class="n">ReadFromText</span><span class="p">(</span>
+<span class="s">'gs://dataflow-samples/shakespeare/kinglear.txt'</span><span class="p">)</span>
 </code></pre>
     </div>
   </li>
   <li>
     <p>A <a href="/documentation/programming-guide/#transforms-pardo">ParDo</a> transform that invokes a <code class="highlighter-rouge">DoFn</code> (defined in-line as an anonymous class) on each element that tokenizes the text lines into individual words. The input for this transform is the <code class="highlighter-rouge">PCollection</code> of text lines generated by the previous <code class="highlighter-rouge">TextIO.Read</code> transform. The <code class="highlighter-rouge">ParDo</code> transform outputs a new <code class="highlighter-rouge">PCollection</code>, where each element represents an individual word in the text.</p>
 
-    <div class="language-java highlighter-rouge"><pre class="highlight"><code>        <span class="o">.</span><span class="na">apply</span><span class="o">(</span><span class="s">"ExtractWords"</span><span class="o">,</span> <span class="n">ParDo</span><span class="o">.</span><span class="na">of</span><span class="o">(</span><span class="k">new</span> <span class="n">DoFn</span><span class="o">&lt;</span><span class="n">String</span><span class="o">,</span> <span class="n">String</span><span class="o">&gt;()</span> <span class="o">{</span>
-            <span class="nd">@ProcessElement</span>
-            <span class="kd">public</span> <span class="kt">void</span> <span class="nf">processElement</span><span class="o">(</span><span class="n">ProcessContext</span> <span class="n">c</span><span class="o">)</span> <span class="o">{</span>
-                <span class="k">for</span> <span class="o">(</span><span class="n">String</span> <span class="n">word</span> <span class="o">:</span> <span class="n">c</span><span class="o">.</span><span class="na">element</span><span class="o">().</span><span class="na">split</span><span class="o">(</span><span class="s">"[^a-zA-Z']+"</span><span class="o">))</span> <span class="o">{</span>
-                    <span class="k">if</span> <span class="o">(!</span><span class="n">word</span><span class="o">.</span><span class="na">isEmpty</span><span class="o">())</span> <span class="o">{</span>
-                        <span class="n">c</span><span class="o">.</span><span class="na">output</span><span class="o">(</span><span class="n">word</span><span class="o">);</span>
-                    <span class="o">}</span>
-                <span class="o">}</span>
+    <div class="language-java highlighter-rouge"><pre class="highlight"><code><span class="o">.</span><span class="na">apply</span><span class="o">(</span><span class="s">"ExtractWords"</span><span class="o">,</span> <span class="n">ParDo</span><span class="o">.</span><span class="na">of</span><span class="o">(</span><span class="k">new</span> <span class="n">DoFn</span><span class="o">&lt;</span><span class="n">String</span><span class="o">,</span> <span class="n">String</span><span class="o">&gt;()</span> <span class="o">{</span>
+    <span class="nd">@ProcessElement</span>
+    <span class="kd">public</span> <span class="kt">void</span> <span class="nf">processElement</span><span class="o">(</span><span class="n">ProcessContext</span> <span class="n">c</span><span class="o">)</span> <span class="o">{</span>
+        <span class="k">for</span> <span class="o">(</span><span class="n">String</span> <span class="n">word</span> <span class="o">:</span> <span class="n">c</span><span class="o">.</span><span class="na">element</span><span class="o">().</span><span class="na">split</span><span class="o">(</span><span class="s">"[^a-zA-Z']+"</span><span class="o">))</span> <span class="o">{</span>
+            <span class="k">if</span> <span class="o">(!</span><span class="n">word</span><span class="o">.</span><span class="na">isEmpty</span><span class="o">())</span> <span class="o">{</span>
+                <span class="n">c</span><span class="o">.</span><span class="na">output</span><span class="o">(</span><span class="n">word</span><span class="o">);</span>
             <span class="o">}</span>
-        <span class="o">}))</span>
+        <span class="o">}</span>
+    <span class="o">}</span>
+<span class="o">}))</span>
+</code></pre>
+    </div>
+
+    <div class="language-py highlighter-rouge"><pre class="highlight"><code><span class="c"># The Flatmap transform is a simplified version of ParDo.</span>
+<span class="o">|</span> <span class="s">'ExtractWords'</span> <span class="o">&gt;&gt;</span> <span class="n">beam</span><span class="o">.</span><span class="n">FlatMap</span><span class="p">(</span><span class="k">lambda</span> <span class="n">x</span><span class="p">:</span> <span class="n">re</span><span class="o">.</span><span class="n">findall</span><span class="p">(</span><span class="s">r'[A-Za-z</span><span class="se">\'</span><span class="s">]+'</span><span class="p">,</span> <span class="n">x</span><span class="p">))</span>
 </code></pre>
     </div>
   </li>
@@ -281,28 +314,40 @@ Figure 1: The pipeline data flow.</p>
 
     <p>In this pipeline, the input for <code class="highlighter-rouge">Count</code> is the <code class="highlighter-rouge">PCollection</code> of individual words generated by the previous <code class="highlighter-rouge">ParDo</code>, and the output is a <code class="highlighter-rouge">PCollection</code> of key/value pairs where each key represents a unique word in the text and the associated value is the occurrence count for each.</p>
 
-    <div class="language-java highlighter-rouge"><pre class="highlight"><code>        <span class="o">.</span><span class="na">apply</span><span class="o">(</span><span class="n">Count</span><span class="o">.&lt;</span><span class="n">String</span><span class="o">&gt;</span><span class="n">perElement</span><span class="o">())</span>
+    <div class="language-java highlighter-rouge"><pre class="highlight"><code><span class="o">.</span><span class="na">apply</span><span class="o">(</span><span class="n">Count</span><span class="o">.&lt;</span><span class="n">String</span><span class="o">&gt;</span><span class="n">perElement</span><span class="o">())</span>
+</code></pre>
+    </div>
+
+    <div class="language-py highlighter-rouge"><pre class="highlight"><code><span class="o">|</span> <span class="n">beam</span><span class="o">.</span><span class="n">combiners</span><span class="o">.</span><span class="n">Count</span><span class="o">.</span><span class="n">PerElement</span><span class="p">()</span>
 </code></pre>
     </div>
   </li>
   <li>
     <p>The next transform formats each of the key/value pairs of unique words and occurrence counts into a printable string suitable for writing to an output file.</p>
 
-    <p><code class="highlighter-rouge">MapElements</code> is a higher-level composite transform that encapsulates a simple <code class="highlighter-rouge">ParDo</code>; for each element in the input <code class="highlighter-rouge">PCollection</code>, <code class="highlighter-rouge">MapElements</code> applies a function that produces exactly one output element. In this example, <code class="highlighter-rouge">MapElements</code> invokes a <code class="highlighter-rouge">SimpleFunction</code> (defined in-line as an anonymous class) that does the formatting. As input, <code class="highlighter-rouge">MapElements</code> takes a <code class="highlighter-rouge">PCollection</code> of key/value pairs generated by <code class="highlighter-rouge">Count</code>, and produces a new <code class="highlighter-rouge">PCollection</code> of printable strings.</p>
+    <p>The map transform is a higher-level composite transform that encapsulates a simple <code class="highlighter-rouge">ParDo</code>; for each element in the input <code class="highlighter-rouge">PCollection</code>, the map transform applies a function that produces exactly one output element.</p>
 
-    <div class="language-java highlighter-rouge"><pre class="highlight"><code>        <span class="o">.</span><span class="na">apply</span><span class="o">(</span><span class="s">"FormatResults"</span><span class="o">,</span> <span class="n">MapElements</span><span class="o">.</span><span class="na">via</span><span class="o">(</span><span class="k">new</span> <span class="n">SimpleFunction</span><span class="o">&lt;</span><span class="n">KV</span><span class="o">&lt;</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">&gt;,</span> <span class="n">String</span><span class="o">&gt;()</span> <span class="o">{</span>
-            <span class="nd">@Override</span>
-            <span class="kd">public</span> <span class="n">String</span> <span class="nf">apply</span><span class="o">(</span><span class="n">KV</span><span class="o">&lt;</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">&gt;</span> <span class="n">input</span><span class="o">)</span> <span class="o">{</span>
-                <span class="k">return</span> <span class="n">input</span><span class="o">.</span><span class="na">getKey</span><span class="o">()</span> <span class="o">+</span> <span class="s">": "</span> <span class="o">+</span> <span class="n">input</span><span class="o">.</span><span class="na">getValue</span><span class="o">();</span>
-            <span class="o">}</span>
-        <span class="o">}))</span>
+    <div class="language-java highlighter-rouge"><pre class="highlight"><code><span class="o">.</span><span class="na">apply</span><span class="o">(</span><span class="s">"FormatResults"</span><span class="o">,</span> <span class="n">MapElements</span><span class="o">.</span><span class="na">via</span><span class="o">(</span><span class="k">new</span> <span class="n">SimpleFunction</span><span class="o">&lt;</span><span class="n">KV</span><span class="o">&lt;</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">&gt;,</span> <span class="n">String</span><span class="o">&gt;()</span> <span class="o">{</span>
+    <span class="nd">@Override</span>
+    <span class="kd">public</span> <span class="n">String</span> <span class="nf">apply</span><span class="o">(</span><span class="n">KV</span><span class="o">&lt;</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">&gt;</span> <span class="n">input</span><span class="o">)</span> <span class="o">{</span>
+        <span class="k">return</span> <span class="n">input</span><span class="o">.</span><span class="na">getKey</span><span class="o">()</span> <span class="o">+</span> <span class="s">": "</span> <span class="o">+</span> <span class="n">input</span><span class="o">.</span><span class="na">getValue</span><span class="o">();</span>
+    <span class="o">}</span>
+<span class="o">}))</span>
+</code></pre>
+    </div>
+
+    <div class="language-py highlighter-rouge"><pre class="highlight"><code><span class="o">|</span> <span class="n">beam</span><span class="o">.</span><span class="n">Map</span><span class="p">(</span><span class="k">lambda</span> <span class="p">(</span><span class="n">word</span><span class="p">,</span> <span class="n">count</span><span class="p">):</span> <span class="s">'</span><span class="si">%</span><span class="s">s: </span><span class="si">%</span><span class="s">s'</span> <span class="o">%</span> <span class="p">(</span><span class="n">word</span><span class="p">,</span> <span class="n">count</span><span class="p">))</span>
 </code></pre>
     </div>
   </li>
   <li>
-    <p>A text file <code class="highlighter-rouge">Write</code>. This transform takes the final <code class="highlighter-rouge">PCollection</code> of formatted Strings as input and writes each element to an output text file. Each element in the input <code class="highlighter-rouge">PCollection</code> represents one line of text in the resulting output file.</p>
+    <p>A text file write transform. This transform takes the final <code class="highlighter-rouge">PCollection</code> of formatted Strings as input and writes each element to an output text file. Each element in the input <code class="highlighter-rouge">PCollection</code> represents one line of text in the resulting output file.</p>
 
-    <div class="language-java highlighter-rouge"><pre class="highlight"><code>        <span class="o">.</span><span class="na">apply</span><span class="o">(</span><span class="n">TextIO</span><span class="o">.</span><span class="na">Write</span><span class="o">.</span><span class="na">to</span><span class="o">(</span><span class="s">"wordcounts"</span><span class="o">));</span>
+    <div class="language-java highlighter-rouge"><pre class="highlight"><code><span class="o">.</span><span class="na">apply</span><span class="o">(</span><span class="n">TextIO</span><span class="o">.</span><span class="na">Write</span><span class="o">.</span><span class="na">to</span><span class="o">(</span><span class="s">"wordcounts"</span><span class="o">));</span>
+</code></pre>
+    </div>
+
+    <div class="language-py highlighter-rouge"><pre class="highlight"><code><span class="o">|</span> <span class="n">beam</span><span class="o">.</span><span class="n">io</span><span class="o">.</span><span class="n">WriteToText</span><span class="p">(</span><span class="s">'gs://my-bucket/counts.txt'</span><span class="p">)</span>
 </code></pre>
     </div>
   </li>
@@ -318,6 +363,10 @@ Figure 1: The pipeline data flow.</p>
 </code></pre>
 </div>
 
+<div class="language-py highlighter-rouge"><pre class="highlight"><code><span class="n">result</span> <span class="o">=</span> <span class="n">p</span><span class="o">.</span><span class="n">run</span><span class="p">()</span>
+</code></pre>
+</div>
+
 <p>Note that the <code class="highlighter-rouge">run</code> method is asynchronous. For a blocking execution instead, run your pipeline appending the <code class="highlighter-rouge">waitUntilFinish</code> method.</p>
 
 <h2 id="wordcount-example">WordCount Example</h2>
@@ -342,9 +391,9 @@ Figure 1: The pipeline data flow.</p>
 
 <p>When using <code class="highlighter-rouge">ParDo</code> transforms, you need to specify the processing operation that gets applied to each element in the input <code class="highlighter-rouge">PCollection</code>. This processing operation is a subclass of the SDK class <code class="highlighter-rouge">DoFn</code>. You can create the <code class="highlighter-rouge">DoFn</code> subclasses for each <code class="highlighter-rouge">ParDo</code> inline, as an anonymous inner class instance, as is done in the previous example (Minimal WordCount). However, it\u2019s often a good idea to define the <code class="highlighter-rouge">DoFn</code> at the global level, which makes it easier to unit test and can make the <code class="highlighter-rouge">ParDo</code> code more readable.</p>
 
-<p>In this example, <code class="highlighter-rouge">ExtractWordsFn</code> is a <code class="highlighter-rouge">DoFn</code> that is defined as a static class:</p>
+<div class="language-java highlighter-rouge"><pre class="highlight"><code><span class="c1">// In this example, ExtractWordsFn is a DoFn that is defined as a static class:</span>
 
-<div class="language-java highlighter-rouge"><pre class="highlight"><code><span class="kd">static</span> <span class="kd">class</span> <span class="nc">ExtractWordsFn</span> <span class="kd">extends</span> <span class="n">DoFn</span><span class="o">&lt;</span><span class="n">String</span><span class="o">,</span> <span class="n">String</span><span class="o">&gt;</span> <span class="o">{</span>
+<span class="kd">static</span> <span class="kd">class</span> <span class="nc">ExtractWordsFn</span> <span class="kd">extends</span> <span class="n">DoFn</span><span class="o">&lt;</span><span class="n">String</span><span class="o">,</span> <span class="n">String</span><span class="o">&gt;</span> <span class="o">{</span>
     <span class="o">...</span>
 
     <span class="nd">@ProcessElement</span>
@@ -355,9 +404,15 @@ Figure 1: The pipeline data flow.</p>
 </code></pre>
 </div>
 
-<p>This <code class="highlighter-rouge">DoFn</code> (<code class="highlighter-rouge">ExtractWordsFn</code>) is the processing operation that <code class="highlighter-rouge">ParDo</code> applies to the <code class="highlighter-rouge">PCollection</code> of words:</p>
+<div class="language-py highlighter-rouge"><pre class="highlight"><code><span class="c"># In this example, the DoFns are defined as classes:</span>
+
+<span class="k">class</span> <span class="nc">FormatAsTextFn</span><span class="p">(</span><span class="n">beam</span><span class="o">.</span><span class="n">DoFn</span><span class="p">):</span>
 
-<div class="language-java highlighter-rouge"><pre class="highlight"><code><span class="n">PCollection</span><span class="o">&lt;</span><span class="n">String</span><span class="o">&gt;</span> <span class="n">words</span> <span class="o">=</span> <span class="n">lines</span><span class="o">.</span><span class="na">apply</span><span class="o">(</span><span class="n">ParDo</span><span class="o">.</span><span class="na">of</span><span class="o">(</span><span class="k">new</span> <span class="n">ExtractWordsFn</span><span class="o">()));</span>
+  <span class="k">def</span> <span class="nf">process</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
+    <span class="n">word</span><span class="p">,</span> <span class="n">count</span> <span class="o">=</span> <span class="n">context</span><span class="o">.</span><span class="n">element</span>
+    <span class="k">yield</span> <span class="s">'</span><span class="si">%</span><span class="s">s: </span><span class="si">%</span><span class="s">s'</span> <span class="o">%</span> <span class="p">(</span><span class="n">word</span><span class="p">,</span> <span class="n">count</span><span class="p">)</span>
+
+<span class="n">formatted</span> <span class="o">=</span> <span class="n">counts</span> <span class="o">|</span> <span class="n">beam</span><span class="o">.</span><span class="n">ParDo</span><span class="p">(</span><span class="n">FormatAsTextFn</span><span class="p">())</span>
 </code></pre>
 </div>
 
@@ -396,6 +451,21 @@ Figure 1: The pipeline data flow.</p>
 </code></pre>
 </div>
 
+<div class="language-py highlighter-rouge"><pre class="highlight"><code><span class="k">class</span> <span class="nc">CountWords</span><span class="p">(</span><span class="n">beam</span><span class="o">.</span><span class="n">PTransform</span><span class="p">):</span>
+
+  <span class="k">def</span> <span class="nf">expand</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pcoll</span><span class="p">):</span>
+    <span class="k">return</span> <span class="p">(</span><span class="n">pcoll</span>
+            <span class="c"># Convert lines of text into individual words.</span>
+            <span class="o">|</span> <span class="n">beam</span><span class="o">.</span><span class="n">FlatMap</span><span class="p">(</span>
+                <span class="s">'ExtractWords'</span><span class="p">,</span> <span class="k">lambda</span> <span class="n">x</span><span class="p">:</span> <span class="n">re</span><span class="o">.</span><span class="n">findall</span><span class="p">(</span><span class="s">r'[A-Za-z</span><span class="se">\'</span><span class="s">]+'</span><span class="p">,</span> <span class="n">x</span><span class="p">))</span>
+
+            <span class="c"># Count the number of times each word occurs.</span>
+            <span class="o">|</span> <span class="n">beam</span><span class="o">.</span><span class="n">combiners</span><span class="o">.</span><span class="n">Count</span><span class="o">.</span><span class="n">PerElement</span><span class="p">())</span>
+
+<span class="n">counts</span> <span class="o">=</span> <span class="n">lines</span> <span class="o">|</span> <span class="n">CountWords</span><span class="p">()</span>
+</code></pre>
+</div>
+
 <h3 id="using-parameterizable-pipelineoptions">Using Parameterizable PipelineOptions</h3>
 
 <p>You can hard-code various execution options when you run your pipeline. However, the more common way is to define your own configuration options via command-line argument parsing. Defining your configuration options via the command-line makes the code more easily portable across different runners.</p>
@@ -419,6 +489,19 @@ Figure 1: The pipeline data flow.</p>
 </code></pre>
 </div>
 
+<div class="language-py highlighter-rouge"><pre class="highlight"><code><span class="k">class</span> <span class="nc">WordCountOptions</span><span class="p">(</span><span class="n">PipelineOptions</span><span class="p">):</span>
+
+  <span class="nd">@classmethod</span>
+  <span class="k">def</span> <span class="nf">_add_argparse_args</span><span class="p">(</span><span class="n">cls</span><span class="p">,</span> <span class="n">parser</span><span class="p">):</span>
+    <span class="n">parser</span><span class="o">.</span><span class="n">add_argument</span><span class="p">(</span><span class="s">'--input'</span><span class="p">,</span>
+                        <span class="n">help</span><span class="o">=</span><span class="s">'Input for the pipeline'</span><span class="p">,</span>
+                        <span class="n">default</span><span class="o">=</span><span class="s">'gs://my-bucket/input'</span><span class="p">)</span>
+
+<span class="n">options</span> <span class="o">=</span> <span class="n">PipelineOptions</span><span class="p">(</span><span class="n">argv</span><span class="p">)</span>
+<span class="n">p</span> <span class="o">=</span> <span class="n">beam</span><span class="o">.</span><span class="n">Pipeline</span><span class="p">(</span><span class="n">options</span><span class="o">=</span><span class="n">options</span><span class="p">)</span>
+</code></pre>
+</div>
+
 <h2 id="debugging-wordcount-example">Debugging WordCount Example</h2>
 
 <p>The Debugging WordCount example demonstrates some best practices for instrumenting your pipeline code.</p>
@@ -446,9 +529,9 @@ Figure 1: The pipeline data flow.</p>
 
 <p>If you execute your pipeline using <code class="highlighter-rouge">DataflowRunner</code>, you can use Google Cloud Logging. Google Cloud Logging (currently in beta) aggregates the logs from all of your Dataflow job\u2019s workers to a single location in the Google Cloud Platform Console. You can use Cloud Logging to search and access the logs from all of the Compute Engine instances that Dataflow has spun up to complete your Dataflow job. You can add logging statements into your pipeline\u2019s <code class="highlighter-rouge">DoFn</code> instances that will appear in Cloud Logging as your pipeline runs.</p>
 
-<p>In this example, we use <code class="highlighter-rouge">.trace</code> and <code class="highlighter-rouge">.debug</code>:</p>
+<div class="language-java highlighter-rouge"><pre class="highlight"><code><span class="c1">// This example uses .trace and .debug:</span>
 
-<div class="language-java highlighter-rouge"><pre class="highlight"><code><span class="kd">public</span> <span class="kd">class</span> <span class="nc">DebuggingWordCount</span> <span class="o">{</span>
+<span class="kd">public</span> <span class="kd">class</span> <span class="nc">DebuggingWordCount</span> <span class="o">{</span>
 
   <span class="kd">public</span> <span class="kd">static</span> <span class="kd">class</span> <span class="nc">FilterTextFn</span> <span class="kd">extends</span> <span class="n">DoFn</span><span class="o">&lt;</span><span class="n">KV</span><span class="o">&lt;</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">&gt;,</span> <span class="n">KV</span><span class="o">&lt;</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">&gt;&gt;</span> <span class="o">{</span>
     <span class="o">...</span>
@@ -467,6 +550,44 @@ Figure 1: The pipeline data flow.</p>
 </code></pre>
 </div>
 
+<div class="language-py highlighter-rouge"><pre class="highlight"><code><span class="c"># [START example_wordcount_debugging_aggregators]</span>
+<span class="kn">import</span> <span class="nn">logging</span>
+
+<span class="k">class</span> <span class="nc">FilterTextFn</span><span class="p">(</span><span class="n">beam</span><span class="o">.</span><span class="n">DoFn</span><span class="p">):</span>
+  <span class="s">"""A DoFn that filters for a specific key based on a regular expression."""</span>
+
+  <span class="c"># A custom aggregator can track values in your pipeline as it runs. Create</span>
+  <span class="c"># custom aggregators matched_word and unmatched_words.</span>
+  <span class="n">matched_words</span> <span class="o">=</span> <span class="n">beam</span><span class="o">.</span><span class="n">Aggregator</span><span class="p">(</span><span class="s">'matched_words'</span><span class="p">)</span>
+  <span class="n">umatched_words</span> <span class="o">=</span> <span class="n">beam</span><span class="o">.</span><span class="n">Aggregator</span><span class="p">(</span><span class="s">'umatched_words'</span><span class="p">)</span>
+
+  <span class="k">def</span> <span class="nf">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pattern</span><span class="p">):</span>
+    <span class="bp">self</span><span class="o">.</span><span class="n">pattern</span> <span class="o">=</span> <span class="n">pattern</span>
+
+  <span class="k">def</span> <span class="nf">process</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
+    <span class="n">word</span><span class="p">,</span> <span class="n">_</span> <span class="o">=</span> <span class="n">context</span><span class="o">.</span><span class="n">element</span>
+    <span class="k">if</span> <span class="n">re</span><span class="o">.</span><span class="n">match</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">pattern</span><span class="p">,</span> <span class="n">word</span><span class="p">):</span>
+      <span class="c"># Log at INFO level each element we match. When executing this pipeline</span>
+      <span class="c"># using the Dataflow service, these log lines will appear in the Cloud</span>
+      <span class="c"># Logging UI.</span>
+      <span class="n">logging</span><span class="o">.</span><span class="n">info</span><span class="p">(</span><span class="s">'Matched </span><span class="si">%</span><span class="s">s'</span><span class="p">,</span> <span class="n">word</span><span class="p">)</span>
+
+      <span class="c"># Add 1 to the custom aggregator matched_words</span>
+      <span class="n">context</span><span class="o">.</span><span class="n">aggregate_to</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">matched_words</span><span class="p">,</span> <span class="mi">1</span><span class="p">)</span>
+      <span class="k">yield</span> <span class="n">context</span><span class="o">.</span><span class="n">element</span>
+    <span class="k">else</span><span class="p">:</span>
+      <span class="c"># Log at the "DEBUG" level each element that is not matched. Different</span>
+      <span class="c"># log levels can be used to control the verbosity of logging providing</span>
+      <span class="c"># an effective mechanism to filter less important information. Note</span>
+      <span class="c"># currently only "INFO" and higher level logs are emitted to the Cloud</span>
+      <span class="c"># Logger. This log message will not be visible in the Cloud Logger.</span>
+      <span class="n">logging</span><span class="o">.</span><span class="n">debug</span><span class="p">(</span><span class="s">'Did not match </span><span class="si">%</span><span class="s">s'</span><span class="p">,</span> <span class="n">word</span><span class="p">)</span>
+
+      <span class="c"># Add 1 to the custom aggregator umatched_words</span>
+      <span class="n">context</span><span class="o">.</span><span class="n">aggregate_to</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">umatched_words</span><span class="p">,</span> <span class="mi">1</span><span class="p">)</span>
+</code></pre>
+</div>
+
 <p>If you execute your pipeline using <code class="highlighter-rouge">DataflowRunner</code>, you can control the worker log levels. Dataflow workers that execute user code are configured to log to Cloud Logging by default at \u201cINFO\u201d log level and higher. You can override log levels for specific logging namespaces by specifying: <code class="highlighter-rouge">--workerLogLevelOverrides={"Name1":"Level1","Name2":"Level2",...}</code>. For example, by specifying <code class="highlighter-rouge">--workerLogLevelOverrides={"org.apache.beam.examples":"DEBUG"}</code> when executing this pipeline using the Dataflow service, Cloud Logging would contain only \u201cDEBUG\u201d or higher level logs for the package in addition to the default \u201cINFO\u201d or higher level logs.</p>
 
 <p>The default Dataflow worker logging configuration can be overridden by specifying <code class="highlighter-rouge">--defaultWorkerLogLevel=&lt;one of TRACE, DEBUG, INFO, WARN, ERROR&gt;</code>. For example, by specifying <code class="highlighter-rouge">--defaultWorkerLogLevel=DEBUG</code> when executing this pipeline with the Dataflow service, Cloud Logging would contain all \u201cDEBUG\u201d or higher level logs. Note that changing the default worker log level to TRACE or DEBUG will significantly increase the amount of logs output.</p>
@@ -500,6 +621,10 @@ Figure 1: The pipeline data flow.</p>
 </code></pre>
 </div>
 
+<div class="language-py highlighter-rouge"><pre class="highlight"><code><span class="n">This</span> <span class="n">feature</span> <span class="ow">is</span> <span class="ow">not</span> <span class="n">yet</span> <span class="n">available</span> <span class="ow">in</span> <span class="n">the</span> <span class="n">Beam</span> <span class="n">SDK</span> <span class="k">for</span> <span class="n">Python</span><span class="o">.</span>
+</code></pre>
+</div>
+
 <h2 id="windowedwordcount">WindowedWordCount</h2>
 
 <p>This example, <code class="highlighter-rouge">WindowedWordCount</code>, counts words in text just as the previous examples did, but introduces several advanced concepts.</p>
@@ -531,6 +656,10 @@ Figure 1: The pipeline data flow.</p>
 </code></pre>
 </div>
 
+<div class="language-py highlighter-rouge"><pre class="highlight"><code><span class="n">This</span> <span class="n">feature</span> <span class="ow">is</span> <span class="ow">not</span> <span class="n">yet</span> <span class="n">available</span> <span class="ow">in</span> <span class="n">the</span> <span class="n">Beam</span> <span class="n">SDK</span> <span class="k">for</span> <span class="n">Python</span><span class="o">.</span>
+</code></pre>
+</div>
+
 <h3 id="adding-timestamps-to-data">Adding Timestamps to Data</h3>
 
 <p>Each element in a <code class="highlighter-rouge">PCollection</code> has an associated <strong>timestamp</strong>. The timestamp for each element is initially assigned by the source that creates the <code class="highlighter-rouge">PCollection</code> and can be adjusted by a <code class="highlighter-rouge">DoFn</code>. In this example the input is bounded. For the purpose of the example, the <code class="highlighter-rouge">DoFn</code> method named <code class="highlighter-rouge">AddTimestampsFn</code> (invoked by <code class="highlighter-rouge">ParDo</code>) will set a timestamp for each element in the <code class="highlighter-rouge">PCollection</code>.</p>
@@ -539,6 +668,10 @@ Figure 1: The pipeline data flow.</p>
 </code></pre>
 </div>
 
+<div class="language-py highlighter-rouge"><pre class="highlight"><code><span class="n">This</span> <span class="n">feature</span> <span class="ow">is</span> <span class="ow">not</span> <span class="n">yet</span> <span class="n">available</span> <span class="ow">in</span> <span class="n">the</span> <span class="n">Beam</span> <span class="n">SDK</span> <span class="k">for</span> <span class="n">Python</span><span class="o">.</span>
+</code></pre>
+</div>
+
 <p>Below is the code for <code class="highlighter-rouge">AddTimestampFn</code>, a <code class="highlighter-rouge">DoFn</code> invoked by <code class="highlighter-rouge">ParDo</code>, that sets the data element of the timestamp given the element itself. For example, if the elements were log lines, this <code class="highlighter-rouge">ParDo</code> could parse the time out of the log string and set it as the element\u2019s timestamp. There are no timestamps inherent in the works of Shakespeare, so in this case we\u2019ve made up random timestamps just to illustrate the concept. Each line of the input text will get a random associated timestamp sometime in a 2-hour period.</p>
 
 <div class="language-java highlighter-rouge"><pre class="highlight"><code><span class="kd">static</span> <span class="kd">class</span> <span class="nc">AddTimestampFn</span> <span class="kd">extends</span> <span class="n">DoFn</span><span class="o">&lt;</span><span class="n">String</span><span class="o">,</span> <span class="n">String</span><span class="o">&gt;</span> <span class="o">{</span>
@@ -562,6 +695,10 @@ Figure 1: The pipeline data flow.</p>
 </code></pre>
 </div>
 
+<div class="language-py highlighter-rouge"><pre class="highlight"><code><span class="n">This</span> <span class="n">feature</span> <span class="ow">is</span> <span class="ow">not</span> <span class="n">yet</span> <span class="n">available</span> <span class="ow">in</span> <span class="n">the</span> <span class="n">Beam</span> <span class="n">SDK</span> <span class="k">for</span> <span class="n">Python</span><span class="o">.</span>
+</code></pre>
+</div>
+
 <h3 id="windowing">Windowing</h3>
 
 <p>Beam uses a concept called <strong>Windowing</strong> to subdivide a <code class="highlighter-rouge">PCollection</code> according to the timestamps of its individual elements. <code class="highlighter-rouge">PTransforms</code> that aggregate multiple elements, process each <code class="highlighter-rouge">PCollection</code> as a succession of multiple, finite windows, even though the entire collection itself may be of infinite size (unbounded).</p>
@@ -574,11 +711,19 @@ Figure 1: The pipeline data flow.</p>
 </code></pre>
 </div>
 
+<div class="language-py highlighter-rouge"><pre class="highlight"><code><span class="n">This</span> <span class="n">feature</span> <span class="ow">is</span> <span class="ow">not</span> <span class="n">yet</span> <span class="n">available</span> <span class="ow">in</span> <span class="n">the</span> <span class="n">Beam</span> <span class="n">SDK</span> <span class="k">for</span> <span class="n">Python</span><span class="o">.</span>
+</code></pre>
+</div>
+
 <h3 id="reusing-ptransforms-over-windowed-pcollections">Reusing PTransforms over windowed PCollections</h3>
 
 <p>You can reuse existing <code class="highlighter-rouge">PTransform</code>s, that were created for manipulating simple <code class="highlighter-rouge">PCollection</code>s, over windowed <code class="highlighter-rouge">PCollection</code>s as well.</p>
 
-<div class="highlighter-rouge"><pre class="highlight"><code>PCollection&lt;KV&lt;String, Long&gt;&gt; wordCounts = windowedWords.apply(new WordCount.CountWords());
+<div class="language-java highlighter-rouge"><pre class="highlight"><code><span class="n">PCollection</span><span class="o">&lt;</span><span class="n">KV</span><span class="o">&lt;</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">&gt;&gt;</span> <span class="n">wordCounts</span> <span class="o">=</span> <span class="n">windowedWords</span><span class="o">.</span><span class="na">apply</span><span class="o">(</span><span class="k">new</span> <span class="n">WordCount</span><span class="o">.</span><span class="na">CountWords</span><span class="o">());</span>
+</code></pre>
+</div>
+
+<div class="language-py highlighter-rouge"><pre class="highlight"><code><span class="n">This</span> <span class="n">feature</span> <span class="ow">is</span> <span class="ow">not</span> <span class="n">yet</span> <span class="n">available</span> <span class="ow">in</span> <span class="n">the</span> <span class="n">Beam</span> <span class="n">SDK</span> <span class="k">for</span> <span class="n">Python</span><span class="o">.</span>
 </code></pre>
 </div>
 
@@ -597,6 +742,10 @@ Figure 1: The pipeline data flow.</p>
 </code></pre>
 </div>
 
+<div class="language-py highlighter-rouge"><pre class="highlight"><code><span class="n">This</span> <span class="n">feature</span> <span class="ow">is</span> <span class="ow">not</span> <span class="n">yet</span> <span class="n">available</span> <span class="ow">in</span> <span class="n">the</span> <span class="n">Beam</span> <span class="n">SDK</span> <span class="k">for</span> <span class="n">Python</span><span class="o">.</span>
+</code></pre>
+</div>
+
 
       </div>
 

http://git-wip-us.apache.org/repos/asf/beam-site/blob/9727633e/content/getting-started/index.html
----------------------------------------------------------------------
diff --git a/content/getting-started/index.html b/content/getting-started/index.html
index 593fc12..87e96c0 100644
--- a/content/getting-started/index.html
+++ b/content/getting-started/index.html
@@ -2,9 +2,9 @@
 <html lang="en-US">
 <meta charset="utf-8">
 <title>Redirecting\u2026</title>
-<link rel="canonical" href="http://beam.apache.org/get-started/quickstart/">
-<meta http-equiv="refresh" content="0; url=http://beam.apache.org/get-started/quickstart/">
+<link rel="canonical" href="http://beam.apache.org/get-started/quickstart-java/">
+<meta http-equiv="refresh" content="0; url=http://beam.apache.org/get-started/quickstart-java/">
 <h1>Redirecting\u2026</h1>
-<a href="http://beam.apache.org/get-started/quickstart/">Click here if you are not redirected.</a>
-<script>location="http://beam.apache.org/get-started/quickstart/"</script>
+<a href="http://beam.apache.org/get-started/quickstart-java/">Click here if you are not redirected.</a>
+<script>location="http://beam.apache.org/get-started/quickstart-java/"</script>
 </html>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/9727633e/content/index.html
----------------------------------------------------------------------
diff --git a/content/index.html b/content/index.html
index 4ff3412..129ef79 100644
--- a/content/index.html
+++ b/content/index.html
@@ -52,7 +52,8 @@
 		  <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Get Started <span class="caret"></span></a>
 		  <ul class="dropdown-menu">
 			  <li><a href="/get-started/beam-overview/">Beam Overview</a></li>
-              <li><a href="/get-started/quickstart/">Quickstart</a></li>
+        <li><a href="/get-started/quickstart-java/">Quickstart - Java</a></li>
+        <li><a href="/get-started/quickstart-py/">Quickstart - Python</a></li>
 			  <li role="separator" class="divider"></li>
 			  <li class="dropdown-header">Example Walkthroughs</li>
 			  <li><a href="/get-started/wordcount-example/">WordCount</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/9727633e/content/privacy_policy/index.html
----------------------------------------------------------------------
diff --git a/content/privacy_policy/index.html b/content/privacy_policy/index.html
index bf0a19f..bed2e5d 100644
--- a/content/privacy_policy/index.html
+++ b/content/privacy_policy/index.html
@@ -52,7 +52,8 @@
 		  <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Get Started <span class="caret"></span></a>
 		  <ul class="dropdown-menu">
 			  <li><a href="/get-started/beam-overview/">Beam Overview</a></li>
-              <li><a href="/get-started/quickstart/">Quickstart</a></li>
+        <li><a href="/get-started/quickstart-java/">Quickstart - Java</a></li>
+        <li><a href="/get-started/quickstart-py/">Quickstart - Python</a></li>
 			  <li role="separator" class="divider"></li>
 			  <li class="dropdown-header">Example Walkthroughs</li>
 			  <li><a href="/get-started/wordcount-example/">WordCount</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/9727633e/content/use/quickstart/index.html
----------------------------------------------------------------------
diff --git a/content/use/quickstart/index.html b/content/use/quickstart/index.html
index 593fc12..87e96c0 100644
--- a/content/use/quickstart/index.html
+++ b/content/use/quickstart/index.html
@@ -2,9 +2,9 @@
 <html lang="en-US">
 <meta charset="utf-8">
 <title>Redirecting\u2026</title>
-<link rel="canonical" href="http://beam.apache.org/get-started/quickstart/">
-<meta http-equiv="refresh" content="0; url=http://beam.apache.org/get-started/quickstart/">
+<link rel="canonical" href="http://beam.apache.org/get-started/quickstart-java/">
+<meta http-equiv="refresh" content="0; url=http://beam.apache.org/get-started/quickstart-java/">
 <h1>Redirecting\u2026</h1>
-<a href="http://beam.apache.org/get-started/quickstart/">Click here if you are not redirected.</a>
-<script>location="http://beam.apache.org/get-started/quickstart/"</script>
+<a href="http://beam.apache.org/get-started/quickstart-java/">Click here if you are not redirected.</a>
+<script>location="http://beam.apache.org/get-started/quickstart-java/"</script>
 </html>


[4/5] beam-site git commit: Regenerate website

Posted by da...@apache.org.
Regenerate website


Project: http://git-wip-us.apache.org/repos/asf/beam-site/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam-site/commit/9727633e
Tree: http://git-wip-us.apache.org/repos/asf/beam-site/tree/9727633e
Diff: http://git-wip-us.apache.org/repos/asf/beam-site/diff/9727633e

Branch: refs/heads/asf-site
Commit: 9727633ebc73db41c7b7e7e9129ece6a36f59d8c
Parents: 852da20
Author: Davor Bonaci <da...@google.com>
Authored: Mon Jan 30 17:36:48 2017 -0800
Committer: Davor Bonaci <da...@google.com>
Committed: Mon Jan 30 17:36:48 2017 -0800

----------------------------------------------------------------------
 .../2016/03/17/capability-matrix.html           |   3 +-
 .../2016/04/03/presentation-materials.html      |   3 +-
 .../sdk/2016/02/25/python-sdk-now-public.html   |   3 +-
 .../beam/release/2016/06/15/first-release.html  |   3 +-
 .../10/11/strata-hadoop-world-and-beam.html     |   3 +-
 .../website/2016/02/22/beam-has-a-logo.html     |   3 +-
 .../blog/2016/05/18/splitAtFraction-method.html |   3 +-
 .../05/27/where-is-my-pcollection-dot-map.html  |   3 +-
 .../06/13/flink-batch-runner-milestone.html     |   3 +-
 content/blog/2016/08/03/six-months.html         |   3 +-
 content/blog/2016/10/20/test-stream.html        |   3 +-
 content/blog/2017/01/09/added-apex-runner.html  |   3 +-
 content/blog/2017/01/10/beam-graduates.html     |   3 +-
 content/blog/index.html                         |   3 +-
 content/coming-soon.html                        |   3 +-
 .../contribute/contribution-guide/index.html    |   3 +-
 content/contribute/design-principles/index.html |   3 +-
 content/contribute/index.html                   |   3 +-
 content/contribute/logos/index.html             |   3 +-
 content/contribute/maturity-model/index.html    |   3 +-
 .../presentation-materials/index.html           |   3 +-
 .../ptransform-style-guide/index.html           |   3 +-
 content/contribute/release-guide/index.html     |   3 +-
 content/contribute/source-repository/index.html |   3 +-
 content/contribute/team/index.html              |   3 +-
 content/contribute/testing/index.html           |   3 +-
 content/contribute/work-in-progress/index.html  |   3 +-
 content/documentation/index.html                |   3 +-
 .../pipelines/create-your-pipeline/index.html   |   3 +-
 .../pipelines/design-your-pipeline/index.html   |   3 +-
 .../pipelines/test-your-pipeline/index.html     |   3 +-
 .../documentation/programming-guide/index.html  |   3 +-
 content/documentation/resources/index.html      |   3 +-
 content/documentation/runners/apex/index.html   |   3 +-
 .../runners/capability-matrix/index.html        |   3 +-
 .../documentation/runners/dataflow/index.html   |   3 +-
 content/documentation/runners/direct/index.html |   3 +-
 content/documentation/runners/flink/index.html  |   3 +-
 content/documentation/runners/spark/index.html  |   3 +-
 content/documentation/sdks/java/index.html      |   3 +-
 content/documentation/sdks/python/index.html    |   3 +-
 content/get-started/beam-overview/index.html    |  14 +-
 content/get-started/downloads/index.html        |   3 +-
 content/get-started/index.html                  |   3 +-
 .../mobile-gaming-example/index.html            |   3 +-
 content/get-started/quickstart-java/index.html  | 395 ++++++++++++++++++
 content/get-started/quickstart-py/index.html    | 288 +++++++++++++
 content/get-started/quickstart/index.html       | 400 +------------------
 content/get-started/support/index.html          |   3 +-
 .../get-started/wordcount-example/index.html    | 205 ++++++++--
 content/getting-started/index.html              |   8 +-
 content/index.html                              |   3 +-
 content/privacy_policy/index.html               |   3 +-
 content/use/quickstart/index.html               |   8 +-
 54 files changed, 982 insertions(+), 477 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam-site/blob/9727633e/content/beam/capability/2016/03/17/capability-matrix.html
----------------------------------------------------------------------
diff --git a/content/beam/capability/2016/03/17/capability-matrix.html b/content/beam/capability/2016/03/17/capability-matrix.html
index 17e9269..72d5dfc 100644
--- a/content/beam/capability/2016/03/17/capability-matrix.html
+++ b/content/beam/capability/2016/03/17/capability-matrix.html
@@ -51,7 +51,8 @@
 		  <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Get Started <span class="caret"></span></a>
 		  <ul class="dropdown-menu">
 			  <li><a href="/get-started/beam-overview/">Beam Overview</a></li>
-              <li><a href="/get-started/quickstart/">Quickstart</a></li>
+        <li><a href="/get-started/quickstart-java/">Quickstart - Java</a></li>
+        <li><a href="/get-started/quickstart-py/">Quickstart - Python</a></li>
 			  <li role="separator" class="divider"></li>
 			  <li class="dropdown-header">Example Walkthroughs</li>
 			  <li><a href="/get-started/wordcount-example/">WordCount</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/9727633e/content/beam/capability/2016/04/03/presentation-materials.html
----------------------------------------------------------------------
diff --git a/content/beam/capability/2016/04/03/presentation-materials.html b/content/beam/capability/2016/04/03/presentation-materials.html
index 61e0377..13b8e18 100644
--- a/content/beam/capability/2016/04/03/presentation-materials.html
+++ b/content/beam/capability/2016/04/03/presentation-materials.html
@@ -51,7 +51,8 @@
 		  <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Get Started <span class="caret"></span></a>
 		  <ul class="dropdown-menu">
 			  <li><a href="/get-started/beam-overview/">Beam Overview</a></li>
-              <li><a href="/get-started/quickstart/">Quickstart</a></li>
+        <li><a href="/get-started/quickstart-java/">Quickstart - Java</a></li>
+        <li><a href="/get-started/quickstart-py/">Quickstart - Python</a></li>
 			  <li role="separator" class="divider"></li>
 			  <li class="dropdown-header">Example Walkthroughs</li>
 			  <li><a href="/get-started/wordcount-example/">WordCount</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/9727633e/content/beam/python/sdk/2016/02/25/python-sdk-now-public.html
----------------------------------------------------------------------
diff --git a/content/beam/python/sdk/2016/02/25/python-sdk-now-public.html b/content/beam/python/sdk/2016/02/25/python-sdk-now-public.html
index 4bc8bfe..25cdaad 100644
--- a/content/beam/python/sdk/2016/02/25/python-sdk-now-public.html
+++ b/content/beam/python/sdk/2016/02/25/python-sdk-now-public.html
@@ -51,7 +51,8 @@
 		  <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Get Started <span class="caret"></span></a>
 		  <ul class="dropdown-menu">
 			  <li><a href="/get-started/beam-overview/">Beam Overview</a></li>
-              <li><a href="/get-started/quickstart/">Quickstart</a></li>
+        <li><a href="/get-started/quickstart-java/">Quickstart - Java</a></li>
+        <li><a href="/get-started/quickstart-py/">Quickstart - Python</a></li>
 			  <li role="separator" class="divider"></li>
 			  <li class="dropdown-header">Example Walkthroughs</li>
 			  <li><a href="/get-started/wordcount-example/">WordCount</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/9727633e/content/beam/release/2016/06/15/first-release.html
----------------------------------------------------------------------
diff --git a/content/beam/release/2016/06/15/first-release.html b/content/beam/release/2016/06/15/first-release.html
index 8dd43c4..309ea7b 100644
--- a/content/beam/release/2016/06/15/first-release.html
+++ b/content/beam/release/2016/06/15/first-release.html
@@ -51,7 +51,8 @@
 		  <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Get Started <span class="caret"></span></a>
 		  <ul class="dropdown-menu">
 			  <li><a href="/get-started/beam-overview/">Beam Overview</a></li>
-              <li><a href="/get-started/quickstart/">Quickstart</a></li>
+        <li><a href="/get-started/quickstart-java/">Quickstart - Java</a></li>
+        <li><a href="/get-started/quickstart-py/">Quickstart - Python</a></li>
 			  <li role="separator" class="divider"></li>
 			  <li class="dropdown-header">Example Walkthroughs</li>
 			  <li><a href="/get-started/wordcount-example/">WordCount</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/9727633e/content/beam/update/2016/10/11/strata-hadoop-world-and-beam.html
----------------------------------------------------------------------
diff --git a/content/beam/update/2016/10/11/strata-hadoop-world-and-beam.html b/content/beam/update/2016/10/11/strata-hadoop-world-and-beam.html
index 455271f..a02c380 100644
--- a/content/beam/update/2016/10/11/strata-hadoop-world-and-beam.html
+++ b/content/beam/update/2016/10/11/strata-hadoop-world-and-beam.html
@@ -51,7 +51,8 @@
 		  <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Get Started <span class="caret"></span></a>
 		  <ul class="dropdown-menu">
 			  <li><a href="/get-started/beam-overview/">Beam Overview</a></li>
-              <li><a href="/get-started/quickstart/">Quickstart</a></li>
+        <li><a href="/get-started/quickstart-java/">Quickstart - Java</a></li>
+        <li><a href="/get-started/quickstart-py/">Quickstart - Python</a></li>
 			  <li role="separator" class="divider"></li>
 			  <li class="dropdown-header">Example Walkthroughs</li>
 			  <li><a href="/get-started/wordcount-example/">WordCount</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/9727633e/content/beam/update/website/2016/02/22/beam-has-a-logo.html
----------------------------------------------------------------------
diff --git a/content/beam/update/website/2016/02/22/beam-has-a-logo.html b/content/beam/update/website/2016/02/22/beam-has-a-logo.html
index 09ded66..897b0b6 100644
--- a/content/beam/update/website/2016/02/22/beam-has-a-logo.html
+++ b/content/beam/update/website/2016/02/22/beam-has-a-logo.html
@@ -51,7 +51,8 @@
 		  <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Get Started <span class="caret"></span></a>
 		  <ul class="dropdown-menu">
 			  <li><a href="/get-started/beam-overview/">Beam Overview</a></li>
-              <li><a href="/get-started/quickstart/">Quickstart</a></li>
+        <li><a href="/get-started/quickstart-java/">Quickstart - Java</a></li>
+        <li><a href="/get-started/quickstart-py/">Quickstart - Python</a></li>
 			  <li role="separator" class="divider"></li>
 			  <li class="dropdown-header">Example Walkthroughs</li>
 			  <li><a href="/get-started/wordcount-example/">WordCount</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/9727633e/content/blog/2016/05/18/splitAtFraction-method.html
----------------------------------------------------------------------
diff --git a/content/blog/2016/05/18/splitAtFraction-method.html b/content/blog/2016/05/18/splitAtFraction-method.html
index 3096435..283f617 100644
--- a/content/blog/2016/05/18/splitAtFraction-method.html
+++ b/content/blog/2016/05/18/splitAtFraction-method.html
@@ -51,7 +51,8 @@
 		  <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Get Started <span class="caret"></span></a>
 		  <ul class="dropdown-menu">
 			  <li><a href="/get-started/beam-overview/">Beam Overview</a></li>
-              <li><a href="/get-started/quickstart/">Quickstart</a></li>
+        <li><a href="/get-started/quickstart-java/">Quickstart - Java</a></li>
+        <li><a href="/get-started/quickstart-py/">Quickstart - Python</a></li>
 			  <li role="separator" class="divider"></li>
 			  <li class="dropdown-header">Example Walkthroughs</li>
 			  <li><a href="/get-started/wordcount-example/">WordCount</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/9727633e/content/blog/2016/05/27/where-is-my-pcollection-dot-map.html
----------------------------------------------------------------------
diff --git a/content/blog/2016/05/27/where-is-my-pcollection-dot-map.html b/content/blog/2016/05/27/where-is-my-pcollection-dot-map.html
index 4b0652f..de021f0 100644
--- a/content/blog/2016/05/27/where-is-my-pcollection-dot-map.html
+++ b/content/blog/2016/05/27/where-is-my-pcollection-dot-map.html
@@ -51,7 +51,8 @@
 		  <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Get Started <span class="caret"></span></a>
 		  <ul class="dropdown-menu">
 			  <li><a href="/get-started/beam-overview/">Beam Overview</a></li>
-              <li><a href="/get-started/quickstart/">Quickstart</a></li>
+        <li><a href="/get-started/quickstart-java/">Quickstart - Java</a></li>
+        <li><a href="/get-started/quickstart-py/">Quickstart - Python</a></li>
 			  <li role="separator" class="divider"></li>
 			  <li class="dropdown-header">Example Walkthroughs</li>
 			  <li><a href="/get-started/wordcount-example/">WordCount</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/9727633e/content/blog/2016/06/13/flink-batch-runner-milestone.html
----------------------------------------------------------------------
diff --git a/content/blog/2016/06/13/flink-batch-runner-milestone.html b/content/blog/2016/06/13/flink-batch-runner-milestone.html
index d3bc2f4..a4ee820 100644
--- a/content/blog/2016/06/13/flink-batch-runner-milestone.html
+++ b/content/blog/2016/06/13/flink-batch-runner-milestone.html
@@ -51,7 +51,8 @@
 		  <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Get Started <span class="caret"></span></a>
 		  <ul class="dropdown-menu">
 			  <li><a href="/get-started/beam-overview/">Beam Overview</a></li>
-              <li><a href="/get-started/quickstart/">Quickstart</a></li>
+        <li><a href="/get-started/quickstart-java/">Quickstart - Java</a></li>
+        <li><a href="/get-started/quickstart-py/">Quickstart - Python</a></li>
 			  <li role="separator" class="divider"></li>
 			  <li class="dropdown-header">Example Walkthroughs</li>
 			  <li><a href="/get-started/wordcount-example/">WordCount</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/9727633e/content/blog/2016/08/03/six-months.html
----------------------------------------------------------------------
diff --git a/content/blog/2016/08/03/six-months.html b/content/blog/2016/08/03/six-months.html
index d183491..e84c0c8 100644
--- a/content/blog/2016/08/03/six-months.html
+++ b/content/blog/2016/08/03/six-months.html
@@ -51,7 +51,8 @@
 		  <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Get Started <span class="caret"></span></a>
 		  <ul class="dropdown-menu">
 			  <li><a href="/get-started/beam-overview/">Beam Overview</a></li>
-              <li><a href="/get-started/quickstart/">Quickstart</a></li>
+        <li><a href="/get-started/quickstart-java/">Quickstart - Java</a></li>
+        <li><a href="/get-started/quickstart-py/">Quickstart - Python</a></li>
 			  <li role="separator" class="divider"></li>
 			  <li class="dropdown-header">Example Walkthroughs</li>
 			  <li><a href="/get-started/wordcount-example/">WordCount</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/9727633e/content/blog/2016/10/20/test-stream.html
----------------------------------------------------------------------
diff --git a/content/blog/2016/10/20/test-stream.html b/content/blog/2016/10/20/test-stream.html
index 211834f..69dcc97 100644
--- a/content/blog/2016/10/20/test-stream.html
+++ b/content/blog/2016/10/20/test-stream.html
@@ -51,7 +51,8 @@
 		  <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Get Started <span class="caret"></span></a>
 		  <ul class="dropdown-menu">
 			  <li><a href="/get-started/beam-overview/">Beam Overview</a></li>
-              <li><a href="/get-started/quickstart/">Quickstart</a></li>
+        <li><a href="/get-started/quickstart-java/">Quickstart - Java</a></li>
+        <li><a href="/get-started/quickstart-py/">Quickstart - Python</a></li>
 			  <li role="separator" class="divider"></li>
 			  <li class="dropdown-header">Example Walkthroughs</li>
 			  <li><a href="/get-started/wordcount-example/">WordCount</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/9727633e/content/blog/2017/01/09/added-apex-runner.html
----------------------------------------------------------------------
diff --git a/content/blog/2017/01/09/added-apex-runner.html b/content/blog/2017/01/09/added-apex-runner.html
index 6cc8d6c..d9a740d 100644
--- a/content/blog/2017/01/09/added-apex-runner.html
+++ b/content/blog/2017/01/09/added-apex-runner.html
@@ -51,7 +51,8 @@
 		  <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Get Started <span class="caret"></span></a>
 		  <ul class="dropdown-menu">
 			  <li><a href="/get-started/beam-overview/">Beam Overview</a></li>
-              <li><a href="/get-started/quickstart/">Quickstart</a></li>
+        <li><a href="/get-started/quickstart-java/">Quickstart - Java</a></li>
+        <li><a href="/get-started/quickstart-py/">Quickstart - Python</a></li>
 			  <li role="separator" class="divider"></li>
 			  <li class="dropdown-header">Example Walkthroughs</li>
 			  <li><a href="/get-started/wordcount-example/">WordCount</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/9727633e/content/blog/2017/01/10/beam-graduates.html
----------------------------------------------------------------------
diff --git a/content/blog/2017/01/10/beam-graduates.html b/content/blog/2017/01/10/beam-graduates.html
index dab7f53..a4edd6e 100644
--- a/content/blog/2017/01/10/beam-graduates.html
+++ b/content/blog/2017/01/10/beam-graduates.html
@@ -51,7 +51,8 @@
 		  <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Get Started <span class="caret"></span></a>
 		  <ul class="dropdown-menu">
 			  <li><a href="/get-started/beam-overview/">Beam Overview</a></li>
-              <li><a href="/get-started/quickstart/">Quickstart</a></li>
+        <li><a href="/get-started/quickstart-java/">Quickstart - Java</a></li>
+        <li><a href="/get-started/quickstart-py/">Quickstart - Python</a></li>
 			  <li role="separator" class="divider"></li>
 			  <li class="dropdown-header">Example Walkthroughs</li>
 			  <li><a href="/get-started/wordcount-example/">WordCount</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/9727633e/content/blog/index.html
----------------------------------------------------------------------
diff --git a/content/blog/index.html b/content/blog/index.html
index 0ede785..5b5397d 100644
--- a/content/blog/index.html
+++ b/content/blog/index.html
@@ -52,7 +52,8 @@
 		  <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Get Started <span class="caret"></span></a>
 		  <ul class="dropdown-menu">
 			  <li><a href="/get-started/beam-overview/">Beam Overview</a></li>
-              <li><a href="/get-started/quickstart/">Quickstart</a></li>
+        <li><a href="/get-started/quickstart-java/">Quickstart - Java</a></li>
+        <li><a href="/get-started/quickstart-py/">Quickstart - Python</a></li>
 			  <li role="separator" class="divider"></li>
 			  <li class="dropdown-header">Example Walkthroughs</li>
 			  <li><a href="/get-started/wordcount-example/">WordCount</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/9727633e/content/coming-soon.html
----------------------------------------------------------------------
diff --git a/content/coming-soon.html b/content/coming-soon.html
index 4369047..603b93e 100644
--- a/content/coming-soon.html
+++ b/content/coming-soon.html
@@ -52,7 +52,8 @@
 		  <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Get Started <span class="caret"></span></a>
 		  <ul class="dropdown-menu">
 			  <li><a href="/get-started/beam-overview/">Beam Overview</a></li>
-              <li><a href="/get-started/quickstart/">Quickstart</a></li>
+        <li><a href="/get-started/quickstart-java/">Quickstart - Java</a></li>
+        <li><a href="/get-started/quickstart-py/">Quickstart - Python</a></li>
 			  <li role="separator" class="divider"></li>
 			  <li class="dropdown-header">Example Walkthroughs</li>
 			  <li><a href="/get-started/wordcount-example/">WordCount</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/9727633e/content/contribute/contribution-guide/index.html
----------------------------------------------------------------------
diff --git a/content/contribute/contribution-guide/index.html b/content/contribute/contribution-guide/index.html
index 14a21a1..9bd596d 100644
--- a/content/contribute/contribution-guide/index.html
+++ b/content/contribute/contribution-guide/index.html
@@ -52,7 +52,8 @@
 		  <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Get Started <span class="caret"></span></a>
 		  <ul class="dropdown-menu">
 			  <li><a href="/get-started/beam-overview/">Beam Overview</a></li>
-              <li><a href="/get-started/quickstart/">Quickstart</a></li>
+        <li><a href="/get-started/quickstart-java/">Quickstart - Java</a></li>
+        <li><a href="/get-started/quickstart-py/">Quickstart - Python</a></li>
 			  <li role="separator" class="divider"></li>
 			  <li class="dropdown-header">Example Walkthroughs</li>
 			  <li><a href="/get-started/wordcount-example/">WordCount</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/9727633e/content/contribute/design-principles/index.html
----------------------------------------------------------------------
diff --git a/content/contribute/design-principles/index.html b/content/contribute/design-principles/index.html
index 046cbd7..78cebf4 100644
--- a/content/contribute/design-principles/index.html
+++ b/content/contribute/design-principles/index.html
@@ -52,7 +52,8 @@
 		  <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Get Started <span class="caret"></span></a>
 		  <ul class="dropdown-menu">
 			  <li><a href="/get-started/beam-overview/">Beam Overview</a></li>
-              <li><a href="/get-started/quickstart/">Quickstart</a></li>
+        <li><a href="/get-started/quickstart-java/">Quickstart - Java</a></li>
+        <li><a href="/get-started/quickstart-py/">Quickstart - Python</a></li>
 			  <li role="separator" class="divider"></li>
 			  <li class="dropdown-header">Example Walkthroughs</li>
 			  <li><a href="/get-started/wordcount-example/">WordCount</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/9727633e/content/contribute/index.html
----------------------------------------------------------------------
diff --git a/content/contribute/index.html b/content/contribute/index.html
index e5303e8..9c3dbea 100644
--- a/content/contribute/index.html
+++ b/content/contribute/index.html
@@ -52,7 +52,8 @@
 		  <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Get Started <span class="caret"></span></a>
 		  <ul class="dropdown-menu">
 			  <li><a href="/get-started/beam-overview/">Beam Overview</a></li>
-              <li><a href="/get-started/quickstart/">Quickstart</a></li>
+        <li><a href="/get-started/quickstart-java/">Quickstart - Java</a></li>
+        <li><a href="/get-started/quickstart-py/">Quickstart - Python</a></li>
 			  <li role="separator" class="divider"></li>
 			  <li class="dropdown-header">Example Walkthroughs</li>
 			  <li><a href="/get-started/wordcount-example/">WordCount</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/9727633e/content/contribute/logos/index.html
----------------------------------------------------------------------
diff --git a/content/contribute/logos/index.html b/content/contribute/logos/index.html
index 6ed790e..816e031 100644
--- a/content/contribute/logos/index.html
+++ b/content/contribute/logos/index.html
@@ -52,7 +52,8 @@
 		  <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Get Started <span class="caret"></span></a>
 		  <ul class="dropdown-menu">
 			  <li><a href="/get-started/beam-overview/">Beam Overview</a></li>
-              <li><a href="/get-started/quickstart/">Quickstart</a></li>
+        <li><a href="/get-started/quickstart-java/">Quickstart - Java</a></li>
+        <li><a href="/get-started/quickstart-py/">Quickstart - Python</a></li>
 			  <li role="separator" class="divider"></li>
 			  <li class="dropdown-header">Example Walkthroughs</li>
 			  <li><a href="/get-started/wordcount-example/">WordCount</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/9727633e/content/contribute/maturity-model/index.html
----------------------------------------------------------------------
diff --git a/content/contribute/maturity-model/index.html b/content/contribute/maturity-model/index.html
index 9cbf5c5..946a891 100644
--- a/content/contribute/maturity-model/index.html
+++ b/content/contribute/maturity-model/index.html
@@ -52,7 +52,8 @@
 		  <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Get Started <span class="caret"></span></a>
 		  <ul class="dropdown-menu">
 			  <li><a href="/get-started/beam-overview/">Beam Overview</a></li>
-              <li><a href="/get-started/quickstart/">Quickstart</a></li>
+        <li><a href="/get-started/quickstart-java/">Quickstart - Java</a></li>
+        <li><a href="/get-started/quickstart-py/">Quickstart - Python</a></li>
 			  <li role="separator" class="divider"></li>
 			  <li class="dropdown-header">Example Walkthroughs</li>
 			  <li><a href="/get-started/wordcount-example/">WordCount</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/9727633e/content/contribute/presentation-materials/index.html
----------------------------------------------------------------------
diff --git a/content/contribute/presentation-materials/index.html b/content/contribute/presentation-materials/index.html
index ab19c76..995cb07 100644
--- a/content/contribute/presentation-materials/index.html
+++ b/content/contribute/presentation-materials/index.html
@@ -52,7 +52,8 @@
 		  <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Get Started <span class="caret"></span></a>
 		  <ul class="dropdown-menu">
 			  <li><a href="/get-started/beam-overview/">Beam Overview</a></li>
-              <li><a href="/get-started/quickstart/">Quickstart</a></li>
+        <li><a href="/get-started/quickstart-java/">Quickstart - Java</a></li>
+        <li><a href="/get-started/quickstart-py/">Quickstart - Python</a></li>
 			  <li role="separator" class="divider"></li>
 			  <li class="dropdown-header">Example Walkthroughs</li>
 			  <li><a href="/get-started/wordcount-example/">WordCount</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/9727633e/content/contribute/ptransform-style-guide/index.html
----------------------------------------------------------------------
diff --git a/content/contribute/ptransform-style-guide/index.html b/content/contribute/ptransform-style-guide/index.html
index dbf9b6b..426b83c 100644
--- a/content/contribute/ptransform-style-guide/index.html
+++ b/content/contribute/ptransform-style-guide/index.html
@@ -52,7 +52,8 @@
 		  <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Get Started <span class="caret"></span></a>
 		  <ul class="dropdown-menu">
 			  <li><a href="/get-started/beam-overview/">Beam Overview</a></li>
-              <li><a href="/get-started/quickstart/">Quickstart</a></li>
+        <li><a href="/get-started/quickstart-java/">Quickstart - Java</a></li>
+        <li><a href="/get-started/quickstart-py/">Quickstart - Python</a></li>
 			  <li role="separator" class="divider"></li>
 			  <li class="dropdown-header">Example Walkthroughs</li>
 			  <li><a href="/get-started/wordcount-example/">WordCount</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/9727633e/content/contribute/release-guide/index.html
----------------------------------------------------------------------
diff --git a/content/contribute/release-guide/index.html b/content/contribute/release-guide/index.html
index 8ed28cd..92f43cd 100644
--- a/content/contribute/release-guide/index.html
+++ b/content/contribute/release-guide/index.html
@@ -52,7 +52,8 @@
 		  <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Get Started <span class="caret"></span></a>
 		  <ul class="dropdown-menu">
 			  <li><a href="/get-started/beam-overview/">Beam Overview</a></li>
-              <li><a href="/get-started/quickstart/">Quickstart</a></li>
+        <li><a href="/get-started/quickstart-java/">Quickstart - Java</a></li>
+        <li><a href="/get-started/quickstart-py/">Quickstart - Python</a></li>
 			  <li role="separator" class="divider"></li>
 			  <li class="dropdown-header">Example Walkthroughs</li>
 			  <li><a href="/get-started/wordcount-example/">WordCount</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/9727633e/content/contribute/source-repository/index.html
----------------------------------------------------------------------
diff --git a/content/contribute/source-repository/index.html b/content/contribute/source-repository/index.html
index 991e2c4..87bcd98 100644
--- a/content/contribute/source-repository/index.html
+++ b/content/contribute/source-repository/index.html
@@ -52,7 +52,8 @@
 		  <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Get Started <span class="caret"></span></a>
 		  <ul class="dropdown-menu">
 			  <li><a href="/get-started/beam-overview/">Beam Overview</a></li>
-              <li><a href="/get-started/quickstart/">Quickstart</a></li>
+        <li><a href="/get-started/quickstart-java/">Quickstart - Java</a></li>
+        <li><a href="/get-started/quickstart-py/">Quickstart - Python</a></li>
 			  <li role="separator" class="divider"></li>
 			  <li class="dropdown-header">Example Walkthroughs</li>
 			  <li><a href="/get-started/wordcount-example/">WordCount</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/9727633e/content/contribute/team/index.html
----------------------------------------------------------------------
diff --git a/content/contribute/team/index.html b/content/contribute/team/index.html
index f950b73..3281eb3 100644
--- a/content/contribute/team/index.html
+++ b/content/contribute/team/index.html
@@ -52,7 +52,8 @@
 		  <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Get Started <span class="caret"></span></a>
 		  <ul class="dropdown-menu">
 			  <li><a href="/get-started/beam-overview/">Beam Overview</a></li>
-              <li><a href="/get-started/quickstart/">Quickstart</a></li>
+        <li><a href="/get-started/quickstart-java/">Quickstart - Java</a></li>
+        <li><a href="/get-started/quickstart-py/">Quickstart - Python</a></li>
 			  <li role="separator" class="divider"></li>
 			  <li class="dropdown-header">Example Walkthroughs</li>
 			  <li><a href="/get-started/wordcount-example/">WordCount</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/9727633e/content/contribute/testing/index.html
----------------------------------------------------------------------
diff --git a/content/contribute/testing/index.html b/content/contribute/testing/index.html
index 7878126..683c8e3 100644
--- a/content/contribute/testing/index.html
+++ b/content/contribute/testing/index.html
@@ -52,7 +52,8 @@
 		  <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Get Started <span class="caret"></span></a>
 		  <ul class="dropdown-menu">
 			  <li><a href="/get-started/beam-overview/">Beam Overview</a></li>
-              <li><a href="/get-started/quickstart/">Quickstart</a></li>
+        <li><a href="/get-started/quickstart-java/">Quickstart - Java</a></li>
+        <li><a href="/get-started/quickstart-py/">Quickstart - Python</a></li>
 			  <li role="separator" class="divider"></li>
 			  <li class="dropdown-header">Example Walkthroughs</li>
 			  <li><a href="/get-started/wordcount-example/">WordCount</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/9727633e/content/contribute/work-in-progress/index.html
----------------------------------------------------------------------
diff --git a/content/contribute/work-in-progress/index.html b/content/contribute/work-in-progress/index.html
index 8f1d499..992cf46 100644
--- a/content/contribute/work-in-progress/index.html
+++ b/content/contribute/work-in-progress/index.html
@@ -52,7 +52,8 @@
 		  <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Get Started <span class="caret"></span></a>
 		  <ul class="dropdown-menu">
 			  <li><a href="/get-started/beam-overview/">Beam Overview</a></li>
-              <li><a href="/get-started/quickstart/">Quickstart</a></li>
+        <li><a href="/get-started/quickstart-java/">Quickstart - Java</a></li>
+        <li><a href="/get-started/quickstart-py/">Quickstart - Python</a></li>
 			  <li role="separator" class="divider"></li>
 			  <li class="dropdown-header">Example Walkthroughs</li>
 			  <li><a href="/get-started/wordcount-example/">WordCount</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/9727633e/content/documentation/index.html
----------------------------------------------------------------------
diff --git a/content/documentation/index.html b/content/documentation/index.html
index a791a12..88234b7 100644
--- a/content/documentation/index.html
+++ b/content/documentation/index.html
@@ -52,7 +52,8 @@
 		  <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Get Started <span class="caret"></span></a>
 		  <ul class="dropdown-menu">
 			  <li><a href="/get-started/beam-overview/">Beam Overview</a></li>
-              <li><a href="/get-started/quickstart/">Quickstart</a></li>
+        <li><a href="/get-started/quickstart-java/">Quickstart - Java</a></li>
+        <li><a href="/get-started/quickstart-py/">Quickstart - Python</a></li>
 			  <li role="separator" class="divider"></li>
 			  <li class="dropdown-header">Example Walkthroughs</li>
 			  <li><a href="/get-started/wordcount-example/">WordCount</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/9727633e/content/documentation/pipelines/create-your-pipeline/index.html
----------------------------------------------------------------------
diff --git a/content/documentation/pipelines/create-your-pipeline/index.html b/content/documentation/pipelines/create-your-pipeline/index.html
index bcb3b6b..d3763d9 100644
--- a/content/documentation/pipelines/create-your-pipeline/index.html
+++ b/content/documentation/pipelines/create-your-pipeline/index.html
@@ -52,7 +52,8 @@
 		  <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Get Started <span class="caret"></span></a>
 		  <ul class="dropdown-menu">
 			  <li><a href="/get-started/beam-overview/">Beam Overview</a></li>
-              <li><a href="/get-started/quickstart/">Quickstart</a></li>
+        <li><a href="/get-started/quickstart-java/">Quickstart - Java</a></li>
+        <li><a href="/get-started/quickstart-py/">Quickstart - Python</a></li>
 			  <li role="separator" class="divider"></li>
 			  <li class="dropdown-header">Example Walkthroughs</li>
 			  <li><a href="/get-started/wordcount-example/">WordCount</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/9727633e/content/documentation/pipelines/design-your-pipeline/index.html
----------------------------------------------------------------------
diff --git a/content/documentation/pipelines/design-your-pipeline/index.html b/content/documentation/pipelines/design-your-pipeline/index.html
index 57cd016..ed233d2 100644
--- a/content/documentation/pipelines/design-your-pipeline/index.html
+++ b/content/documentation/pipelines/design-your-pipeline/index.html
@@ -52,7 +52,8 @@
 		  <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Get Started <span class="caret"></span></a>
 		  <ul class="dropdown-menu">
 			  <li><a href="/get-started/beam-overview/">Beam Overview</a></li>
-              <li><a href="/get-started/quickstart/">Quickstart</a></li>
+        <li><a href="/get-started/quickstart-java/">Quickstart - Java</a></li>
+        <li><a href="/get-started/quickstart-py/">Quickstart - Python</a></li>
 			  <li role="separator" class="divider"></li>
 			  <li class="dropdown-header">Example Walkthroughs</li>
 			  <li><a href="/get-started/wordcount-example/">WordCount</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/9727633e/content/documentation/pipelines/test-your-pipeline/index.html
----------------------------------------------------------------------
diff --git a/content/documentation/pipelines/test-your-pipeline/index.html b/content/documentation/pipelines/test-your-pipeline/index.html
index dc298d5..6116ac2 100644
--- a/content/documentation/pipelines/test-your-pipeline/index.html
+++ b/content/documentation/pipelines/test-your-pipeline/index.html
@@ -52,7 +52,8 @@
 		  <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Get Started <span class="caret"></span></a>
 		  <ul class="dropdown-menu">
 			  <li><a href="/get-started/beam-overview/">Beam Overview</a></li>
-              <li><a href="/get-started/quickstart/">Quickstart</a></li>
+        <li><a href="/get-started/quickstart-java/">Quickstart - Java</a></li>
+        <li><a href="/get-started/quickstart-py/">Quickstart - Python</a></li>
 			  <li role="separator" class="divider"></li>
 			  <li class="dropdown-header">Example Walkthroughs</li>
 			  <li><a href="/get-started/wordcount-example/">WordCount</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/9727633e/content/documentation/programming-guide/index.html
----------------------------------------------------------------------
diff --git a/content/documentation/programming-guide/index.html b/content/documentation/programming-guide/index.html
index 2810e5d..80eee5f 100644
--- a/content/documentation/programming-guide/index.html
+++ b/content/documentation/programming-guide/index.html
@@ -52,7 +52,8 @@
 		  <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Get Started <span class="caret"></span></a>
 		  <ul class="dropdown-menu">
 			  <li><a href="/get-started/beam-overview/">Beam Overview</a></li>
-              <li><a href="/get-started/quickstart/">Quickstart</a></li>
+        <li><a href="/get-started/quickstart-java/">Quickstart - Java</a></li>
+        <li><a href="/get-started/quickstart-py/">Quickstart - Python</a></li>
 			  <li role="separator" class="divider"></li>
 			  <li class="dropdown-header">Example Walkthroughs</li>
 			  <li><a href="/get-started/wordcount-example/">WordCount</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/9727633e/content/documentation/resources/index.html
----------------------------------------------------------------------
diff --git a/content/documentation/resources/index.html b/content/documentation/resources/index.html
index 1206e10..6ef61a6 100644
--- a/content/documentation/resources/index.html
+++ b/content/documentation/resources/index.html
@@ -52,7 +52,8 @@
 		  <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Get Started <span class="caret"></span></a>
 		  <ul class="dropdown-menu">
 			  <li><a href="/get-started/beam-overview/">Beam Overview</a></li>
-              <li><a href="/get-started/quickstart/">Quickstart</a></li>
+        <li><a href="/get-started/quickstart-java/">Quickstart - Java</a></li>
+        <li><a href="/get-started/quickstart-py/">Quickstart - Python</a></li>
 			  <li role="separator" class="divider"></li>
 			  <li class="dropdown-header">Example Walkthroughs</li>
 			  <li><a href="/get-started/wordcount-example/">WordCount</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/9727633e/content/documentation/runners/apex/index.html
----------------------------------------------------------------------
diff --git a/content/documentation/runners/apex/index.html b/content/documentation/runners/apex/index.html
index e6b7491..fb30535 100644
--- a/content/documentation/runners/apex/index.html
+++ b/content/documentation/runners/apex/index.html
@@ -52,7 +52,8 @@
 		  <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Get Started <span class="caret"></span></a>
 		  <ul class="dropdown-menu">
 			  <li><a href="/get-started/beam-overview/">Beam Overview</a></li>
-              <li><a href="/get-started/quickstart/">Quickstart</a></li>
+        <li><a href="/get-started/quickstart-java/">Quickstart - Java</a></li>
+        <li><a href="/get-started/quickstart-py/">Quickstart - Python</a></li>
 			  <li role="separator" class="divider"></li>
 			  <li class="dropdown-header">Example Walkthroughs</li>
 			  <li><a href="/get-started/wordcount-example/">WordCount</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/9727633e/content/documentation/runners/capability-matrix/index.html
----------------------------------------------------------------------
diff --git a/content/documentation/runners/capability-matrix/index.html b/content/documentation/runners/capability-matrix/index.html
index af8d80a..cd29ec3 100644
--- a/content/documentation/runners/capability-matrix/index.html
+++ b/content/documentation/runners/capability-matrix/index.html
@@ -52,7 +52,8 @@
 		  <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Get Started <span class="caret"></span></a>
 		  <ul class="dropdown-menu">
 			  <li><a href="/get-started/beam-overview/">Beam Overview</a></li>
-              <li><a href="/get-started/quickstart/">Quickstart</a></li>
+        <li><a href="/get-started/quickstart-java/">Quickstart - Java</a></li>
+        <li><a href="/get-started/quickstart-py/">Quickstart - Python</a></li>
 			  <li role="separator" class="divider"></li>
 			  <li class="dropdown-header">Example Walkthroughs</li>
 			  <li><a href="/get-started/wordcount-example/">WordCount</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/9727633e/content/documentation/runners/dataflow/index.html
----------------------------------------------------------------------
diff --git a/content/documentation/runners/dataflow/index.html b/content/documentation/runners/dataflow/index.html
index be4f8d6..9a0da20 100644
--- a/content/documentation/runners/dataflow/index.html
+++ b/content/documentation/runners/dataflow/index.html
@@ -52,7 +52,8 @@
 		  <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Get Started <span class="caret"></span></a>
 		  <ul class="dropdown-menu">
 			  <li><a href="/get-started/beam-overview/">Beam Overview</a></li>
-              <li><a href="/get-started/quickstart/">Quickstart</a></li>
+        <li><a href="/get-started/quickstart-java/">Quickstart - Java</a></li>
+        <li><a href="/get-started/quickstart-py/">Quickstart - Python</a></li>
 			  <li role="separator" class="divider"></li>
 			  <li class="dropdown-header">Example Walkthroughs</li>
 			  <li><a href="/get-started/wordcount-example/">WordCount</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/9727633e/content/documentation/runners/direct/index.html
----------------------------------------------------------------------
diff --git a/content/documentation/runners/direct/index.html b/content/documentation/runners/direct/index.html
index 5356362..c7a8e2d 100644
--- a/content/documentation/runners/direct/index.html
+++ b/content/documentation/runners/direct/index.html
@@ -52,7 +52,8 @@
 		  <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Get Started <span class="caret"></span></a>
 		  <ul class="dropdown-menu">
 			  <li><a href="/get-started/beam-overview/">Beam Overview</a></li>
-              <li><a href="/get-started/quickstart/">Quickstart</a></li>
+        <li><a href="/get-started/quickstart-java/">Quickstart - Java</a></li>
+        <li><a href="/get-started/quickstart-py/">Quickstart - Python</a></li>
 			  <li role="separator" class="divider"></li>
 			  <li class="dropdown-header">Example Walkthroughs</li>
 			  <li><a href="/get-started/wordcount-example/">WordCount</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/9727633e/content/documentation/runners/flink/index.html
----------------------------------------------------------------------
diff --git a/content/documentation/runners/flink/index.html b/content/documentation/runners/flink/index.html
index dcf1e9b..1e5137b 100644
--- a/content/documentation/runners/flink/index.html
+++ b/content/documentation/runners/flink/index.html
@@ -52,7 +52,8 @@
 		  <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Get Started <span class="caret"></span></a>
 		  <ul class="dropdown-menu">
 			  <li><a href="/get-started/beam-overview/">Beam Overview</a></li>
-              <li><a href="/get-started/quickstart/">Quickstart</a></li>
+        <li><a href="/get-started/quickstart-java/">Quickstart - Java</a></li>
+        <li><a href="/get-started/quickstart-py/">Quickstart - Python</a></li>
 			  <li role="separator" class="divider"></li>
 			  <li class="dropdown-header">Example Walkthroughs</li>
 			  <li><a href="/get-started/wordcount-example/">WordCount</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/9727633e/content/documentation/runners/spark/index.html
----------------------------------------------------------------------
diff --git a/content/documentation/runners/spark/index.html b/content/documentation/runners/spark/index.html
index cec31e7..4854673 100644
--- a/content/documentation/runners/spark/index.html
+++ b/content/documentation/runners/spark/index.html
@@ -52,7 +52,8 @@
 		  <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Get Started <span class="caret"></span></a>
 		  <ul class="dropdown-menu">
 			  <li><a href="/get-started/beam-overview/">Beam Overview</a></li>
-              <li><a href="/get-started/quickstart/">Quickstart</a></li>
+        <li><a href="/get-started/quickstart-java/">Quickstart - Java</a></li>
+        <li><a href="/get-started/quickstart-py/">Quickstart - Python</a></li>
 			  <li role="separator" class="divider"></li>
 			  <li class="dropdown-header">Example Walkthroughs</li>
 			  <li><a href="/get-started/wordcount-example/">WordCount</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/9727633e/content/documentation/sdks/java/index.html
----------------------------------------------------------------------
diff --git a/content/documentation/sdks/java/index.html b/content/documentation/sdks/java/index.html
index fee4321..22de4ae 100644
--- a/content/documentation/sdks/java/index.html
+++ b/content/documentation/sdks/java/index.html
@@ -52,7 +52,8 @@
 		  <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Get Started <span class="caret"></span></a>
 		  <ul class="dropdown-menu">
 			  <li><a href="/get-started/beam-overview/">Beam Overview</a></li>
-              <li><a href="/get-started/quickstart/">Quickstart</a></li>
+        <li><a href="/get-started/quickstart-java/">Quickstart - Java</a></li>
+        <li><a href="/get-started/quickstart-py/">Quickstart - Python</a></li>
 			  <li role="separator" class="divider"></li>
 			  <li class="dropdown-header">Example Walkthroughs</li>
 			  <li><a href="/get-started/wordcount-example/">WordCount</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/9727633e/content/documentation/sdks/python/index.html
----------------------------------------------------------------------
diff --git a/content/documentation/sdks/python/index.html b/content/documentation/sdks/python/index.html
index 2c97da8..f2955cd 100644
--- a/content/documentation/sdks/python/index.html
+++ b/content/documentation/sdks/python/index.html
@@ -52,7 +52,8 @@
 		  <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Get Started <span class="caret"></span></a>
 		  <ul class="dropdown-menu">
 			  <li><a href="/get-started/beam-overview/">Beam Overview</a></li>
-              <li><a href="/get-started/quickstart/">Quickstart</a></li>
+        <li><a href="/get-started/quickstart-java/">Quickstart - Java</a></li>
+        <li><a href="/get-started/quickstart-py/">Quickstart - Python</a></li>
 			  <li role="separator" class="divider"></li>
 			  <li class="dropdown-header">Example Walkthroughs</li>
 			  <li><a href="/get-started/wordcount-example/">WordCount</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/9727633e/content/get-started/beam-overview/index.html
----------------------------------------------------------------------
diff --git a/content/get-started/beam-overview/index.html b/content/get-started/beam-overview/index.html
index 1ae1f09..7cf3f44 100644
--- a/content/get-started/beam-overview/index.html
+++ b/content/get-started/beam-overview/index.html
@@ -52,7 +52,8 @@
 		  <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Get Started <span class="caret"></span></a>
 		  <ul class="dropdown-menu">
 			  <li><a href="/get-started/beam-overview/">Beam Overview</a></li>
-              <li><a href="/get-started/quickstart/">Quickstart</a></li>
+        <li><a href="/get-started/quickstart-java/">Quickstart - Java</a></li>
+        <li><a href="/get-started/quickstart-py/">Quickstart - Python</a></li>
 			  <li role="separator" class="divider"></li>
 			  <li class="dropdown-header">Example Walkthroughs</li>
 			  <li><a href="/get-started/wordcount-example/">WordCount</a></li>
@@ -209,7 +210,16 @@
 
 <h2 id="getting-started-with-apache-beam">Getting Started with Apache Beam</h2>
 
-<p>Get started using Beam for your data processing tasks by following the <a href="/get-started/quickstart">Quickstart</a> and the <a href="/get-started/wordcount-example">WordCount Examples Walkthrough</a>.</p>
+<p>Get started using Beam for your data processing tasks.</p>
+
+<ol>
+  <li>
+    <p>Follow the Quickstart for the <a href="/get-started/quickstart-java">Java SDK</a> or the <a href="/get-started/quickstart-py">Python SDK</a>.</p>
+  </li>
+  <li>
+    <p>See the <a href="/get-started/wordcount-example">WordCount Examples Walkthrough</a> for examples that introduce various features of the SDKs.</p>
+  </li>
+</ol>
 
       </div>
 

http://git-wip-us.apache.org/repos/asf/beam-site/blob/9727633e/content/get-started/downloads/index.html
----------------------------------------------------------------------
diff --git a/content/get-started/downloads/index.html b/content/get-started/downloads/index.html
index 9dc7978..514e452 100644
--- a/content/get-started/downloads/index.html
+++ b/content/get-started/downloads/index.html
@@ -52,7 +52,8 @@
 		  <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Get Started <span class="caret"></span></a>
 		  <ul class="dropdown-menu">
 			  <li><a href="/get-started/beam-overview/">Beam Overview</a></li>
-              <li><a href="/get-started/quickstart/">Quickstart</a></li>
+        <li><a href="/get-started/quickstart-java/">Quickstart - Java</a></li>
+        <li><a href="/get-started/quickstart-py/">Quickstart - Python</a></li>
 			  <li role="separator" class="divider"></li>
 			  <li class="dropdown-header">Example Walkthroughs</li>
 			  <li><a href="/get-started/wordcount-example/">WordCount</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/9727633e/content/get-started/index.html
----------------------------------------------------------------------
diff --git a/content/get-started/index.html b/content/get-started/index.html
index d05f179..76ba604 100644
--- a/content/get-started/index.html
+++ b/content/get-started/index.html
@@ -52,7 +52,8 @@
 		  <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Get Started <span class="caret"></span></a>
 		  <ul class="dropdown-menu">
 			  <li><a href="/get-started/beam-overview/">Beam Overview</a></li>
-              <li><a href="/get-started/quickstart/">Quickstart</a></li>
+        <li><a href="/get-started/quickstart-java/">Quickstart - Java</a></li>
+        <li><a href="/get-started/quickstart-py/">Quickstart - Python</a></li>
 			  <li role="separator" class="divider"></li>
 			  <li class="dropdown-header">Example Walkthroughs</li>
 			  <li><a href="/get-started/wordcount-example/">WordCount</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/9727633e/content/get-started/mobile-gaming-example/index.html
----------------------------------------------------------------------
diff --git a/content/get-started/mobile-gaming-example/index.html b/content/get-started/mobile-gaming-example/index.html
index 1506502..ced93cb 100644
--- a/content/get-started/mobile-gaming-example/index.html
+++ b/content/get-started/mobile-gaming-example/index.html
@@ -52,7 +52,8 @@
 		  <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Get Started <span class="caret"></span></a>
 		  <ul class="dropdown-menu">
 			  <li><a href="/get-started/beam-overview/">Beam Overview</a></li>
-              <li><a href="/get-started/quickstart/">Quickstart</a></li>
+        <li><a href="/get-started/quickstart-java/">Quickstart - Java</a></li>
+        <li><a href="/get-started/quickstart-py/">Quickstart - Python</a></li>
 			  <li role="separator" class="divider"></li>
 			  <li class="dropdown-header">Example Walkthroughs</li>
 			  <li><a href="/get-started/wordcount-example/">WordCount</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/9727633e/content/get-started/quickstart-java/index.html
----------------------------------------------------------------------
diff --git a/content/get-started/quickstart-java/index.html b/content/get-started/quickstart-java/index.html
new file mode 100644
index 0000000..751c35b
--- /dev/null
+++ b/content/get-started/quickstart-java/index.html
@@ -0,0 +1,395 @@
+<!DOCTYPE html>
+<html lang="en">
+
+  <head>
+  <meta charset="utf-8">
+  <meta http-equiv="X-UA-Compatible" content="IE=edge">
+  <meta name="viewport" content="width=device-width, initial-scale=1">
+
+  <title>Beam Quickstart for Java</title>
+  <meta name="description" content="Apache Beam is an open source, unified model and set of language-specific SDKs for defining and executing data processing workflows, and also data ingestion and integration flows, supporting Enterprise Integration Patterns (EIPs) and Domain Specific Languages (DSLs). Dataflow pipelines simplify the mechanics of large-scale batch and streaming data processing and can run on a number of runtimes like Apache Flink, Apache Spark, and Google Cloud Dataflow (a cloud service). Beam also brings DSL in different languages, allowing users to easily implement their data integration processes.
+">
+
+  <link rel="stylesheet" href="/styles/site.css">
+  <link rel="stylesheet" href="/css/theme.css">
+  <script src="https://ajax.googleapis.com/ajax/libs/jquery/2.2.0/jquery.min.js"></script>
+  <script src="/js/bootstrap.min.js"></script>
+  <script src="/js/language-switch.js"></script>
+  <link rel="canonical" href="http://beam.apache.org/get-started/quickstart-java/" data-proofer-ignore>
+  <link rel="alternate" type="application/rss+xml" title="Apache Beam" href="http://beam.apache.org/feed.xml">
+  <script>
+    (function(i,s,o,g,r,a,m){i['GoogleAnalyticsObject']=r;i[r]=i[r]||function(){
+    (i[r].q=i[r].q||[]).push(arguments)},i[r].l=1*new Date();a=s.createElement(o),
+    m=s.getElementsByTagName(o)[0];a.async=1;a.src=g;m.parentNode.insertBefore(a,m)
+    })(window,document,'script','//www.google-analytics.com/analytics.js','ga');
+
+    ga('create', 'UA-73650088-1', 'auto');
+    ga('send', 'pageview');
+
+  </script>
+  <link rel="shortcut icon" type="image/x-icon" href="/images/favicon.ico">
+</head>
+
+
+  <body role="document">
+
+    <nav class="navbar navbar-default navbar-fixed-top">
+  <div class="container">
+    <div class="navbar-header">
+      <a href="/" class="navbar-brand" >
+        <img alt="Brand" style="height: 25px" src="/images/beam_logo_navbar.png">
+      </a>
+      <button type="button" class="navbar-toggle collapsed" data-toggle="collapse" data-target="#navbar" aria-expanded="false" aria-controls="navbar">
+        <span class="sr-only">Toggle navigation</span>
+        <span class="icon-bar"></span>
+        <span class="icon-bar"></span>
+        <span class="icon-bar"></span>
+      </button>
+    </div>
+    <div id="navbar" class="navbar-collapse collapse">
+      <ul class="nav navbar-nav">
+        <li class="dropdown">
+		  <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Get Started <span class="caret"></span></a>
+		  <ul class="dropdown-menu">
+			  <li><a href="/get-started/beam-overview/">Beam Overview</a></li>
+        <li><a href="/get-started/quickstart-java/">Quickstart - Java</a></li>
+        <li><a href="/get-started/quickstart-py/">Quickstart - Python</a></li>
+			  <li role="separator" class="divider"></li>
+			  <li class="dropdown-header">Example Walkthroughs</li>
+			  <li><a href="/get-started/wordcount-example/">WordCount</a></li>
+			  <li><a href="/get-started/mobile-gaming-example/">Mobile Gaming</a></li>
+              <li role="separator" class="divider"></li>
+              <li class="dropdown-header">Resources</li>
+              <li><a href="/get-started/downloads">Downloads</a></li>
+              <li><a href="/get-started/support">Support</a></li>
+		  </ul>
+	    </li>
+        <li class="dropdown">
+		  <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Documentation <span class="caret"></span></a>
+		  <ul class="dropdown-menu">
+			  <li><a href="/documentation">Using the Documentation</a></li>
+			  <li role="separator" class="divider"></li>
+			  <li class="dropdown-header">Beam Concepts</li>
+			  <li><a href="/documentation/programming-guide/">Programming Guide</a></li>
+			  <li><a href="/documentation/resources/">Additional Resources</a></li>
+			  <li role="separator" class="divider"></li>
+              <li class="dropdown-header">Pipeline Fundamentals</li>
+              <li><a href="/documentation/pipelines/design-your-pipeline/">Design Your Pipeline</a></li>
+              <li><a href="/documentation/pipelines/create-your-pipeline/">Create Your Pipeline</a></li>
+              <li><a href="/documentation/pipelines/test-your-pipeline/">Test Your Pipeline</a></li>
+              <li role="separator" class="divider"></li>
+			  <li class="dropdown-header">SDKs</li>
+			  <li><a href="/documentation/sdks/java/">Java SDK</a></li>
+			  <li><a href="/documentation/sdks/javadoc/0.4.0/" target="_blank">Java SDK API Reference <img src="/images/external-link-icon.png"
+                 width="14" height="14"
+                 alt="External link."></a>
+        </li>
+        <li><a href="/documentation/sdks/python/">Python SDK</a></li>
+			  <li role="separator" class="divider"></li>
+			  <li class="dropdown-header">Runners</li>
+			  <li><a href="/documentation/runners/capability-matrix/">Capability Matrix</a></li>
+			  <li><a href="/documentation/runners/direct/">Direct Runner</a></li>
+			  <li><a href="/documentation/runners/apex/">Apache Apex Runner</a></li>
+			  <li><a href="/documentation/runners/flink/">Apache Flink Runner</a></li>
+			  <li><a href="/documentation/runners/spark/">Apache Spark Runner</a></li>
+			  <li><a href="/documentation/runners/dataflow/">Cloud Dataflow Runner</a></li>
+		  </ul>
+	    </li>
+        <li class="dropdown">
+		  <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Contribute <span class="caret"></span></a>
+		  <ul class="dropdown-menu">
+			  <li><a href="/contribute">Get Started Contributing</a></li>
+        <li role="separator" class="divider"></li>
+        <li class="dropdown-header">Guides</li>
+			  <li><a href="/contribute/contribution-guide/">Contribution Guide</a></li>
+        <li><a href="/contribute/testing/">Testing Guide</a></li>
+        <li><a href="/contribute/release-guide/">Release Guide</a></li>
+        <li><a href="/contribute/ptransform-style-guide/">PTransform Style Guide</a></li>
+        <li role="separator" class="divider"></li>
+        <li class="dropdown-header">Technical References</li>
+        <li><a href="/contribute/design-principles/">Design Principles</a></li>
+			  <li><a href="/contribute/work-in-progress/">Ongoing Projects</a></li>
+        <li><a href="/contribute/source-repository/">Source Repository</a></li>      
+        <li role="separator" class="divider"></li>
+			  <li class="dropdown-header">Promotion</li>
+        <li><a href="/contribute/presentation-materials/">Presentation Materials</a></li>
+        <li><a href="/contribute/logos/">Logos and Design</a></li>
+        <li role="separator" class="divider"></li>
+        <li><a href="/contribute/maturity-model/">Maturity Model</a></li>
+        <li><a href="/contribute/team/">Team</a></li>
+		  </ul>
+	    </li>
+
+        <li><a href="/blog">Blog</a></li>
+      </ul>
+      <ul class="nav navbar-nav navbar-right">
+        <li class="dropdown">
+          <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false"><img src="https://www.apache.org/foundation/press/kit/feather_small.png" alt="Apache Logo" style="height:24px;">Apache Software Foundation<span class="caret"></span></a>
+          <ul class="dropdown-menu dropdown-menu-right">
+            <li><a href="http://www.apache.org/">ASF Homepage</a></li>
+            <li><a href="http://www.apache.org/licenses/">License</a></li>
+            <li><a href="http://www.apache.org/security/">Security</a></li>
+            <li><a href="http://www.apache.org/foundation/thanks.html">Thanks</a></li>
+            <li><a href="http://www.apache.org/foundation/sponsorship.html">Sponsorship</a></li>
+            <li><a href="https://www.apache.org/foundation/policies/conduct">Code of Conduct</a></li>
+          </ul>
+        </li>
+      </ul>
+    </div><!--/.nav-collapse -->
+  </div>
+</nav>
+
+
+<link rel="stylesheet" href="">
+
+
+    <div class="container" role="main">
+
+      <div class="row">
+        <h1 id="apache-beam-java-sdk-quickstart">Apache Beam Java SDK Quickstart</h1>
+
+<p>This Quickstart will walk you through executing your first Beam pipeline to run <a href="/get-started/wordcount-example">WordCount</a>, written using Beam\u2019s <a href="/documentation/sdks/java">Java SDK</a>, on a <a href="/documentation#runners">runner</a> of your choice.</p>
+
+<ul id="markdown-toc">
+  <li><a href="#set-up-your-development-environment" id="markdown-toc-set-up-your-development-environment">Set up your Development Environment</a></li>
+  <li><a href="#get-the-wordcount-code" id="markdown-toc-get-the-wordcount-code">Get the WordCount Code</a></li>
+  <li><a href="#run-wordcount" id="markdown-toc-run-wordcount">Run WordCount</a></li>
+  <li><a href="#inspect-the-results" id="markdown-toc-inspect-the-results">Inspect the results</a></li>
+  <li><a href="#next-steps" id="markdown-toc-next-steps">Next Steps</a></li>
+</ul>
+
+<h2 id="set-up-your-development-environment">Set up your Development Environment</h2>
+
+<ol>
+  <li>
+    <p>Download and install the <a href="http://www.oracle.com/technetwork/java/javase/downloads/index.html">Java Development Kit (JDK)</a> version 1.7 or later. Verify that the <a href="https://docs.oracle.com/javase/8/docs/technotes/guides/troubleshoot/envvars001.html">JAVA_HOME</a> environment variable is set and points to your JDK installation.</p>
+  </li>
+  <li>
+    <p>Download and install <a href="http://maven.apache.org/download.cgi">Apache Maven</a> by following Maven\u2019s <a href="http://maven.apache.org/install.html">installation guide</a> for your specific operating system.</p>
+  </li>
+</ol>
+
+<h2 id="get-the-wordcount-code">Get the WordCount Code</h2>
+
+<p>The easiest way to get a copy of the WordCount pipeline is to use the following command to generate a simple Maven project that contains Beam\u2019s WordCount examples and builds against the most recent Beam release:</p>
+
+<div class="highlighter-rouge"><pre class="highlight"><code>$ mvn archetype:generate \
+      -DarchetypeRepository=https://repository.apache.org/content/groups/snapshots \
+      -DarchetypeGroupId=org.apache.beam \
+      -DarchetypeArtifactId=beam-sdks-java-maven-archetypes-examples \
+      -DarchetypeVersion=LATEST \
+      -DgroupId=org.example \
+      -DartifactId=word-count-beam \
+      -Dversion="0.1" \
+      -Dpackage=org.apache.beam.examples \
+      -DinteractiveMode=false
+</code></pre>
+</div>
+
+<p>This will create a directory <code class="highlighter-rouge">word-count-beam</code> that contains a simple <code class="highlighter-rouge">pom.xml</code> and a series of example pipelines that count words in text files.</p>
+
+<div class="highlighter-rouge"><pre class="highlight"><code>$ cd word-count-beam/
+
+$ ls
+pom.xml	src
+
+$ ls src/main/java/org/apache/beam/examples/
+DebuggingWordCount.java	WindowedWordCount.java	common
+MinimalWordCount.java	WordCount.java
+</code></pre>
+</div>
+
+<p>For a detailed introduction to the Beam concepts used in these examples, see the <a href="/get-started/wordcount-example">WordCount Example Walkthrough</a>. Here, we\u2019ll just focus on executing <code class="highlighter-rouge">WordCount.java</code>.</p>
+
+<h2 id="run-wordcount">Run WordCount</h2>
+
+<p>A single Beam pipeline can run on multiple Beam <a href="/documentation#runners">runners</a>, including the <a href="/documentation/runners/apex">ApexRunner</a>, <a href="/documentation/runners/flink">FlinkRunner</a>, <a href="/documentation/runners/spark">SparkRunner</a> or <a href="/documentation/runners/dataflow">DataflowRunner</a>. The <a href="/documentation/runners/direct">DirectRunner</a> is a common runner for getting started, as it runs locally on your machine and requires no specific setup.</p>
+
+<p>After you\u2019ve chosen which runner you\u2019d like to use:</p>
+
+<ol>
+  <li>Ensure you\u2019ve done any runner-specific setup.</li>
+  <li>Build your commandline by:
+    <ol>
+      <li>Specifying a specific runner with <code class="highlighter-rouge">--runner=&lt;runner&gt;</code> (defaults to the <a href="/documentation/runners/direct">DirectRunner</a>)</li>
+      <li>Adding any runner-specific required options</li>
+      <li>Choosing input files and an output location are accessible on the chosen runner. (For example, you can\u2019t access a local file if you are running the pipeline on an external cluster.)</li>
+    </ol>
+  </li>
+  <li>Run your first WordCount pipeline.</li>
+</ol>
+
+<div class="runner-direct highlighter-rouge"><pre class="highlight"><code>$ mvn compile exec:java -Dexec.mainClass=org.apache.beam.examples.WordCount \
+     -Dexec.args="--inputFile=pom.xml --output=counts" -Pdirect-runner
+</code></pre>
+</div>
+
+<div class="runner-apex highlighter-rouge"><pre class="highlight"><code>$ mvn compile exec:java -Dexec.mainClass=org.apache.beam.examples.WordCount \
+     -Dexec.args="--inputFile=pom.xml --output=counts --runner=ApexRunner" -Papex-runner
+</code></pre>
+</div>
+
+<div class="runner-flink-local highlighter-rouge"><pre class="highlight"><code>$ mvn compile exec:java -Dexec.mainClass=org.apache.beam.examples.WordCount \
+     -Dexec.args="--runner=FlinkRunner --inputFile=pom.xml --output=counts" -Pflink-runner
+</code></pre>
+</div>
+
+<div class="runner-flink-cluster highlighter-rouge"><pre class="highlight"><code>$ mvn package exec:java -Dexec.mainClass=org.apache.beam.examples.WordCount \
+     -Dexec.args="--runner=FlinkRunner --flinkMaster=&lt;flink master&gt; --filesToStage=target/word-count-beam-bundled-0.1.jar \
+                  --inputFile=/path/to/quickstart/pom.xml --output=/tmp/counts" -Pflink-runner
+
+You can monitor the running job by visiting the Flink dashboard at http://&lt;flink master&gt;:8081
+</code></pre>
+</div>
+
+<div class="runner-spark highlighter-rouge"><pre class="highlight"><code>$ mvn compile exec:java -Dexec.mainClass=org.apache.beam.examples.WordCount \
+     -Dexec.args="--runner=SparkRunner --inputFile=pom.xml --output=counts" -Pspark-runner
+</code></pre>
+</div>
+
+<div class="runner-dataflow highlighter-rouge"><pre class="highlight"><code>$ mvn compile exec:java -Dexec.mainClass=org.apache.beam.examples.WordCount \
+	 -Dexec.args="--runner=DataflowRunner --gcpTempLocation=gs://&lt;your-gcs-bucket&gt;/tmp \
+	              --inputFile=gs://apache-beam-samples/shakespeare/* --output=gs://&lt;your-gcs-bucket&gt;/counts" \
+     -Pdataflow-runner
+</code></pre>
+</div>
+
+<h2 id="inspect-the-results">Inspect the results</h2>
+
+<p>Once the pipeline has completed, you can view the output. You\u2019ll notice that there may be multiple output files prefixed by <code class="highlighter-rouge">count</code>. The exact number of these files is decided by the runner, giving it the flexibility to do efficient, distributed execution.</p>
+
+<div class="runner-direct highlighter-rouge"><pre class="highlight"><code>$ ls counts*
+</code></pre>
+</div>
+
+<div class="runner-apex highlighter-rouge"><pre class="highlight"><code>$ ls counts*
+</code></pre>
+</div>
+
+<div class="runner-flink-local highlighter-rouge"><pre class="highlight"><code>$ ls counts*
+</code></pre>
+</div>
+
+<div class="runner-flink-cluster highlighter-rouge"><pre class="highlight"><code>$ ls /tmp/counts*
+</code></pre>
+</div>
+
+<div class="runner-spark highlighter-rouge"><pre class="highlight"><code>$ ls counts*
+</code></pre>
+</div>
+
+<div class="runner-dataflow highlighter-rouge"><pre class="highlight"><code>$ gsutil ls gs://&lt;your-gcs-bucket&gt;/counts*
+</code></pre>
+</div>
+
+<p>When you look into the contents of the file, you\u2019ll see that they contain unique words and the number of occurrences of each word. The order of elements within the file may differ because the Beam model does not generally guarantee ordering, again to allow runners to optimize for efficiency.</p>
+
+<div class="runner-direct highlighter-rouge"><pre class="highlight"><code>$ more counts*
+api: 9
+bundled: 1
+old: 4
+Apache: 2
+The: 1
+limitations: 1
+Foundation: 1
+...
+</code></pre>
+</div>
+
+<div class="runner-apex highlighter-rouge"><pre class="highlight"><code>$ cat counts*
+BEAM: 1
+have: 1
+simple: 1
+skip: 4
+PAssert: 1
+...
+</code></pre>
+</div>
+
+<div class="runner-flink-local highlighter-rouge"><pre class="highlight"><code>$ more counts*
+The: 1
+api: 9
+old: 4
+Apache: 2
+limitations: 1
+bundled: 1
+Foundation: 1
+...
+</code></pre>
+</div>
+
+<div class="runner-flink-cluster highlighter-rouge"><pre class="highlight"><code>$ more /tmp/counts*
+The: 1
+api: 9
+old: 4
+Apache: 2
+limitations: 1
+bundled: 1
+Foundation: 1
+...
+</code></pre>
+</div>
+
+<div class="runner-spark highlighter-rouge"><pre class="highlight"><code>$ more counts*
+beam: 27
+SF: 1
+fat: 1
+job: 1
+limitations: 1
+require: 1
+of: 11
+profile: 10
+...
+</code></pre>
+</div>
+
+<div class="runner-dataflow highlighter-rouge"><pre class="highlight"><code>$ gsutil cat gs://&lt;your-gcs-bucket&gt;/counts*
+feature: 15
+smother'st: 1
+revelry: 1
+bashfulness: 1
+Bashful: 1
+Below: 2
+deserves: 32
+barrenly: 1
+...
+</code></pre>
+</div>
+
+<h2 id="next-steps">Next Steps</h2>
+
+<ul>
+  <li>Learn more about these WordCount examples in the <a href="/get-started/wordcount-example">WordCount Example Walkthrough</a>.</li>
+  <li>Dive in to some of our favorite <a href="/documentation/resources">articles and presentations</a>.</li>
+  <li>Join the Beam <a href="/get-started/support#mailing-lists">users@</a> mailing list.</li>
+</ul>
+
+<p>Please don\u2019t hesitate to <a href="/get-started/support">reach out</a> if you encounter any issues!</p>
+
+
+      </div>
+
+
+    <hr>
+  <div class="row">
+      <div class="col-xs-12">
+          <footer>
+              <p class="text-center">
+                &copy; Copyright
+                <a href="http://www.apache.org">The Apache Software Foundation</a>,
+                2017. All Rights Reserved.
+              </p>
+              <p class="text-center">
+                <a href="/privacy_policy">Privacy Policy</a> |
+                <a href="/feed.xml">RSS Feed</a>
+              </p>
+          </footer>
+      </div>
+  </div>
+  <!-- container div end -->
+</div>
+
+
+  </body>
+
+</html>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/9727633e/content/get-started/quickstart-py/index.html
----------------------------------------------------------------------
diff --git a/content/get-started/quickstart-py/index.html b/content/get-started/quickstart-py/index.html
new file mode 100644
index 0000000..948c41c
--- /dev/null
+++ b/content/get-started/quickstart-py/index.html
@@ -0,0 +1,288 @@
+<!DOCTYPE html>
+<html lang="en">
+
+  <head>
+  <meta charset="utf-8">
+  <meta http-equiv="X-UA-Compatible" content="IE=edge">
+  <meta name="viewport" content="width=device-width, initial-scale=1">
+
+  <title>Beam Quickstart for Python</title>
+  <meta name="description" content="Apache Beam is an open source, unified model and set of language-specific SDKs for defining and executing data processing workflows, and also data ingestion and integration flows, supporting Enterprise Integration Patterns (EIPs) and Domain Specific Languages (DSLs). Dataflow pipelines simplify the mechanics of large-scale batch and streaming data processing and can run on a number of runtimes like Apache Flink, Apache Spark, and Google Cloud Dataflow (a cloud service). Beam also brings DSL in different languages, allowing users to easily implement their data integration processes.
+">
+
+  <link rel="stylesheet" href="/styles/site.css">
+  <link rel="stylesheet" href="/css/theme.css">
+  <script src="https://ajax.googleapis.com/ajax/libs/jquery/2.2.0/jquery.min.js"></script>
+  <script src="/js/bootstrap.min.js"></script>
+  <script src="/js/language-switch.js"></script>
+  <link rel="canonical" href="http://beam.apache.org/get-started/quickstart-py/" data-proofer-ignore>
+  <link rel="alternate" type="application/rss+xml" title="Apache Beam" href="http://beam.apache.org/feed.xml">
+  <script>
+    (function(i,s,o,g,r,a,m){i['GoogleAnalyticsObject']=r;i[r]=i[r]||function(){
+    (i[r].q=i[r].q||[]).push(arguments)},i[r].l=1*new Date();a=s.createElement(o),
+    m=s.getElementsByTagName(o)[0];a.async=1;a.src=g;m.parentNode.insertBefore(a,m)
+    })(window,document,'script','//www.google-analytics.com/analytics.js','ga');
+
+    ga('create', 'UA-73650088-1', 'auto');
+    ga('send', 'pageview');
+
+  </script>
+  <link rel="shortcut icon" type="image/x-icon" href="/images/favicon.ico">
+</head>
+
+
+  <body role="document">
+
+    <nav class="navbar navbar-default navbar-fixed-top">
+  <div class="container">
+    <div class="navbar-header">
+      <a href="/" class="navbar-brand" >
+        <img alt="Brand" style="height: 25px" src="/images/beam_logo_navbar.png">
+      </a>
+      <button type="button" class="navbar-toggle collapsed" data-toggle="collapse" data-target="#navbar" aria-expanded="false" aria-controls="navbar">
+        <span class="sr-only">Toggle navigation</span>
+        <span class="icon-bar"></span>
+        <span class="icon-bar"></span>
+        <span class="icon-bar"></span>
+      </button>
+    </div>
+    <div id="navbar" class="navbar-collapse collapse">
+      <ul class="nav navbar-nav">
+        <li class="dropdown">
+		  <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Get Started <span class="caret"></span></a>
+		  <ul class="dropdown-menu">
+			  <li><a href="/get-started/beam-overview/">Beam Overview</a></li>
+        <li><a href="/get-started/quickstart-java/">Quickstart - Java</a></li>
+        <li><a href="/get-started/quickstart-py/">Quickstart - Python</a></li>
+			  <li role="separator" class="divider"></li>
+			  <li class="dropdown-header">Example Walkthroughs</li>
+			  <li><a href="/get-started/wordcount-example/">WordCount</a></li>
+			  <li><a href="/get-started/mobile-gaming-example/">Mobile Gaming</a></li>
+              <li role="separator" class="divider"></li>
+              <li class="dropdown-header">Resources</li>
+              <li><a href="/get-started/downloads">Downloads</a></li>
+              <li><a href="/get-started/support">Support</a></li>
+		  </ul>
+	    </li>
+        <li class="dropdown">
+		  <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Documentation <span class="caret"></span></a>
+		  <ul class="dropdown-menu">
+			  <li><a href="/documentation">Using the Documentation</a></li>
+			  <li role="separator" class="divider"></li>
+			  <li class="dropdown-header">Beam Concepts</li>
+			  <li><a href="/documentation/programming-guide/">Programming Guide</a></li>
+			  <li><a href="/documentation/resources/">Additional Resources</a></li>
+			  <li role="separator" class="divider"></li>
+              <li class="dropdown-header">Pipeline Fundamentals</li>
+              <li><a href="/documentation/pipelines/design-your-pipeline/">Design Your Pipeline</a></li>
+              <li><a href="/documentation/pipelines/create-your-pipeline/">Create Your Pipeline</a></li>
+              <li><a href="/documentation/pipelines/test-your-pipeline/">Test Your Pipeline</a></li>
+              <li role="separator" class="divider"></li>
+			  <li class="dropdown-header">SDKs</li>
+			  <li><a href="/documentation/sdks/java/">Java SDK</a></li>
+			  <li><a href="/documentation/sdks/javadoc/0.4.0/" target="_blank">Java SDK API Reference <img src="/images/external-link-icon.png"
+                 width="14" height="14"
+                 alt="External link."></a>
+        </li>
+        <li><a href="/documentation/sdks/python/">Python SDK</a></li>
+			  <li role="separator" class="divider"></li>
+			  <li class="dropdown-header">Runners</li>
+			  <li><a href="/documentation/runners/capability-matrix/">Capability Matrix</a></li>
+			  <li><a href="/documentation/runners/direct/">Direct Runner</a></li>
+			  <li><a href="/documentation/runners/apex/">Apache Apex Runner</a></li>
+			  <li><a href="/documentation/runners/flink/">Apache Flink Runner</a></li>
+			  <li><a href="/documentation/runners/spark/">Apache Spark Runner</a></li>
+			  <li><a href="/documentation/runners/dataflow/">Cloud Dataflow Runner</a></li>
+		  </ul>
+	    </li>
+        <li class="dropdown">
+		  <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Contribute <span class="caret"></span></a>
+		  <ul class="dropdown-menu">
+			  <li><a href="/contribute">Get Started Contributing</a></li>
+        <li role="separator" class="divider"></li>
+        <li class="dropdown-header">Guides</li>
+			  <li><a href="/contribute/contribution-guide/">Contribution Guide</a></li>
+        <li><a href="/contribute/testing/">Testing Guide</a></li>
+        <li><a href="/contribute/release-guide/">Release Guide</a></li>
+        <li><a href="/contribute/ptransform-style-guide/">PTransform Style Guide</a></li>
+        <li role="separator" class="divider"></li>
+        <li class="dropdown-header">Technical References</li>
+        <li><a href="/contribute/design-principles/">Design Principles</a></li>
+			  <li><a href="/contribute/work-in-progress/">Ongoing Projects</a></li>
+        <li><a href="/contribute/source-repository/">Source Repository</a></li>      
+        <li role="separator" class="divider"></li>
+			  <li class="dropdown-header">Promotion</li>
+        <li><a href="/contribute/presentation-materials/">Presentation Materials</a></li>
+        <li><a href="/contribute/logos/">Logos and Design</a></li>
+        <li role="separator" class="divider"></li>
+        <li><a href="/contribute/maturity-model/">Maturity Model</a></li>
+        <li><a href="/contribute/team/">Team</a></li>
+		  </ul>
+	    </li>
+
+        <li><a href="/blog">Blog</a></li>
+      </ul>
+      <ul class="nav navbar-nav navbar-right">
+        <li class="dropdown">
+          <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false"><img src="https://www.apache.org/foundation/press/kit/feather_small.png" alt="Apache Logo" style="height:24px;">Apache Software Foundation<span class="caret"></span></a>
+          <ul class="dropdown-menu dropdown-menu-right">
+            <li><a href="http://www.apache.org/">ASF Homepage</a></li>
+            <li><a href="http://www.apache.org/licenses/">License</a></li>
+            <li><a href="http://www.apache.org/security/">Security</a></li>
+            <li><a href="http://www.apache.org/foundation/thanks.html">Thanks</a></li>
+            <li><a href="http://www.apache.org/foundation/sponsorship.html">Sponsorship</a></li>
+            <li><a href="https://www.apache.org/foundation/policies/conduct">Code of Conduct</a></li>
+          </ul>
+        </li>
+      </ul>
+    </div><!--/.nav-collapse -->
+  </div>
+</nav>
+
+
+<link rel="stylesheet" href="">
+
+
+    <div class="container" role="main">
+
+      <div class="row">
+        <h1 id="apache-beam-python-sdk-quickstart">Apache Beam Python SDK Quickstart</h1>
+
+<p>This guide shows you how to set up your Python development environment, get the Apache Beam SDK for Python, and run an example pipeline.</p>
+
+<ul id="markdown-toc">
+  <li><a href="#set-up-your-environment" id="markdown-toc-set-up-your-environment">Set up your environment</a>    <ul>
+      <li><a href="#install-pip" id="markdown-toc-install-pip">Install pip</a></li>
+      <li><a href="#install-python-virtual-environment" id="markdown-toc-install-python-virtual-environment">Install Python virtual environment</a></li>
+    </ul>
+  </li>
+  <li><a href="#get-apache-beam" id="markdown-toc-get-apache-beam">Get Apache Beam</a>    <ul>
+      <li><a href="#create-and-activate-a-virtual-environment" id="markdown-toc-create-and-activate-a-virtual-environment">Create and activate a virtual environment</a></li>
+      <li><a href="#download-and-install" id="markdown-toc-download-and-install">Download and install</a></li>
+    </ul>
+  </li>
+  <li><a href="#execute-a-pipeline-locally" id="markdown-toc-execute-a-pipeline-locally">Execute a pipeline locally</a></li>
+  <li><a href="#next-steps" id="markdown-toc-next-steps">Next Steps</a></li>
+</ul>
+
+<h2 id="set-up-your-environment">Set up your environment</h2>
+
+<h3 id="install-pip">Install pip</h3>
+
+<p>Install <a href="https://pip.pypa.io/en/stable/installing/">pip</a>, Python\u2019s package manager. Check that you have version 7.0.0 or newer, by running:</p>
+
+<div class="highlighter-rouge"><pre class="highlight"><code>pip --version
+</code></pre>
+</div>
+
+<h3 id="install-python-virtual-environment">Install Python virtual environment</h3>
+
+<p>It is recommended that you install a <a href="http://docs.python-guide.org/en/latest/dev/virtualenvs/">Python virtual environment</a>
+for initial experiments. If you do not have <code class="highlighter-rouge">virtualenv</code> version 13.1.0 or newer, install it by running:</p>
+
+<div class="highlighter-rouge"><pre class="highlight"><code>pip install --upgrade virtualenv
+</code></pre>
+</div>
+
+<p>If you do not want to use a Python virtual environment (not recommended), ensure <code class="highlighter-rouge">setuptools</code> is installed on your machine. If you do not have <code class="highlighter-rouge">setuptools</code> version 17.1 or newer, install it by running:</p>
+
+<div class="highlighter-rouge"><pre class="highlight"><code>pip install --upgrade setuptools
+</code></pre>
+</div>
+
+<h2 id="get-apache-beam">Get Apache Beam</h2>
+
+<h3 id="create-and-activate-a-virtual-environment">Create and activate a virtual environment</h3>
+
+<p>A virtual environment is a directory tree containing its own Python distribution. To create a virtual environment, create a directory and run:</p>
+
+<div class="highlighter-rouge"><pre class="highlight"><code>virtualenv /path/to/directory
+</code></pre>
+</div>
+
+<p>A virtual environment needs to be activated for each shell that is to use it.
+Activating it sets some environment variables that point to the virtual
+environment\u2019s directories.</p>
+
+<p>To activate a virtual environment in Bash, run:</p>
+
+<div class="highlighter-rouge"><pre class="highlight"><code>. /path/to/directory/bin/activate
+</code></pre>
+</div>
+
+<p>That is, source the script <code class="highlighter-rouge">bin/activate</code> under the virtual environment directory you created.</p>
+
+<p>For instructions using other shells, see the <a href="https://virtualenv.pypa.io/en/stable/userguide/#activate-script">virtualenv documentation</a>.</p>
+
+<h3 id="download-and-install">Download and install</h3>
+
+<ol>
+  <li>
+    <p>Clone the Apache Beam repo from GitHub: 
+  <code class="highlighter-rouge">git clone https://github.com/apache/beam.git --branch python-sdk</code></p>
+  </li>
+  <li>
+    <p>Navigate to the <code class="highlighter-rouge">python</code> directory: 
+  <code class="highlighter-rouge">cd beam/sdks/python/</code></p>
+  </li>
+  <li>
+    <p>Create the Apache Beam Python SDK installation package: 
+  <code class="highlighter-rouge">python setup.py sdist</code></p>
+  </li>
+  <li>
+    <p>Navigate to the <code class="highlighter-rouge">dist</code> directory:
+  <code class="highlighter-rouge">cd dist/</code></p>
+  </li>
+  <li>
+    <p>Install the Apache Beam SDK
+  <code class="highlighter-rouge">pip install apache-beam-sdk-*.tar.gz</code></p>
+  </li>
+</ol>
+
+<h2 id="execute-a-pipeline-locally">Execute a pipeline locally</h2>
+
+<p>The Apache Beam <a href="https://github.com/apache/beam/tree/python-sdk/sdks/python/apache_beam/examples">examples</a> directory has many examples. All examples can be run locally by passing the required arguments described in the example script.</p>
+
+<p>For example, to run <code class="highlighter-rouge">wordcount.py</code>, run:</p>
+
+<div class="highlighter-rouge"><pre class="highlight"><code>python -m apache_beam.examples.wordcount --input gs://dataflow-samples/shakespeare/kinglear.txt --output output.txt
+</code></pre>
+</div>
+
+<h2 id="next-steps">Next Steps</h2>
+
+<ul>
+  <li>Learn more about these WordCount examples in the <a href="/get-started/wordcount-example">WordCount Example Walkthrough</a>.</li>
+  <li>Dive in to some of our favorite <a href="/documentation/resources">articles and presentations</a>.</li>
+  <li>Join the Beam <a href="/get-started/support#mailing-lists">users@</a> mailing list.</li>
+</ul>
+
+<p>Please don\u2019t hesitate to <a href="/get-started/support">reach out</a> if you encounter any issues!</p>
+
+
+      </div>
+
+
+    <hr>
+  <div class="row">
+      <div class="col-xs-12">
+          <footer>
+              <p class="text-center">
+                &copy; Copyright
+                <a href="http://www.apache.org">The Apache Software Foundation</a>,
+                2017. All Rights Reserved.
+              </p>
+              <p class="text-center">
+                <a href="/privacy_policy">Privacy Policy</a> |
+                <a href="/feed.xml">RSS Feed</a>
+              </p>
+          </footer>
+      </div>
+  </div>
+  <!-- container div end -->
+</div>
+
+
+  </body>
+
+</html>


[5/5] beam-site git commit: This closes #130

Posted by da...@apache.org.
This closes #130


Project: http://git-wip-us.apache.org/repos/asf/beam-site/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam-site/commit/b81afa39
Tree: http://git-wip-us.apache.org/repos/asf/beam-site/tree/b81afa39
Diff: http://git-wip-us.apache.org/repos/asf/beam-site/diff/b81afa39

Branch: refs/heads/asf-site
Commit: b81afa390b4efc3f82fb6856fcbf293c13b6ca59
Parents: 2f4d86d 9727633
Author: Davor Bonaci <da...@google.com>
Authored: Mon Jan 30 17:36:48 2017 -0800
Committer: Davor Bonaci <da...@google.com>
Committed: Mon Jan 30 17:36:48 2017 -0800

----------------------------------------------------------------------
 .../2016/03/17/capability-matrix.html           |   3 +-
 .../2016/04/03/presentation-materials.html      |   3 +-
 .../sdk/2016/02/25/python-sdk-now-public.html   |   3 +-
 .../beam/release/2016/06/15/first-release.html  |   3 +-
 .../10/11/strata-hadoop-world-and-beam.html     |   3 +-
 .../website/2016/02/22/beam-has-a-logo.html     |   3 +-
 .../blog/2016/05/18/splitAtFraction-method.html |   3 +-
 .../05/27/where-is-my-pcollection-dot-map.html  |   3 +-
 .../06/13/flink-batch-runner-milestone.html     |   3 +-
 content/blog/2016/08/03/six-months.html         |   3 +-
 content/blog/2016/10/20/test-stream.html        |   3 +-
 content/blog/2017/01/09/added-apex-runner.html  |   3 +-
 content/blog/2017/01/10/beam-graduates.html     |   3 +-
 content/blog/index.html                         |   3 +-
 content/coming-soon.html                        |   3 +-
 .../contribute/contribution-guide/index.html    |   3 +-
 content/contribute/design-principles/index.html |   3 +-
 content/contribute/index.html                   |   3 +-
 content/contribute/logos/index.html             |   3 +-
 content/contribute/maturity-model/index.html    |   3 +-
 .../presentation-materials/index.html           |   3 +-
 .../ptransform-style-guide/index.html           |   3 +-
 content/contribute/release-guide/index.html     |   3 +-
 content/contribute/source-repository/index.html |   3 +-
 content/contribute/team/index.html              |   3 +-
 content/contribute/testing/index.html           |   3 +-
 content/contribute/work-in-progress/index.html  |   3 +-
 content/documentation/index.html                |   3 +-
 .../pipelines/create-your-pipeline/index.html   |   3 +-
 .../pipelines/design-your-pipeline/index.html   |   3 +-
 .../pipelines/test-your-pipeline/index.html     |   3 +-
 .../documentation/programming-guide/index.html  |   3 +-
 content/documentation/resources/index.html      |   3 +-
 content/documentation/runners/apex/index.html   |   3 +-
 .../runners/capability-matrix/index.html        |   3 +-
 .../documentation/runners/dataflow/index.html   |   3 +-
 content/documentation/runners/direct/index.html |   3 +-
 content/documentation/runners/flink/index.html  |   3 +-
 content/documentation/runners/spark/index.html  |   3 +-
 content/documentation/sdks/java/index.html      |   3 +-
 content/documentation/sdks/python/index.html    |   3 +-
 content/get-started/beam-overview/index.html    |  14 +-
 content/get-started/downloads/index.html        |   3 +-
 content/get-started/index.html                  |   3 +-
 .../mobile-gaming-example/index.html            |   3 +-
 content/get-started/quickstart-java/index.html  | 395 ++++++++++++++++++
 content/get-started/quickstart-py/index.html    | 288 +++++++++++++
 content/get-started/quickstart/index.html       | 400 +------------------
 content/get-started/support/index.html          |   3 +-
 .../get-started/wordcount-example/index.html    | 205 ++++++++--
 content/getting-started/index.html              |   8 +-
 content/index.html                              |   3 +-
 content/privacy_policy/index.html               |   3 +-
 content/use/quickstart/index.html               |   8 +-
 src/_includes/header.html                       |   3 +-
 src/get-started/beam-overview.md                |   6 +-
 src/get-started/quickstart-java.md              | 236 +++++++++++
 src/get-started/quickstart-py.md                |  97 +++++
 src/get-started/quickstart.md                   | 235 -----------
 src/get-started/wordcount-example.md            | 163 ++++++--
 60 files changed, 1444 insertions(+), 755 deletions(-)
----------------------------------------------------------------------