You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by ke...@apache.org on 2017/11/07 16:54:17 UTC

[01/50] [abbrv] beam git commit: Changed the mutation detector to be based on structural value only

Repository: beam
Updated Branches:
  refs/heads/mr-runner 5fa0b14d2 -> b6f22aa76


Changed the mutation detector to be based on structural value only


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

Branch: refs/heads/mr-runner
Commit: e3f6d6f1f0c1f9c9ca00ade17c4afedb7d3fef6b
Parents: 239319b
Author: Innocent Djiofack <dj...@gmail.com>
Authored: Tue Jul 25 23:41:02 2017 -0400
Committer: Luke Cwik <lc...@google.com>
Committed: Wed Nov 1 10:40:43 2017 -0700

----------------------------------------------------------------------
 .../org/apache/beam/sdk/util/CoderUtils.java    |  2 +-
 .../apache/beam/sdk/util/MutationDetectors.java | 79 +++++++-------------
 .../beam/sdk/util/MutationDetectorsTest.java    | 56 ++++++++++++++
 3 files changed, 85 insertions(+), 52 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/e3f6d6f1/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CoderUtils.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CoderUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CoderUtils.java
index da77829..cfd8fde 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CoderUtils.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CoderUtils.java
@@ -153,7 +153,7 @@ public final class CoderUtils {
    * {@link Coder}.
    */
   public static <T> T clone(Coder<T> coder, T value) throws CoderException {
-    return decodeFromByteArray(coder, encodeToByteArray(coder, value, Coder.Context.OUTER));
+    return decodeFromByteArray(coder, encodeToByteArray(coder, value));
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/beam/blob/e3f6d6f1/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MutationDetectors.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MutationDetectors.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MutationDetectors.java
index 3b593bf..3556667 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MutationDetectors.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MutationDetectors.java
@@ -17,8 +17,6 @@
  */
 package org.apache.beam.sdk.util;
 
-import java.util.Arrays;
-import java.util.Objects;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.CoderException;
 
@@ -30,12 +28,12 @@ public class MutationDetectors {
   private MutationDetectors() {}
 
   /**
-     * Creates a new {@code MutationDetector} for the provided {@code value} that uses the provided
-     * {@link Coder} to perform deep copies and comparisons by serializing and deserializing values.
-     *
-     * <p>It is permissible for {@code value} to be {@code null}. Since {@code null} is immutable,
-     * the mutation check will always succeed.
-     */
+   * Creates a new {@code MutationDetector} for the provided {@code value} that uses the provided
+   * {@link Coder} to perform deep copies and comparisons by serializing and deserializing values.
+   *
+   * <p>It is permissible for {@code value} to be {@code null}. Since {@code null} is immutable,
+   * the mutation check will always succeed.
+   */
   public static <T> MutationDetector forValueWithCoder(T value, Coder<T> coder)
       throws CoderException {
     if (value == null) {
@@ -59,7 +57,6 @@ public class MutationDetectors {
    * A {@link MutationDetector} for {@code null}, which is immutable.
    */
   private static class NoopMutationDetector implements MutationDetector {
-
     @Override
     public void verifyUnmodified() { }
 
@@ -76,6 +73,7 @@ public class MutationDetectors {
   private static class CodedValueMutationDetector<T> implements MutationDetector {
 
     private final Coder<T> coder;
+    private final T clonedOriginalValue;
 
     /**
      * A saved pointer to an in-memory value provided upon construction, which we will check for
@@ -97,11 +95,23 @@ public class MutationDetectors {
     private final T clonedOriginalObject;
 
     /**
+     * The structural value from {@link #possiblyModifiedObject}. It will be used during every call
+     * to {@link #verifyUnmodified}, which could be called many times throughout the lifetime of
+     * this {@link CodedValueMutationDetector}.
+     */
+    private final Object originalStructuralValue;
+
+    /**
      * Create a mutation detector for the provided {@code value}, using the provided {@link Coder}
      * for cloning and checking serialized forms for equality.
      */
     public CodedValueMutationDetector(T value, Coder<T> coder) throws CoderException {
       this.coder = coder;
+      // We need to clone the original value before getting it's structural value.
+      // If the object is consistent with equals, the Structural value will be the exact
+      // same object reference making it impossible to detect changes.
+      clonedOriginalValue = CoderUtils.clone(coder, value);
+      this.originalStructuralValue = coder.structuralValue(clonedOriginalValue);
       this.possiblyModifiedObject = value;
       this.encodedOriginalObject = CoderUtils.encodeToByteArray(coder, value);
       this.clonedOriginalObject = CoderUtils.decodeFromByteArray(coder, encodedOriginalObject);
@@ -117,49 +127,16 @@ public class MutationDetectors {
     }
 
     private void verifyUnmodifiedThrowingCheckedExceptions() throws CoderException {
-      // If either object believes they are equal, we trust that and short-circuit deeper checks.
-      if (Objects.equals(possiblyModifiedObject, clonedOriginalObject)
-          || Objects.equals(clonedOriginalObject, possiblyModifiedObject)) {
-        return;
-      }
-
-      // Since retainedObject is in general an instance of a subclass of T, when it is cloned to
-      // clonedObject using a Coder<T>, the two will generally be equivalent viewed as a T, but in
-      // general neither retainedObject.equals(clonedObject) nor clonedObject.equals(retainedObject)
-      // will hold.
-      //
-      // For example, CoderUtils.clone(IterableCoder<Integer>, IterableSubclass<Integer>) will
-      // produce an ArrayList<Integer> with the same contents as the IterableSubclass, but the
-      // latter will quite reasonably not consider itself equivalent to an ArrayList (and vice
-      // versa).
-      //
-      // To enable a reasonable comparison, we clone retainedObject again here, converting it to
-      // the same sort of T that the Coder<T> output when it created clonedObject.
-      T clonedPossiblyModifiedObject = CoderUtils.clone(coder, possiblyModifiedObject);
-
-      // If deepEquals() then we trust the equals implementation.
-      // This deliberately allows fields to escape this check.
-      if (Objects.deepEquals(clonedPossiblyModifiedObject, clonedOriginalObject)) {
-        return;
+      // Since there is no guarantee that cloning an object via the coder will
+      // return the exact same type as value, We are cloning the possiblyModifiedObject
+      // before getting it's structural value. This way we are guaranteed to compare the same
+      // types.
+      T possiblyModifiedClonedValue = CoderUtils.clone(coder, possiblyModifiedObject);
+      Object newStructuralValue = coder.structuralValue(possiblyModifiedClonedValue);
+      if (originalStructuralValue.equals(newStructuralValue)) {
+          return;
       }
-
-      // If not deepEquals(), the class may just have a poor equals() implementation.
-      // So we next try checking their serialized forms. We re-serialize instead of checking
-      // encodedObject, because the Coder may treat it differently.
-      //
-      // For example, an unbounded Iterable will be encoded in an unbounded way, but decoded into an
-      // ArrayList, which will then be re-encoded in a bounded format. So we really do need to
-      // encode-decode-encode retainedObject.
-      if (Arrays.equals(
-          CoderUtils.encodeToByteArray(coder, clonedOriginalObject),
-          CoderUtils.encodeToByteArray(coder, clonedPossiblyModifiedObject))) {
-        return;
-      }
-
-      // If we got here, then they are not deepEquals() and do not have deepEquals() encodings.
-      // Even if there is some conceptual sense in which the objects are equivalent, it has not
-      // been adequately expressed in code.
-      illegalMutation(clonedOriginalObject, clonedPossiblyModifiedObject);
+      illegalMutation(clonedOriginalObject, possiblyModifiedClonedValue);
     }
 
     private void illegalMutation(T previousValue, T newValue) throws CoderException {

http://git-wip-us.apache.org/repos/asf/beam/blob/e3f6d6f1/sdks/java/core/src/test/java/org/apache/beam/sdk/util/MutationDetectorsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/MutationDetectorsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/MutationDetectorsTest.java
index ebd8297..29e727b 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/MutationDetectorsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/MutationDetectorsTest.java
@@ -20,11 +20,17 @@ package org.apache.beam.sdk.util;
 import com.google.common.collect.FluentIterable;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 import java.util.Arrays;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.beam.sdk.coders.AtomicCoder;
 import org.apache.beam.sdk.coders.ByteArrayCoder;
+import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.IterableCoder;
 import org.apache.beam.sdk.coders.ListCoder;
 import org.apache.beam.sdk.coders.VarIntCoder;
@@ -39,10 +45,48 @@ import org.junit.runners.JUnit4;
  */
 @RunWith(JUnit4.class)
 public class MutationDetectorsTest {
+  /**
+   * Solely used to test that immutability is enforced from the SDK's perspective and not from
+   * Java's {@link Object#equals} method. Note that we do not expect users to create such
+   * an implementation.
+   */
+  private class ForSDKMutationDetectionTestCoder extends AtomicCoder<Object> {
+    // Use a unique instance that is returned as the structural value making all structural
+    // values of this coder equivalent to each other.
+    private final Object uniqueInstance = new Object();
+
+    @Override
+    public void encode(Object value, OutputStream outStream) throws  IOException {
+    }
+
+    @Override
+    public Object decode(InputStream inStream) throws  IOException {
+      return new AtomicInteger();
+    }
+
+    @Override
+    public Object structuralValue(Object value) {
+      return uniqueInstance;
+    }
+  }
 
   @Rule public ExpectedException thrown = ExpectedException.none();
 
   /**
+   * Tests that mutation detection is enforced from the SDK point of view
+   * (Based on the {@link Coder#structuralValue}) and not from the Java's equals method.
+   */
+  @Test
+  public void testMutationBasedOnStructuralValue() throws Exception {
+    AtomicInteger value = new AtomicInteger();
+    MutationDetector detector =
+        MutationDetectors.forValueWithCoder(value, new ForSDKMutationDetectionTestCoder());
+    // Even though we modified the value, we are relying on the fact that the structural
+    // value will be used to compare equality
+    value.incrementAndGet();
+    detector.verifyUnmodified();
+  }
+  /**
    * Tests that {@link MutationDetectors#forValueWithCoder} detects a mutation to a list.
    */
   @Test
@@ -93,6 +137,18 @@ public class MutationDetectorsTest {
   }
 
   /**
+   * Tests that {@link MutationDetectors#forValueWithCoder} does not false positive on a
+   * {@link Set} coded as an {@link Iterable}.
+   */
+  @Test
+  public void testStructuralValue() throws Exception {
+    Set<Integer> value = Sets.newHashSet(Arrays.asList(1, 2, 3, 4));
+    MutationDetector detector =
+            MutationDetectors.forValueWithCoder(value, IterableCoder.of(VarIntCoder.of()));
+    detector.verifyUnmodified();
+  }
+
+  /**
    * Tests that {@link MutationDetectors#forValueWithCoder} does not false positive on an
    * {@link Iterable} that is not known to be bounded; after coder-based cloning the bound
    * will be known and it will be a {@link List} so it will encode more compactly the second


[28/50] [abbrv] beam git commit: This closes #4053

Posted by ke...@apache.org.
This closes #4053


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

Branch: refs/heads/mr-runner
Commit: 766b4f3973044bc5e8546a030bf3816b5446b9b1
Parents: e25d217 fcd864a
Author: Thomas Groh <tg...@google.com>
Authored: Mon Nov 6 11:02:27 2017 -0800
Committer: Thomas Groh <tg...@google.com>
Committed: Mon Nov 6 11:02:27 2017 -0800

----------------------------------------------------------------------
 .../beam/model/fnexecution_v1/beam_fn_api.pb.go | 2729 ++++++++++++++
 .../fnexecution_v1/beam_provision_api.pb.go     |   57 +-
 sdks/go/pkg/beam/model/gen.go                   |    5 +-
 .../jobmanagement_v1/beam_artifact_api.pb.go    |   68 +-
 .../model/jobmanagement_v1/beam_job_api.pb.go   |  903 +++++
 .../model/pipeline_v1/beam_runner_api.pb.go     | 3491 ++++++++++++++++++
 .../pkg/beam/model/pipeline_v1/endpoints.pb.go  |  160 +
 .../model/pipeline_v1/standard_window_fns.pb.go |  120 +
 sdks/go/pkg/beam/util/syscallx/syscall.go       |    7 +-
 sdks/java/container/boot.go                     |   12 +-
 sdks/python/container/boot.go                   |   31 +-
 11 files changed, 7459 insertions(+), 124 deletions(-)
----------------------------------------------------------------------



[25/50] [abbrv] beam git commit: Add a runners/java-fn-execution module

Posted by ke...@apache.org.
Add a runners/java-fn-execution module

This contains libraries for runner authors to create Fn API services and
RPCs.


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

Branch: refs/heads/mr-runner
Commit: 927a8db1397bc43c6cb253d6ca856afdbfa472a3
Parents: fdd5971
Author: Thomas Groh <tg...@google.com>
Authored: Wed Oct 11 18:47:51 2017 -0700
Committer: Thomas Groh <tg...@google.com>
Committed: Thu Nov 2 18:25:12 2017 -0700

----------------------------------------------------------------------
 pom.xml                                         |   6 +
 runners/java-fn-execution/pom.xml               | 105 +++++++++++++++
 .../beam/runners/fnexecution/ServerFactory.java | 104 +++++++++++++++
 .../beam/runners/fnexecution/package-info.java  |  23 ++++
 .../runners/fnexecution/ServerFactoryTest.java  | 128 +++++++++++++++++++
 runners/pom.xml                                 |   1 +
 6 files changed, 367 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/927a8db1/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index f1eee91..b2ab5d7 100644
--- a/pom.xml
+++ b/pom.xml
@@ -684,6 +684,12 @@
 
       <dependency>
         <groupId>org.apache.beam</groupId>
+        <artifactId>beam-runners-java-fn-execution</artifactId>
+        <version>${project.version}</version>
+      </dependency>
+
+      <dependency>
+        <groupId>org.apache.beam</groupId>
         <artifactId>beam-runners-reference-job-orchestrator</artifactId>
         <version>${project.version}</version>
       </dependency>

http://git-wip-us.apache.org/repos/asf/beam/blob/927a8db1/runners/java-fn-execution/pom.xml
----------------------------------------------------------------------
diff --git a/runners/java-fn-execution/pom.xml b/runners/java-fn-execution/pom.xml
new file mode 100644
index 0000000..bd4fcf0
--- /dev/null
+++ b/runners/java-fn-execution/pom.xml
@@ -0,0 +1,105 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+    Licensed to the Apache Software Foundation (ASF) under one or more
+    contributor license agreements.  See the NOTICE file distributed with
+    this work for additional information regarding copyright ownership.
+    The ASF licenses this file to You under the Apache License, Version 2.0
+    (the "License"); you may not use this file except in compliance with
+    the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing, software
+    distributed under the License is distributed on an "AS IS" BASIS,
+    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+    See the License for the specific language governing permissions and
+    limitations under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+
+  <modelVersion>4.0.0</modelVersion>
+
+  <parent>
+    <groupId>org.apache.beam</groupId>
+    <artifactId>beam-runners-parent</artifactId>
+    <version>2.3.0-SNAPSHOT</version>
+    <relativePath>../pom.xml</relativePath>
+  </parent>
+
+  <artifactId>beam-runners-java-fn-execution</artifactId>
+
+  <name>Apache Beam :: Runners :: Java Fn Execution</name>
+
+  <packaging>jar</packaging>
+
+  <build>
+    <plugins>
+      <plugin>
+        <!--  Override Beam parent to allow Java8 -->
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-compiler-plugin</artifactId>
+        <configuration>
+          <source>1.8</source>
+          <target>1.8</target>
+        </configuration>
+      </plugin>
+    </plugins>
+  </build>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>beam-model-pipeline</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>beam-model-fn-execution</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>beam-sdks-java-fn-execution</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-core</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-stub</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-netty</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+
+    <!-- Test dependencies -->
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.hamcrest</groupId>
+      <artifactId>hamcrest-all</artifactId>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>beam-sdks-java-fn-execution</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/beam/blob/927a8db1/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/ServerFactory.java
----------------------------------------------------------------------
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/ServerFactory.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/ServerFactory.java
new file mode 100644
index 0000000..918672a
--- /dev/null
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/ServerFactory.java
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.fnexecution;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import com.google.common.net.HostAndPort;
+import io.grpc.BindableService;
+import io.grpc.Server;
+import io.grpc.netty.NettyServerBuilder;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+import org.apache.beam.harness.channel.SocketAddressFactory;
+import org.apache.beam.model.pipeline.v1.Endpoints;
+
+/**
+ * A {@link Server gRPC server} factory.
+ */
+public abstract class ServerFactory {
+  /**
+   * Create a default {@link ServerFactory}.
+   */
+  public static ServerFactory createDefault() {
+    return new InetSocketAddressServerFactory();
+  }
+
+  /**
+   * Creates an instance of this server using an ephemeral port chosen automatically. The chosen
+   * port is accessible to the caller from the URL set in the input {@link
+   * Endpoints.ApiServiceDescriptor.Builder}.
+   */
+  public abstract Server allocatePortAndCreate(
+      BindableService service, Endpoints.ApiServiceDescriptor.Builder builder) throws IOException;
+
+  /**
+   * Creates an instance of this server at the address specified by the given service descriptor.
+   */
+  public abstract Server create(
+      BindableService service, Endpoints.ApiServiceDescriptor serviceDescriptor) throws IOException;
+
+  /**
+   * Creates a {@link Server gRPC Server} using the default server factory.
+   *
+   * <p>The server is created listening any open port on "localhost".
+   */
+  public static class InetSocketAddressServerFactory extends ServerFactory {
+    private InetSocketAddressServerFactory() {}
+
+    @Override
+    public Server allocatePortAndCreate(
+        BindableService service, Endpoints.ApiServiceDescriptor.Builder apiServiceDescriptor)
+        throws IOException {
+      InetSocketAddress address = new InetSocketAddress(InetAddress.getLoopbackAddress(), 0);
+      Server server = createServer(service, address);
+      apiServiceDescriptor.setUrl(
+          HostAndPort.fromParts(address.getHostName(), server.getPort()).toString());
+      return server;
+    }
+
+    @Override
+    public Server create(BindableService service, Endpoints.ApiServiceDescriptor serviceDescriptor)
+        throws IOException {
+      SocketAddress socketAddress = SocketAddressFactory.createFrom(serviceDescriptor.getUrl());
+      checkArgument(
+          socketAddress instanceof InetSocketAddress,
+          "%s %s requires a host:port socket address, got %s",
+          getClass().getSimpleName(), ServerFactory.class.getSimpleName(),
+          serviceDescriptor.getUrl());
+      return createServer(service, (InetSocketAddress) socketAddress);
+    }
+
+    private static Server createServer(BindableService service, InetSocketAddress socket)
+        throws IOException {
+      Server server =
+          NettyServerBuilder.forPort(socket.getPort())
+              .addService(service)
+              // Set the message size to max value here. The actual size is governed by the
+              // buffer size in the layers above.
+              .maxMessageSize(Integer.MAX_VALUE)
+              .build();
+      server.start();
+      return server;
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/927a8db1/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/package-info.java
----------------------------------------------------------------------
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/package-info.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/package-info.java
new file mode 100644
index 0000000..bc36f5e
--- /dev/null
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/package-info.java
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Utilities used by runners to interact with the fn execution components of the Beam Portability
+ * Framework.
+ */
+package org.apache.beam.runners.fnexecution;

http://git-wip-us.apache.org/repos/asf/beam/blob/927a8db1/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/ServerFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/ServerFactoryTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/ServerFactoryTest.java
new file mode 100644
index 0000000..aa8d246
--- /dev/null
+++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/ServerFactoryTest.java
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.fnexecution;
+
+import static org.hamcrest.Matchers.allOf;
+import static org.hamcrest.Matchers.anyOf;
+import static org.hamcrest.Matchers.contains;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.greaterThan;
+import static org.hamcrest.Matchers.lessThan;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThat;
+
+import com.google.common.net.HostAndPort;
+import com.google.common.util.concurrent.Uninterruptibles;
+import io.grpc.ManagedChannel;
+import io.grpc.Server;
+import io.grpc.stub.CallStreamObserver;
+import io.grpc.stub.StreamObserver;
+import java.net.InetAddress;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.LinkedBlockingQueue;
+import org.apache.beam.harness.channel.ManagedChannelFactory;
+import org.apache.beam.harness.test.TestStreams;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi.Elements;
+import org.apache.beam.model.fnexecution.v1.BeamFnDataGrpc;
+import org.apache.beam.model.pipeline.v1.Endpoints;
+import org.junit.Test;
+
+/**
+ * Tests for {@link ServerFactory}.
+ */
+public class ServerFactoryTest {
+
+  private static final BeamFnApi.Elements CLIENT_DATA = BeamFnApi.Elements.newBuilder()
+      .addData(BeamFnApi.Elements.Data.newBuilder().setInstructionReference("1"))
+      .build();
+  private static final BeamFnApi.Elements SERVER_DATA = BeamFnApi.Elements.newBuilder()
+      .addData(BeamFnApi.Elements.Data.newBuilder().setInstructionReference("1"))
+      .build();
+
+  @Test
+  public void testCreatingDefaultServer() throws Exception {
+    Endpoints.ApiServiceDescriptor apiServiceDescriptor =
+        runTestUsing(ServerFactory.createDefault(), ManagedChannelFactory.createDefault());
+    HostAndPort hostAndPort = HostAndPort.fromString(apiServiceDescriptor.getUrl());
+    assertThat(hostAndPort.getHost(), anyOf(
+        equalTo(InetAddress.getLoopbackAddress().getHostName()),
+        equalTo(InetAddress.getLoopbackAddress().getHostAddress())));
+    assertThat(hostAndPort.getPort(), allOf(greaterThan(0), lessThan(65536)));
+  }
+
+  private Endpoints.ApiServiceDescriptor runTestUsing(
+      ServerFactory serverFactory, ManagedChannelFactory channelFactory) throws Exception {
+    Endpoints.ApiServiceDescriptor.Builder apiServiceDescriptorBuilder =
+        Endpoints.ApiServiceDescriptor.newBuilder();
+
+    Collection<Elements> serverElements = new ArrayList<>();
+    CountDownLatch clientHangedUp = new CountDownLatch(1);
+    CallStreamObserver<Elements> serverInboundObserver =
+        TestStreams.withOnNext(serverElements::add)
+        .withOnCompleted(clientHangedUp::countDown)
+        .build();
+    TestDataService service = new TestDataService(serverInboundObserver);
+    Server server = serverFactory.allocatePortAndCreate(service, apiServiceDescriptorBuilder);
+    assertFalse(server.isShutdown());
+
+    ManagedChannel channel = channelFactory.forDescriptor(apiServiceDescriptorBuilder.build());
+    BeamFnDataGrpc.BeamFnDataStub stub = BeamFnDataGrpc.newStub(channel);
+    Collection<BeamFnApi.Elements> clientElements = new ArrayList<>();
+    CountDownLatch serverHangedUp = new CountDownLatch(1);
+    CallStreamObserver<BeamFnApi.Elements> clientInboundObserver =
+        TestStreams.withOnNext(clientElements::add)
+        .withOnCompleted(serverHangedUp::countDown)
+        .build();
+
+    StreamObserver<Elements> clientOutboundObserver = stub.data(clientInboundObserver);
+    StreamObserver<BeamFnApi.Elements> serverOutboundObserver = service.outboundObservers.take();
+
+    clientOutboundObserver.onNext(CLIENT_DATA);
+    serverOutboundObserver.onNext(SERVER_DATA);
+    clientOutboundObserver.onCompleted();
+    clientHangedUp.await();
+    serverOutboundObserver.onCompleted();
+    serverHangedUp.await();
+
+    assertThat(clientElements, contains(SERVER_DATA));
+    assertThat(serverElements, contains(CLIENT_DATA));
+
+    return apiServiceDescriptorBuilder.build();
+  }
+
+  /** A test gRPC service that uses the provided inbound observer for all clients. */
+  private static class TestDataService extends BeamFnDataGrpc.BeamFnDataImplBase {
+    private final LinkedBlockingQueue<StreamObserver<BeamFnApi.Elements>> outboundObservers;
+    private final StreamObserver<BeamFnApi.Elements> inboundObserver;
+    private TestDataService(StreamObserver<BeamFnApi.Elements> inboundObserver) {
+      this.inboundObserver = inboundObserver;
+      this.outboundObservers = new LinkedBlockingQueue<>();
+    }
+
+    @Override
+    public StreamObserver<BeamFnApi.Elements> data(
+        StreamObserver<BeamFnApi.Elements> outboundObserver) {
+      Uninterruptibles.putUninterruptibly(outboundObservers, outboundObserver);
+      return inboundObserver;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/927a8db1/runners/pom.xml
----------------------------------------------------------------------
diff --git a/runners/pom.xml b/runners/pom.xml
index 164d1b3..df3faa9 100644
--- a/runners/pom.xml
+++ b/runners/pom.xml
@@ -63,6 +63,7 @@
         <jdk>[1.8,)</jdk>
       </activation>
       <modules>
+        <module>java-fn-execution</module>
         <module>gearpump</module>
       </modules>
     </profile>


[06/50] [abbrv] beam git commit: Updates Python datastore wordcount example to take a dataset parameter.

Posted by ke...@apache.org.
Updates Python datastore wordcount example to take a dataset parameter.


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

Branch: refs/heads/mr-runner
Commit: ac4363491e6c300cf34c05bf547cee3ccc37c98e
Parents: b013d7c
Author: chamikara@google.com <ch...@google.com>
Authored: Tue Oct 31 18:37:29 2017 -0700
Committer: chamikara@google.com <ch...@google.com>
Committed: Wed Nov 1 13:03:37 2017 -0700

----------------------------------------------------------------------
 .../examples/cookbook/datastore_wordcount.py    | 24 +++++++++++---------
 .../io/gcp/datastore/v1/datastoreio.py          | 16 +++++++++++--
 2 files changed, 27 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/ac436349/sdks/python/apache_beam/examples/cookbook/datastore_wordcount.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/examples/cookbook/datastore_wordcount.py b/sdks/python/apache_beam/examples/cookbook/datastore_wordcount.py
index 099fb08..7204e3b 100644
--- a/sdks/python/apache_beam/examples/cookbook/datastore_wordcount.py
+++ b/sdks/python/apache_beam/examples/cookbook/datastore_wordcount.py
@@ -32,12 +32,14 @@ counts them and write the output to a set of files.
 
 The following options must be provided to run this pipeline in read-only mode:
 ``
---project YOUR_PROJECT_ID
+--dataset YOUR_DATASET
 --kind YOUR_DATASTORE_KIND
 --output [YOUR_LOCAL_FILE *or* gs://YOUR_OUTPUT_PATH]
 --read_only
 ``
 
+Dataset maps to Project ID for v1 version of datastore.
+
 Read-write Mode: In this mode, this example reads words from an input file,
 converts them to Cloud Datastore ``Entity`` objects and writes them to
 Cloud Datastore using the ``datastoreio.Write`` transform. The second pipeline
@@ -47,7 +49,7 @@ write the output to a set of files.
 
 The following options must be provided to run this pipeline in read-write mode:
 ``
---project YOUR_PROJECT_ID
+--dataset YOUR_DATASET
 --kind YOUR_DATASTORE_KIND
 --output [YOUR_LOCAL_FILE *or* gs://YOUR_OUTPUT_PATH]
 ``
@@ -77,7 +79,6 @@ from apache_beam.io.gcp.datastore.v1.datastoreio import ReadFromDatastore
 from apache_beam.io.gcp.datastore.v1.datastoreio import WriteToDatastore
 from apache_beam.metrics import Metrics
 from apache_beam.metrics.metric import MetricsFilter
-from apache_beam.options.pipeline_options import GoogleCloudOptions
 from apache_beam.options.pipeline_options import PipelineOptions
 from apache_beam.options.pipeline_options import SetupOptions
 
@@ -134,7 +135,7 @@ class EntityWrapper(object):
     return entity
 
 
-def write_to_datastore(project, user_options, pipeline_options):
+def write_to_datastore(user_options, pipeline_options):
   """Creates a pipeline that writes entities to Cloud Datastore."""
   with beam.Pipeline(options=pipeline_options) as p:
 
@@ -144,7 +145,7 @@ def write_to_datastore(project, user_options, pipeline_options):
      | 'create entity' >> beam.Map(
          EntityWrapper(user_options.namespace, user_options.kind,
                        user_options.ancestor).make_entity)
-     | 'write to datastore' >> WriteToDatastore(project))
+     | 'write to datastore' >> WriteToDatastore(user_options.dataset))
 
 
 def make_ancestor_query(kind, namespace, ancestor):
@@ -167,7 +168,7 @@ def make_ancestor_query(kind, namespace, ancestor):
   return query
 
 
-def read_from_datastore(project, user_options, pipeline_options):
+def read_from_datastore(user_options, pipeline_options):
   """Creates a pipeline that reads entities from Cloud Datastore."""
   p = beam.Pipeline(options=pipeline_options)
   # Create a query to read entities from datastore.
@@ -176,7 +177,7 @@ def read_from_datastore(project, user_options, pipeline_options):
 
   # Read entities from Cloud Datastore into a PCollection.
   lines = p | 'read from datastore' >> ReadFromDatastore(
-      project, query, user_options.namespace)
+      user_options.dataset, query, user_options.namespace)
 
   # Count the occurrences of each word.
   def count_ones(word_ones):
@@ -216,6 +217,9 @@ def run(argv=None):
                       dest='input',
                       default='gs://dataflow-samples/shakespeare/kinglear.txt',
                       help='Input file to process.')
+  parser.add_argument('--dataset',
+                      dest='dataset',
+                      help='Dataset ID to read from Cloud Datastore.')
   parser.add_argument('--kind',
                       dest='kind',
                       required=True,
@@ -246,15 +250,13 @@ def run(argv=None):
   # workflow rely on global context (e.g., a module imported at module level).
   pipeline_options = PipelineOptions(pipeline_args)
   pipeline_options.view_as(SetupOptions).save_main_session = True
-  gcloud_options = pipeline_options.view_as(GoogleCloudOptions)
 
   # Write to Datastore if `read_only` options is not specified.
   if not known_args.read_only:
-    write_to_datastore(gcloud_options.project, known_args, pipeline_options)
+    write_to_datastore(known_args, pipeline_options)
 
   # Read entities from Datastore.
-  result = read_from_datastore(gcloud_options.project, known_args,
-                               pipeline_options)
+  result = read_from_datastore(known_args, pipeline_options)
 
   empty_lines_filter = MetricsFilter().with_name('empty_lines')
   query_result = result.metrics().query(empty_lines_filter)

http://git-wip-us.apache.org/repos/asf/beam/blob/ac436349/sdks/python/apache_beam/io/gcp/datastore/v1/datastoreio.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/io/gcp/datastore/v1/datastoreio.py b/sdks/python/apache_beam/io/gcp/datastore/v1/datastoreio.py
index 078002c..13209c1 100644
--- a/sdks/python/apache_beam/io/gcp/datastore/v1/datastoreio.py
+++ b/sdks/python/apache_beam/io/gcp/datastore/v1/datastoreio.py
@@ -89,10 +89,10 @@ class ReadFromDatastore(PTransform):
   _DEFAULT_BUNDLE_SIZE_BYTES = 64 * 1024 * 1024
 
   def __init__(self, project, query, namespace=None, num_splits=0):
-    """Initialize the ReadFromDatastore transform.
+    """Initialize the `ReadFromDatastore` transform.
 
     Args:
-      project: The Project ID
+      project: The ID of the project to read from.
       query: Cloud Datastore query to be read from.
       namespace: An optional namespace.
       num_splits: Number of splits for the query.
@@ -459,7 +459,13 @@ class _Mutate(PTransform):
 
 class WriteToDatastore(_Mutate):
   """A ``PTransform`` to write a ``PCollection[Entity]`` to Cloud Datastore."""
+
   def __init__(self, project):
+    """Initialize the `WriteToDatastore` transform.
+
+    Args:
+      project: The ID of the project to write to.
+    """
 
     # Import here to avoid adding the dependency for local running scenarios.
     try:
@@ -486,6 +492,12 @@ class WriteToDatastore(_Mutate):
 class DeleteFromDatastore(_Mutate):
   """A ``PTransform`` to delete a ``PCollection[Key]`` from Cloud Datastore."""
   def __init__(self, project):
+    """Initialize the `DeleteFromDatastore` transform.
+
+    Args:
+      project: The ID of the project from which the entities will be deleted.
+    """
+
     super(DeleteFromDatastore, self).__init__(
         project, DeleteFromDatastore.to_delete_mutation)
 


[29/50] [abbrv] beam git commit: [BEAM-3114] Generate text proto config properly in container boot code

Posted by ke...@apache.org.
[BEAM-3114] Generate text proto config properly in container boot code


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

Branch: refs/heads/mr-runner
Commit: fcd864ac2c987b4d059ca324903091876b9f4f1f
Parents: 15db1dd
Author: Henning Rohde <he...@google.com>
Authored: Fri Oct 27 14:20:10 2017 -0700
Committer: Thomas Groh <tg...@google.com>
Committed: Mon Nov 6 11:02:27 2017 -0800

----------------------------------------------------------------------
 sdks/java/container/boot.go   | 11 ++++++-----
 sdks/python/container/boot.go | 31 ++++++++++++++++---------------
 2 files changed, 22 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/fcd864ac/sdks/java/container/boot.go
----------------------------------------------------------------------
diff --git a/sdks/java/container/boot.go b/sdks/java/container/boot.go
index 144c4ac..1c80e0b 100644
--- a/sdks/java/container/boot.go
+++ b/sdks/java/container/boot.go
@@ -20,7 +20,6 @@ package main
 import (
 	"context"
 	"flag"
-	"fmt"
 	"log"
 	"os"
 	"path/filepath"
@@ -28,11 +27,13 @@ import (
 	"strings"
 
 	"github.com/apache/beam/sdks/go/pkg/beam/artifact"
-	pb "github.com/apache/beam/sdks/go/pkg/beam/model/fnexecution_v1"
+	fnpb "github.com/apache/beam/sdks/go/pkg/beam/model/fnexecution_v1"
+	pb "github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1"
 	"github.com/apache/beam/sdks/go/pkg/beam/provision"
 	"github.com/apache/beam/sdks/go/pkg/beam/util/execx"
 	"github.com/apache/beam/sdks/go/pkg/beam/util/grpcx"
 	"github.com/apache/beam/sdks/go/pkg/beam/util/syscallx"
+	"github.com/golang/protobuf/proto"
 )
 
 var (
@@ -92,8 +93,8 @@ func main() {
 	// (3) Invoke the Java harness, preserving artifact ordering in classpath.
 
 	os.Setenv("PIPELINE_OPTIONS", options)
-	os.Setenv("LOGGING_API_SERVICE_DESCRIPTOR", fmt.Sprintf("url: \"%v\"\n", *loggingEndpoint))
-	os.Setenv("CONTROL_API_SERVICE_DESCRIPTOR", fmt.Sprintf("url: \"%v\"\n", *controlEndpoint))
+	os.Setenv("LOGGING_API_SERVICE_DESCRIPTOR", proto.MarshalTextString(&pb.ApiServiceDescriptor{Url: *loggingEndpoint}))
+	os.Setenv("CONTROL_API_SERVICE_DESCRIPTOR", proto.MarshalTextString(&pb.ApiServiceDescriptor{Url: *controlEndpoint}))
 
 	const jarsDir = "/opt/apache/beam/jars"
 	cp := []string{
@@ -122,7 +123,7 @@ func main() {
 // that value, it returns 1GB. This is an imperfect heuristic. It aims to
 // ensure there is memory for non-heap use and other overhead, while also not
 // underutilizing the machine.
-func heapSizeLimit(info *pb.ProvisionInfo) uint64 {
+func heapSizeLimit(info *fnpb.ProvisionInfo) uint64 {
 	if provided := info.GetResourceLimits().GetMemory().GetSize(); provided > 0 {
 		return (provided * 80) / 100
 	}

http://git-wip-us.apache.org/repos/asf/beam/blob/fcd864ac/sdks/python/container/boot.go
----------------------------------------------------------------------
diff --git a/sdks/python/container/boot.go b/sdks/python/container/boot.go
index 18b9900..fea0935 100644
--- a/sdks/python/container/boot.go
+++ b/sdks/python/container/boot.go
@@ -18,18 +18,19 @@
 package main
 
 import (
-"context"
-"flag"
-"fmt"
-"log"
-"os"
-"path/filepath"
-"strings"
-
-"github.com/apache/beam/sdks/go/pkg/beam/artifact"
-"github.com/apache/beam/sdks/go/pkg/beam/provision"
-"github.com/apache/beam/sdks/go/pkg/beam/util/execx"
-"github.com/apache/beam/sdks/go/pkg/beam/util/grpcx"
+	"context"
+	"flag"
+	"log"
+	"os"
+	"path/filepath"
+	"strings"
+
+	"github.com/apache/beam/sdks/go/pkg/beam/artifact"
+	pb "github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1"
+	"github.com/apache/beam/sdks/go/pkg/beam/provision"
+	"github.com/apache/beam/sdks/go/pkg/beam/util/execx"
+	"github.com/apache/beam/sdks/go/pkg/beam/util/grpcx"
+	"github.com/golang/protobuf/proto"
 )
 
 var (
@@ -94,8 +95,8 @@ func main() {
 	// (3) Invoke python
 
 	os.Setenv("PIPELINE_OPTIONS", options)
-	os.Setenv("LOGGING_API_SERVICE_DESCRIPTOR", fmt.Sprintf("url: \"%v\"\n", *loggingEndpoint))
-	os.Setenv("CONTROL_API_SERVICE_DESCRIPTOR", fmt.Sprintf("url: \"%v\"\n", *controlEndpoint))
+	os.Setenv("LOGGING_API_SERVICE_DESCRIPTOR", proto.MarshalTextString(&pb.ApiServiceDescriptor{Url: *loggingEndpoint}))
+	os.Setenv("CONTROL_API_SERVICE_DESCRIPTOR", proto.MarshalTextString(&pb.ApiServiceDescriptor{Url: *controlEndpoint}))
 
 	args := []string{
 		"-m",
@@ -119,4 +120,4 @@ func joinPaths(dir string, paths ...string) []string {
 		ret = append(ret, filepath.Join(dir, filepath.FromSlash(p)))
 	}
 	return ret
-}
\ No newline at end of file
+}


[19/50] [abbrv] beam git commit: This closes #4025

Posted by ke...@apache.org.
This closes #4025


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

Branch: refs/heads/mr-runner
Commit: 49254783523b1e9351dc90270fbdf7bc1c5e6309
Parents: 482d178 b2c9fba
Author: chamikara@google.com <ch...@google.com>
Authored: Thu Nov 2 15:20:38 2017 -0700
Committer: chamikara@google.com <ch...@google.com>
Committed: Thu Nov 2 15:20:38 2017 -0700

----------------------------------------------------------------------
 .../io/gcp/bigquery/BigQueryTableSource.java    | 10 ++-
 .../sdk/io/gcp/bigquery/BigQueryIOTest.java     | 81 +++++++++++++++++++-
 2 files changed, 88 insertions(+), 3 deletions(-)
----------------------------------------------------------------------



[47/50] [abbrv] beam git commit: Merge branch 'master' upto commit 269bf89463094a17f51d61d30bedd388b04dc8c0 into mr-runner

Posted by ke...@apache.org.
Merge branch 'master' upto commit 269bf89463094a17f51d61d30bedd388b04dc8c0 into mr-runner


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

Branch: refs/heads/mr-runner
Commit: 7181d0c30db5de8d435f04fd3b2caf78793865b4
Parents: 5fa0b14 269bf89
Author: Pei He <pe...@apache.org>
Authored: Tue Nov 7 15:22:08 2017 +0800
Committer: Pei He <pe...@apache.org>
Committed: Tue Nov 7 15:22:08 2017 +0800

----------------------------------------------------------------------
 .gitattributes                                  |    2 +
 .gitignore                                      |    9 +-
 .test-infra/jenkins/PreCommit_Pipeline.groovy   |  129 +
 .../jenkins/common_job_properties.groovy        |  205 +-
 .test-infra/jenkins/job_00_seed.groovy          |  114 +
 .test-infra/jenkins/job_beam_Java_Build.groovy  |   82 +
 .../jenkins/job_beam_Java_CodeHealth.groovy     |   39 +
 .../job_beam_Java_IntegrationTest.groovy        |   63 +
 .../jenkins/job_beam_Java_UnitTest.groovy       |   48 +
 ...job_beam_PostCommit_Java_MavenInstall.groovy |   21 +-
 ..._PostCommit_Java_MavenInstall_Windows.groovy |    2 +-
 ...ommit_Python_ValidatesRunner_Dataflow.groovy |   54 +
 .../job_beam_PreCommit_Go_MavenInstall.groovy   |   56 +
 .../job_beam_PreCommit_Java_MavenInstall.groovy |   21 +-
 .../jenkins/job_beam_PreCommit_Pipeline.groovy  |   84 +
 ...ob_beam_PreCommit_Python_MavenInstall.groovy |   56 +
 .../job_beam_PreCommit_Website_Merge.groovy     |    3 +
 .../job_beam_PreCommit_Website_Stage.groovy     |    3 +
 .../job_beam_PreCommit_Website_Test.groovy      |    3 +
 .../jenkins/job_beam_Python_UnitTest.groovy     |   40 +
 .../job_beam_Release_NightlySnapshot.groovy     |   22 +-
 .test-infra/jenkins/job_seed.groovy             |   53 -
 .test-infra/jenkins/job_seed_standalone.groovy  |  114 +
 examples/java/pom.xml                           |    8 +-
 .../apache/beam/examples/WindowedWordCount.java |    1 -
 .../examples/cookbook/BigQueryTornadoes.java    |    2 +-
 .../cookbook/CombinePerKeyExamples.java         |    2 +-
 .../beam/examples/cookbook/FilterExamples.java  |    2 +-
 .../beam/examples/cookbook/JoinExamples.java    |    6 +-
 .../examples/cookbook/MaxPerKeyExamples.java    |    2 +-
 examples/java8/pom.xml                          |   14 +-
 .../complete/game/injector/Injector.java        |    2 +-
 .../complete/game/injector/InjectorUtils.java   |    2 +-
 .../examples/website_snippets/Snippets.java     |   87 +
 .../examples/website_snippets/SnippetsTest.java |  114 +
 examples/pom.xml                                |    2 +-
 model/fn-execution/pom.xml                      |  114 +
 .../src/main/proto/beam_fn_api.proto            |  729 ++++
 .../src/main/proto/beam_provision_api.proto     |   99 +
 .../model/fnexecution/v1/standard_coders.yaml   |  195 +
 model/job-management/pom.xml                    |  114 +
 .../src/main/proto/beam_artifact_api.proto      |  134 +
 .../src/main/proto/beam_job_api.proto           |  174 +
 model/pipeline/pom.xml                          |   89 +
 .../src/main/proto/beam_runner_api.proto        |  843 +++++
 model/pipeline/src/main/proto/endpoints.proto   |   47 +
 .../src/main/proto/standard_window_fns.proto    |   54 +
 model/pom.xml                                   |   40 +
 pom.xml                                         |  368 +-
 runners/apex/pom.xml                            |    9 +-
 .../beam/runners/apex/ApexPipelineOptions.java  |    2 +-
 .../beam/runners/apex/ApexRunnerResult.java     |    4 +-
 .../apex/translation/ParDoTranslator.java       |   24 +-
 .../operators/ApexParDoOperator.java            |   22 +-
 .../translation/utils/ApexStateInternals.java   |   17 +-
 .../apex/translation/utils/ValuesSource.java    |    4 -
 .../beam/runners/apex/ApexYarnLauncherTest.java |    9 +-
 .../apex/examples/UnboundedTextSource.java      |    4 -
 .../FlattenPCollectionTranslatorTest.java       |    1 -
 .../translation/GroupByKeyTranslatorTest.java   |    4 -
 .../apex/translation/ParDoTranslatorTest.java   |    4 +-
 .../translation/utils/CollectionSource.java     |    4 -
 runners/core-construction-java/pom.xml          |   19 +-
 .../construction/ArtifactServiceStager.java     |  244 ++
 .../core/construction/CoderTranslation.java     |   22 +-
 .../core/construction/CombineTranslation.java   |  224 +-
 .../CreatePCollectionViewTranslation.java       |   28 +-
 .../construction/DisplayDataTranslation.java    |    4 +-
 .../core/construction/FlattenTranslator.java    |   12 +-
 .../construction/GroupByKeyTranslation.java     |   17 +-
 .../construction/PCollectionTranslation.java    |   10 +-
 .../construction/PTransformTranslation.java     |  292 +-
 .../core/construction/ParDoTranslation.java     |  392 +-
 .../PipelineOptionsTranslation.java             |   51 +
 .../core/construction/PipelineTranslation.java  |  137 +-
 .../core/construction/ReadTranslation.java      |  113 +-
 .../core/construction/RehydratedComponents.java |    9 +-
 .../construction/RunnerPCollectionView.java     |    2 +-
 .../core/construction/SdkComponents.java        |   17 +-
 .../core/construction/SplittableParDo.java      |    8 +
 .../construction/TestStreamTranslation.java     |  170 +-
 .../TransformPayloadTranslatorRegistrar.java    |    2 +
 .../core/construction/TriggerTranslation.java   |   10 +-
 .../construction/WindowIntoTranslation.java     |   30 +-
 .../WindowingStrategyTranslation.java           |  150 +-
 .../construction/WriteFilesTranslation.java     |  245 +-
 .../construction/ArtifactServiceStagerTest.java |  141 +
 .../core/construction/CoderTranslationTest.java |    4 +-
 .../construction/CombineTranslationTest.java    |   20 +-
 .../CreatePCollectionViewTranslationTest.java   |   10 +-
 .../InMemoryArtifactStagerService.java          |  152 +
 .../PCollectionTranslationTest.java             |    2 +-
 .../construction/PTransformMatchersTest.java    |   15 +-
 .../construction/PTransformTranslationTest.java |   39 +-
 .../core/construction/ParDoTranslationTest.java |    9 +-
 .../PipelineOptionsTranslationTest.java         |  143 +
 .../construction/PipelineTranslationTest.java   |    2 +-
 .../core/construction/ReadTranslationTest.java  |   14 +-
 .../core/construction/SdkComponentsTest.java    |    2 +-
 .../construction/TestStreamTranslationTest.java |   12 +-
 .../construction/WindowIntoTranslationTest.java |    2 +-
 .../WindowingStrategyTranslationTest.java       |    2 +-
 .../construction/WriteFilesTranslationTest.java |   15 +-
 runners/core-java/pom.xml                       |   31 +-
 .../runners/core/InMemoryStateInternals.java    |   39 +-
 .../runners/core/InMemoryTimerInternals.java    |   28 +-
 .../beam/runners/core/SideInputHandler.java     |    2 +-
 .../core/SplittableParDoViaKeyedWorkItems.java  |    7 +
 .../apache/beam/runners/core/StateTable.java    |   40 +-
 .../org/apache/beam/runners/core/StateTags.java |   13 +
 .../core/TestInMemoryStateInternals.java        |    6 +-
 .../apache/beam/runners/core/WatermarkHold.java |    6 +-
 .../runners/core/fn/FnApiControlClient.java     |  148 +
 .../core/fn/FnApiControlClientPoolService.java  |   66 +
 .../beam/runners/core/fn/FnDataReceiver.java    |   33 +
 .../beam/runners/core/fn/FnDataService.java     |   81 +
 .../beam/runners/core/fn/SdkHarnessClient.java  |  172 +
 .../runners/core/fn/SdkHarnessDoFnRunner.java   |  102 +
 .../beam/runners/core/fn/package-info.java      |   22 +
 .../core/metrics/MetricsContainerImpl.java      |   40 +
 .../triggers/AfterWatermarkStateMachine.java    |    8 +-
 .../triggers/TriggerStateMachineRunner.java     |    3 +-
 .../core/triggers/TriggerStateMachines.java     |    2 +-
 .../beam/runners/core/ReduceFnTester.java       |   22 +-
 .../beam/runners/core/StateInternalsTest.java   |   59 +
 .../fn/FnApiControlClientPoolServiceTest.java   |   65 +
 .../runners/core/fn/FnApiControlClientTest.java |  139 +
 .../runners/core/fn/SdkHarnessClientTest.java   |   96 +
 .../core/fn/SdkHarnessDoFnRunnerTest.java       |   73 +
 .../core/metrics/MetricsContainerImplTest.java  |   10 +
 .../AfterWatermarkStateMachineTest.java         |   29 +
 .../triggers/TriggerStateMachineTester.java     |   13 +-
 .../core/triggers/TriggerStateMachinesTest.java |    2 +-
 runners/direct-java/pom.xml                     |   14 +-
 .../CopyOnAccessInMemoryStateInternals.java     |   10 +-
 .../beam/runners/direct/DirectGroupByKey.java   |   16 +-
 .../beam/runners/direct/DirectRunner.java       |    5 +-
 .../GroupAlsoByWindowEvaluatorFactory.java      |    2 +-
 .../beam/runners/direct/MultiStepCombine.java   |   18 +-
 .../runners/direct/ParDoEvaluatorFactory.java   |   30 +-
 .../direct/ParDoMultiOverrideFactory.java       |    9 +-
 ...littableProcessElementsEvaluatorFactory.java |    3 +-
 .../direct/StatefulParDoEvaluatorFactory.java   |   15 +-
 .../direct/TestStreamEvaluatorFactory.java      |    9 +-
 .../direct/TransformEvaluatorRegistry.java      |   37 +-
 .../runners/direct/ViewOverrideFactory.java     |    8 +-
 .../direct/WriteWithShardingFactory.java        |   37 +-
 .../direct/BoundedReadEvaluatorFactoryTest.java |    4 -
 .../CopyOnAccessInMemoryStateInternalsTest.java |   74 +-
 .../UnboundedReadEvaluatorFactoryTest.java      |    3 -
 .../direct/WriteWithShardingFactoryTest.java    |   11 +-
 runners/flink/pom.xml                           |    4 +-
 .../runners/flink/CreateStreamingFlinkView.java |    3 +
 .../flink/FlinkBatchPipelineTranslator.java     |    3 +-
 .../runners/flink/FlinkPipelineOptions.java     |   11 +
 .../FlinkStreamingTransformTranslators.java     |  244 +-
 .../functions/FlinkAssignContext.java           |   17 +-
 .../translation/types/CoderTypeSerializer.java  |    2 +-
 .../wrappers/streaming/DoFnOperator.java        |  429 ++-
 .../streaming/SplittableDoFnOperator.java       |    4 +-
 .../wrappers/streaming/WindowDoFnOperator.java  |    4 +-
 .../state/FlinkSplitStateInternals.java         |    8 +-
 .../streaming/state/FlinkStateInternals.java    |   13 +-
 .../beam/runners/flink/PipelineOptionsTest.java |   21 +-
 .../flink/streaming/DoFnOperatorTest.java       |  161 +-
 .../flink/streaming/TestCountingSource.java     |    3 -
 .../types/CoderTypeSerializerTest.java          |   79 +
 runners/gcp/gcemd/Dockerfile                    |   30 +
 runners/gcp/gcemd/main.go                       |   85 +
 runners/gcp/gcemd/pom.xml                       |  154 +
 runners/gcp/gcsproxy/Dockerfile                 |   30 +
 runners/gcp/gcsproxy/main.go                    |   91 +
 runners/gcp/gcsproxy/pom.xml                    |  154 +
 runners/gcp/pom.xml                             |   38 +
 runners/gearpump/pom.xml                        |   21 +-
 .../gearpump/GearpumpPipelineResult.java        |    3 +-
 .../translators/GroupByKeyTranslator.java       |    2 +-
 .../gearpump/translators/io/GearpumpSource.java |    7 +-
 .../gearpump/translators/io/ValuesSource.java   |    8 -
 .../translators/utils/DoFnRunnerFactory.java    |    7 +-
 .../translators/utils/TranslatorUtils.java      |   20 -
 .../FlattenPCollectionsTranslatorTest.java      |    6 +
 runners/google-cloud-dataflow-java/pom.xml      |   22 +-
 .../beam/runners/dataflow/AssignWindows.java    |   84 -
 .../dataflow/BatchStatefulParDoOverrides.java   |   71 +-
 .../dataflow/DataflowPipelineTranslator.java    |    5 +-
 .../beam/runners/dataflow/DataflowRunner.java   |   69 +-
 .../runners/dataflow/DataflowRunnerInfo.java    |   91 +-
 .../dataflow/PrimitiveParDoSingleFactory.java   |    2 +-
 .../dataflow/options/CloudDebuggerOptions.java  |    3 +-
 .../options/DataflowPipelineDebugOptions.java   |   23 +-
 .../options/DataflowPipelineOptions.java        |    8 +
 .../options/DataflowProfilingOptions.java       |    3 +-
 .../options/DataflowWorkerLoggingOptions.java   |    6 +
 ...aultCoderCloudObjectTranslatorRegistrar.java |    2 +
 .../beam/runners/dataflow/util/DoFnInfo.java    |  104 -
 .../beam/runners/dataflow/util/GcsStager.java   |   53 +-
 .../runners/dataflow/util/MonitoringUtil.java   |    2 +-
 .../beam/runners/dataflow/util/PackageUtil.java |  581 +--
 .../beam/runners/dataflow/util/Stager.java      |   32 +-
 .../BatchStatefulParDoOverridesTest.java        |   45 +-
 .../DataflowPipelineTranslatorTest.java         |    4 +-
 .../dataflow/DataflowRunnerInfoTest.java        |    9 +
 .../runners/dataflow/DataflowRunnerTest.java    |  107 +-
 .../dataflow/util/MonitoringUtilTest.java       |    4 +-
 .../runners/dataflow/util/PackageUtilTest.java  |  124 +-
 runners/java-fn-execution/pom.xml               |   91 +
 .../beam/runners/fnexecution/ServerFactory.java |  104 +
 .../beam/runners/fnexecution/package-info.java  |   23 +
 .../runners/fnexecution/ServerFactoryTest.java  |  153 +
 runners/local-artifact-service-java/pom.xml     |  116 +
 .../LocalFileSystemArtifactStagerService.java   |  279 ++
 .../beam/artifact/local/package-info.java       |   22 +
 ...ocalFileSystemArtifactStagerServiceTest.java |  301 ++
 runners/pom.xml                                 |    6 +-
 runners/reference/job-server/pom.xml            |   82 +
 .../reference/job/ReferenceRunnerJobServer.java |   77 +
 .../job/ReferenceRunnerJobService.java          |   79 +
 .../runners/reference/job/package-info.java     |   23 +
 .../job/ReferenceRunnerJobServiceTest.java      |   34 +
 runners/reference/pom.xml                       |   39 +
 runners/spark/pom.xml                           |    6 +-
 .../apache/beam/runners/spark/SparkRunner.java  |    5 +-
 .../beam/runners/spark/io/CreateStream.java     |  104 +-
 .../SparkGroupAlsoByWindowViaWindowSet.java     |  819 ++--
 .../spark/stateful/SparkTimerInternals.java     |   21 +-
 .../spark/translation/BoundedDataset.java       |   17 +-
 .../beam/runners/spark/translation/Dataset.java |    3 +-
 .../spark/translation/EvaluationContext.java    |   23 +-
 .../spark/translation/SparkContextFactory.java  |    2 -
 .../translation/StorageLevelPTransform.java     |   37 -
 .../spark/translation/TransformTranslator.java  |   53 +-
 .../spark/translation/TranslationUtils.java     |   78 +
 .../streaming/StreamingTransformTranslator.java |   86 +-
 .../translation/streaming/UnboundedDataset.java |   27 +-
 .../streaming/WatermarkSyncedDStream.java       |  149 +
 .../spark/util/GlobalWatermarkHolder.java       |  302 +-
 .../runners/spark/SparkPipelineStateTest.java   |    4 +-
 .../runners/spark/SparkRunnerDebuggerTest.java  |   11 +-
 .../spark/translation/StorageLevelTest.java     |   75 -
 .../translation/streaming/CreateStreamTest.java |   33 +-
 .../spark/src/test/resources/log4j.properties   |   11 +-
 sdks/CONTAINERS.md                              |  162 +
 sdks/common/fn-api/pom.xml                      |  109 -
 .../fn-api/src/main/proto/beam_fn_api.proto     |  659 ----
 .../org/apache/beam/fn/v1/standard_coders.yaml  |  195 -
 sdks/common/pom.xml                             |   39 -
 sdks/common/runner-api/pom.xml                  |  109 -
 .../src/main/proto/beam_job_api.proto           |  143 -
 .../src/main/proto/beam_runner_api.proto        |  812 ----
 .../src/main/proto/standard_window_fns.proto    |   53 -
 sdks/go/BUILD.md                                |   63 +
 sdks/go/cmd/beamctl/cmd/artifact.go             |   98 +
 sdks/go/cmd/beamctl/cmd/root.go                 |   56 +
 sdks/go/cmd/beamctl/main.go                     |   31 +
 sdks/go/descriptor.xml                          |   29 +
 sdks/go/pkg/beam/artifact/gcsproxy/retrieval.go |  155 +
 sdks/go/pkg/beam/artifact/gcsproxy/staging.go   |  200 +
 sdks/go/pkg/beam/artifact/materialize.go        |  240 ++
 sdks/go/pkg/beam/artifact/materialize_test.go   |  238 ++
 sdks/go/pkg/beam/artifact/server_test.go        |  213 ++
 sdks/go/pkg/beam/artifact/stage.go              |  238 ++
 sdks/go/pkg/beam/artifact/stage_test.go         |   98 +
 .../beam/model/fnexecution_v1/beam_fn_api.pb.go | 2729 ++++++++++++++
 .../fnexecution_v1/beam_provision_api.pb.go     |  306 ++
 sdks/go/pkg/beam/model/gen.go                   |   22 +
 .../jobmanagement_v1/beam_artifact_api.pb.go    |  690 ++++
 .../model/jobmanagement_v1/beam_job_api.pb.go   |  903 +++++
 .../model/pipeline_v1/beam_runner_api.pb.go     | 3491 ++++++++++++++++++
 .../pkg/beam/model/pipeline_v1/endpoints.pb.go  |  160 +
 .../model/pipeline_v1/standard_window_fns.pb.go |  120 +
 sdks/go/pkg/beam/provision/provision_test.go    |   54 +
 sdks/go/pkg/beam/provision/provison.go          |   80 +
 sdks/go/pkg/beam/util/errorx/guarded.go         |   47 +
 sdks/go/pkg/beam/util/execx/exec.go             |   33 +
 sdks/go/pkg/beam/util/gcsx/gcs.go               |   88 +
 sdks/go/pkg/beam/util/grpcx/dial.go             |   37 +
 sdks/go/pkg/beam/util/grpcx/metadata.go         |   55 +
 sdks/go/pkg/beam/util/syscallx/syscall.go       |   27 +
 .../pkg/beam/util/syscallx/syscall_default.go   |   28 +
 sdks/go/pkg/beam/util/syscallx/syscall_linux.go |   38 +
 sdks/go/pom.xml                                 |  163 +
 sdks/java/build-tools/pom.xml                   |    2 +-
 .../src/main/resources/beam/findbugs-filter.xml |   51 +-
 .../resources/docker/file/openjdk7/Dockerfile   |   49 +
 .../docker/file/openjdk7/docker-entrypoint.sh   |   24 +
 .../resources/docker/file/openjdk8/Dockerfile   |   49 +
 .../docker/file/openjdk8/docker-entrypoint.sh   |   24 +
 .../resources/docker/git/openjdk8/Dockerfile    |   53 +
 .../docker/git/openjdk8/docker-entrypoint.sh    |   22 +
 .../resources/docker/release/python2/Dockerfile |   21 +
 sdks/java/container/Dockerfile                  |   28 +
 sdks/java/container/boot.go                     |  134 +
 sdks/java/container/pom.xml                     |  184 +
 sdks/java/core/pom.xml                          |   15 +-
 .../main/java/org/apache/beam/sdk/Pipeline.java |   12 +-
 .../org/apache/beam/sdk/PipelineResult.java     |   14 +-
 .../beam/sdk/annotations/Experimental.java      |    8 +-
 .../beam/sdk/annotations/package-info.java      |    4 +
 .../apache/beam/sdk/coders/BeamRecordCoder.java |  111 +
 .../apache/beam/sdk/coders/CoderRegistry.java   |    6 +
 .../apache/beam/sdk/coders/DefaultCoder.java    |   17 +-
 .../beam/sdk/coders/LengthPrefixCoder.java      |    3 +-
 .../beam/sdk/coders/SerializableCoder.java      |    5 +-
 .../apache/beam/sdk/coders/StructuredCoder.java |   42 +-
 .../org/apache/beam/sdk/coders/VoidCoder.java   |   12 +-
 .../apache/beam/sdk/coders/package-info.java    |    4 +
 .../java/org/apache/beam/sdk/io/AvroIO.java     |  544 ++-
 .../java/org/apache/beam/sdk/io/AvroSink.java   |    2 +-
 .../java/org/apache/beam/sdk/io/AvroSource.java |   32 +-
 .../apache/beam/sdk/io/BlockBasedSource.java    |   27 +-
 .../apache/beam/sdk/io/CompressedSource.java    |  292 +-
 .../org/apache/beam/sdk/io/Compression.java     |  228 ++
 .../org/apache/beam/sdk/io/CountingSource.java  |    3 -
 .../beam/sdk/io/DefaultFilenamePolicy.java      |   25 +-
 .../org/apache/beam/sdk/io/FileBasedSink.java   |  138 +-
 .../org/apache/beam/sdk/io/FileBasedSource.java |   29 +-
 .../java/org/apache/beam/sdk/io/FileIO.java     |  450 +++
 .../apache/beam/sdk/io/GenerateSequence.java    |    3 +-
 .../org/apache/beam/sdk/io/LocalFileSystem.java |    9 +-
 .../main/java/org/apache/beam/sdk/io/Match.java |  156 -
 .../beam/sdk/io/ReadAllViaFileBasedSource.java  |  100 +-
 .../apache/beam/sdk/io/ReadableFileCoder.java   |   50 +
 .../java/org/apache/beam/sdk/io/Source.java     |    2 +-
 .../java/org/apache/beam/sdk/io/TFRecordIO.java |  171 +-
 .../java/org/apache/beam/sdk/io/TextIO.java     |  590 +--
 .../java/org/apache/beam/sdk/io/TextSource.java |  117 +-
 .../java/org/apache/beam/sdk/io/WriteFiles.java |  127 +-
 .../apache/beam/sdk/io/WriteFilesResult.java    |   81 +
 .../org/apache/beam/sdk/io/package-info.java    |    2 +-
 .../beam/sdk/io/range/ByteKeyRangeTracker.java  |    5 +
 .../beam/sdk/io/range/OffsetRangeTracker.java   |   22 +-
 .../org/apache/beam/sdk/metrics/MetricName.java |    7 +
 .../apache/beam/sdk/metrics/package-info.java   |    4 +
 .../DefaultPipelineOptionsRegistrar.java        |    2 +
 .../beam/sdk/options/ExperimentalOptions.java   |   38 +
 .../beam/sdk/options/PipelineOptions.java       |   36 +
 .../sdk/options/PipelineOptionsFactory.java     |   45 +
 .../beam/sdk/options/SdkHarnessOptions.java     |  173 +
 .../apache/beam/sdk/options/ValueProvider.java  |   28 +-
 .../apache/beam/sdk/options/ValueProviders.java |   15 +-
 .../java/org/apache/beam/sdk/package-info.java  |    4 +
 .../beam/sdk/runners/TransformHierarchy.java    |   19 +-
 .../apache/beam/sdk/runners/package-info.java   |    4 +
 .../org/apache/beam/sdk/state/BagState.java     |    6 +
 .../apache/beam/sdk/state/CombiningState.java   |    5 +
 .../apache/beam/sdk/state/GroupingState.java    |   12 +-
 .../org/apache/beam/sdk/state/MapState.java     |   20 +-
 .../apache/beam/sdk/state/ReadableState.java    |    6 +
 .../apache/beam/sdk/state/ReadableStates.java   |    4 +-
 .../org/apache/beam/sdk/state/SetState.java     |   10 +-
 .../org/apache/beam/sdk/state/StateSpecs.java   |    4 +-
 .../org/apache/beam/sdk/state/package-info.java |    4 +
 .../beam/sdk/testing/FileChecksumMatcher.java   |   41 +-
 .../org/apache/beam/sdk/testing/PAssert.java    |   74 +-
 .../apache/beam/sdk/testing/PaneExtractors.java |   25 +-
 .../beam/sdk/testing/SourceTestUtils.java       |   11 +
 .../beam/sdk/testing/SuccessOrFailure.java      |   24 +-
 .../apache/beam/sdk/testing/TestPipeline.java   |   53 +-
 .../beam/sdk/testing/WindowFnTestUtils.java     |  141 +-
 .../apache/beam/sdk/testing/WindowSupplier.java |    4 +-
 .../apache/beam/sdk/testing/package-info.java   |    5 +
 .../sdk/transforms/ApproximateQuantiles.java    |   10 +-
 .../beam/sdk/transforms/ApproximateUnique.java  |    4 +-
 .../org/apache/beam/sdk/transforms/Combine.java |   10 +-
 .../apache/beam/sdk/transforms/CombineFns.java  |    2 +
 .../apache/beam/sdk/transforms/Contextful.java  |  127 +
 .../org/apache/beam/sdk/transforms/Create.java  |    5 +-
 .../apache/beam/sdk/transforms/Distinct.java    |    2 +
 .../apache/beam/sdk/transforms/DoFnTester.java  |   27 +-
 .../beam/sdk/transforms/FlatMapElements.java    |  150 +-
 .../org/apache/beam/sdk/transforms/Latest.java  |    2 +-
 .../apache/beam/sdk/transforms/MapElements.java |   77 +-
 .../org/apache/beam/sdk/transforms/Max.java     |    5 +-
 .../org/apache/beam/sdk/transforms/Min.java     |    8 +-
 .../apache/beam/sdk/transforms/PTransform.java  |    7 +-
 .../org/apache/beam/sdk/transforms/ParDo.java   |    8 +-
 .../beam/sdk/transforms/Requirements.java       |   61 +
 .../apache/beam/sdk/transforms/Reshuffle.java   |   47 +
 .../org/apache/beam/sdk/transforms/Top.java     |    5 +-
 .../org/apache/beam/sdk/transforms/View.java    |    9 +-
 .../org/apache/beam/sdk/transforms/Watch.java   |   82 +-
 .../apache/beam/sdk/transforms/WithKeys.java    |   12 +-
 .../sdk/transforms/display/DisplayData.java     |   25 +-
 .../sdk/transforms/display/package-info.java    |    4 +
 .../beam/sdk/transforms/join/CoGbkResult.java   |   15 +-
 .../transforms/join/KeyedPCollectionTuple.java  |   10 +-
 .../beam/sdk/transforms/join/RawUnionValue.java |    8 +-
 .../beam/sdk/transforms/join/package-info.java  |    4 +
 .../beam/sdk/transforms/package-info.java       |    4 +
 .../reflect/ByteBuddyDoFnInvokerFactory.java    |    3 +-
 .../sdk/transforms/reflect/DoFnInvoker.java     |   50 +-
 .../sdk/transforms/reflect/DoFnSignatures.java  |    1 +
 .../sdk/transforms/reflect/package-info.java    |    3 +
 .../splittabledofn/OffsetRangeTracker.java      |    5 +-
 .../transforms/splittabledofn/package-info.java |    4 +
 .../windowing/IncompatibleWindowException.java  |    2 +-
 .../MergeOverlappingIntervalWindows.java        |    4 +-
 .../beam/sdk/transforms/windowing/Trigger.java  |   20 +-
 .../beam/sdk/transforms/windowing/Window.java   |    1 +
 .../sdk/transforms/windowing/package-info.java  |    4 +
 .../org/apache/beam/sdk/util/ApiSurface.java    |    2 +
 .../org/apache/beam/sdk/util/CoderUtils.java    |    2 +-
 .../java/org/apache/beam/sdk/util/DoFnInfo.java |  104 +
 .../apache/beam/sdk/util/MutationDetectors.java |   79 +-
 .../beam/sdk/util/SerializableThrowable.java    |   49 +
 .../apache/beam/sdk/util/SerializableUtils.java |   69 +-
 .../org/apache/beam/sdk/values/BeamRecord.java  |  319 ++
 .../apache/beam/sdk/values/BeamRecordType.java  |   96 +
 .../apache/beam/sdk/values/TypeDescriptors.java |   37 +-
 .../java/org/apache/beam/sdk/PipelineTest.java  |   51 +-
 .../apache/beam/sdk/coders/AvroCoderTest.java   |   35 +-
 .../apache/beam/sdk/coders/CommonCoderTest.java |    4 +-
 .../apache/beam/sdk/coders/VoidCoderTest.java   |   11 +-
 .../java/org/apache/beam/sdk/io/AvroIOTest.java |  417 ++-
 .../apache/beam/sdk/io/AvroIOTransformTest.java |  324 --
 .../beam/sdk/io/CompressedSourceTest.java       |   96 +-
 .../apache/beam/sdk/io/FileBasedSinkTest.java   |   43 +-
 .../apache/beam/sdk/io/FileBasedSourceTest.java |    3 -
 .../java/org/apache/beam/sdk/io/FileIOTest.java |  313 ++
 .../org/apache/beam/sdk/io/FileSystemsTest.java |   15 +-
 .../beam/sdk/io/OffsetBasedSourceTest.java      |    3 -
 .../java/org/apache/beam/sdk/io/ReadTest.java   |    6 -
 .../java/org/apache/beam/sdk/io/SimpleSink.java |   23 +-
 .../org/apache/beam/sdk/io/TFRecordIOTest.java  |   35 +-
 .../org/apache/beam/sdk/io/TextIOReadTest.java  | 1305 ++++---
 .../org/apache/beam/sdk/io/TextIOWriteTest.java |   40 +-
 .../org/apache/beam/sdk/io/WriteFilesTest.java  |   11 +-
 .../sdk/io/range/ByteKeyRangeTrackerTest.java   |   23 +
 .../apache/beam/sdk/metrics/MetricsTest.java    |   28 +
 .../sdk/options/PipelineOptionsFactoryTest.java |   39 +
 .../beam/sdk/options/PipelineOptionsTest.java   |   11 +
 .../sdk/options/ProxyInvocationHandlerTest.java |    4 +-
 .../beam/sdk/options/SdkHarnessOptionsTest.java |   76 +
 .../beam/sdk/options/ValueProviderTest.java     |   38 +-
 .../runners/dataflow/TestCountingSource.java    |    3 -
 .../sdk/testing/InterceptingUrlClassLoader.java |   57 +
 .../apache/beam/sdk/testing/PAssertTest.java    |   41 +-
 .../beam/sdk/testing/PaneExtractorsTest.java    |    7 +-
 .../beam/sdk/testing/TestPipelineTest.java      |   37 +-
 .../transforms/ApproximateQuantilesTest.java    |  528 +--
 .../apache/beam/sdk/transforms/CreateTest.java  |   22 +-
 .../sdk/transforms/FlatMapElementsTest.java     |   35 +-
 .../apache/beam/sdk/transforms/FlattenTest.java |   35 +
 .../beam/sdk/transforms/MapElementsTest.java    |   42 +-
 .../apache/beam/sdk/transforms/ParDoTest.java   |   44 +-
 .../apache/beam/sdk/transforms/WatchTest.java   |   46 +-
 .../beam/sdk/transforms/WithKeysTest.java       |   30 +-
 .../transforms/reflect/DoFnInvokersTest.java    |   12 +-
 .../sdk/transforms/windowing/WindowTest.java    |  125 +-
 .../beam/sdk/util/MutationDetectorsTest.java    |   56 +
 .../beam/sdk/util/SerializableUtilsTest.java    |   60 +
 .../beam/sdk/values/TypeDescriptorsTest.java    |   17 +-
 .../google-cloud-platform-core/pom.xml          |    2 +-
 .../extensions/gcp/storage/GcsFileSystem.java   |    5 +-
 .../java/org/apache/beam/sdk/util/GcsUtil.java  |    6 +-
 .../org/apache/beam/sdk/util/GcsUtilTest.java   |   45 +
 sdks/java/extensions/jackson/pom.xml            |    2 +-
 sdks/java/extensions/join-library/pom.xml       |    2 +-
 sdks/java/extensions/pom.xml                    |    4 +-
 sdks/java/extensions/protobuf/pom.xml           |    2 +-
 sdks/java/extensions/sketching/pom.xml          |  104 +
 .../sketching/ApproximateDistinct.java          |  573 +++
 .../sdk/extensions/sketching/package-info.java  |   22 +
 .../sketching/ApproximateDistinctTest.java      |  209 ++
 sdks/java/extensions/sorter/pom.xml             |    2 +-
 sdks/java/extensions/sql/NOTICE                 |   45 +
 sdks/java/extensions/sql/pom.xml                |  275 ++
 .../sdk/extensions/sql/BeamRecordSqlType.java   |  186 +
 .../apache/beam/sdk/extensions/sql/BeamSql.java |  250 ++
 .../sdk/extensions/sql/BeamSqlRecordHelper.java |  217 ++
 .../beam/sdk/extensions/sql/BeamSqlUdf.java     |   43 +
 .../extensions/sql/example/BeamSqlExample.java  |  104 +
 .../extensions/sql/example/package-info.java    |   23 +
 .../sdk/extensions/sql/impl/BeamSqlCli.java     |   65 +
 .../sdk/extensions/sql/impl/BeamSqlEnv.java     |  135 +
 .../interpreter/BeamSqlExpressionExecutor.java  |   44 +
 .../sql/impl/interpreter/BeamSqlFnExecutor.java |  458 +++
 .../operator/BeamSqlCaseExpression.java         |   64 +
 .../operator/BeamSqlCastExpression.java         |  132 +
 .../interpreter/operator/BeamSqlExpression.java |   79 +
 .../operator/BeamSqlInputRefExpression.java     |   48 +
 .../interpreter/operator/BeamSqlPrimitive.java  |  157 +
 .../operator/BeamSqlReinterpretExpression.java  |   55 +
 .../operator/BeamSqlUdfExpression.java          |   92 +
 .../operator/BeamSqlWindowEndExpression.java    |   48 +
 .../operator/BeamSqlWindowExpression.java       |   51 +
 .../operator/BeamSqlWindowStartExpression.java  |   49 +
 .../sql/impl/interpreter/operator/UdafImpl.java |   87 +
 .../arithmetic/BeamSqlArithmeticExpression.java |  124 +
 .../arithmetic/BeamSqlDivideExpression.java     |   37 +
 .../arithmetic/BeamSqlMinusExpression.java      |   36 +
 .../arithmetic/BeamSqlModExpression.java        |   36 +
 .../arithmetic/BeamSqlMultiplyExpression.java   |   36 +
 .../arithmetic/BeamSqlPlusExpression.java       |   36 +
 .../operator/arithmetic/package-info.java       |   22 +
 .../comparison/BeamSqlCompareExpression.java    |   97 +
 .../comparison/BeamSqlEqualsExpression.java     |   49 +
 .../BeamSqlGreaterThanExpression.java           |   49 +
 .../BeamSqlGreaterThanOrEqualsExpression.java   |   49 +
 .../comparison/BeamSqlIsNotNullExpression.java  |   54 +
 .../comparison/BeamSqlIsNullExpression.java     |   54 +
 .../comparison/BeamSqlLessThanExpression.java   |   49 +
 .../BeamSqlLessThanOrEqualsExpression.java      |   49 +
 .../comparison/BeamSqlNotEqualsExpression.java  |   49 +
 .../operator/comparison/package-info.java       |   22 +
 .../date/BeamSqlCurrentDateExpression.java      |   45 +
 .../date/BeamSqlCurrentTimeExpression.java      |   53 +
 .../date/BeamSqlCurrentTimestampExpression.java |   49 +
 .../date/BeamSqlDateCeilExpression.java         |   55 +
 .../date/BeamSqlDateFloorExpression.java        |   55 +
 .../date/BeamSqlDatetimePlusExpression.java     |  129 +
 .../operator/date/BeamSqlExtractExpression.java |  102 +
 .../date/BeamSqlIntervalMultiplyExpression.java |  103 +
 .../operator/date/TimeUnitUtils.java            |   54 +
 .../interpreter/operator/date/package-info.java |   22 +
 .../operator/logical/BeamSqlAndExpression.java  |   48 +
 .../logical/BeamSqlLogicalExpression.java       |   46 +
 .../operator/logical/BeamSqlNotExpression.java  |   54 +
 .../operator/logical/BeamSqlOrExpression.java   |   48 +
 .../operator/logical/package-info.java          |   22 +
 .../operator/math/BeamSqlAbsExpression.java     |   74 +
 .../operator/math/BeamSqlAcosExpression.java    |   40 +
 .../operator/math/BeamSqlAsinExpression.java    |   40 +
 .../operator/math/BeamSqlAtan2Expression.java   |   42 +
 .../operator/math/BeamSqlAtanExpression.java    |   40 +
 .../operator/math/BeamSqlCeilExpression.java    |   45 +
 .../operator/math/BeamSqlCosExpression.java     |   40 +
 .../operator/math/BeamSqlCotExpression.java     |   40 +
 .../operator/math/BeamSqlDegreesExpression.java |   40 +
 .../operator/math/BeamSqlExpExpression.java     |   40 +
 .../operator/math/BeamSqlFloorExpression.java   |   45 +
 .../operator/math/BeamSqlLnExpression.java      |   40 +
 .../operator/math/BeamSqlLogExpression.java     |   40 +
 .../math/BeamSqlMathBinaryExpression.java       |   65 +
 .../math/BeamSqlMathUnaryExpression.java        |   60 +
 .../operator/math/BeamSqlPiExpression.java      |   43 +
 .../operator/math/BeamSqlPowerExpression.java   |   44 +
 .../operator/math/BeamSqlRadiansExpression.java |   40 +
 .../operator/math/BeamSqlRandExpression.java    |   55 +
 .../math/BeamSqlRandIntegerExpression.java      |   59 +
 .../operator/math/BeamSqlRoundExpression.java   |  107 +
 .../operator/math/BeamSqlSignExpression.java    |   72 +
 .../operator/math/BeamSqlSinExpression.java     |   40 +
 .../operator/math/BeamSqlTanExpression.java     |   40 +
 .../math/BeamSqlTruncateExpression.java         |   75 +
 .../interpreter/operator/math/package-info.java |   22 +
 .../impl/interpreter/operator/package-info.java |   22 +
 .../string/BeamSqlCharLengthExpression.java     |   40 +
 .../string/BeamSqlConcatExpression.java         |   63 +
 .../string/BeamSqlInitCapExpression.java        |   56 +
 .../operator/string/BeamSqlLowerExpression.java |   40 +
 .../string/BeamSqlOverlayExpression.java        |   77 +
 .../string/BeamSqlPositionExpression.java       |   73 +
 .../string/BeamSqlStringUnaryExpression.java    |   44 +
 .../string/BeamSqlSubstringExpression.java      |   83 +
 .../operator/string/BeamSqlTrimExpression.java  |  102 +
 .../operator/string/BeamSqlUpperExpression.java |   40 +
 .../operator/string/package-info.java           |   22 +
 .../sql/impl/interpreter/package-info.java      |   22 +
 .../sdk/extensions/sql/impl/package-info.java   |   22 +
 .../sql/impl/planner/BeamQueryPlanner.java      |  168 +
 .../sql/impl/planner/BeamRelDataTypeSystem.java |   40 +
 .../sql/impl/planner/BeamRuleSets.java          |   75 +
 .../sql/impl/planner/package-info.java          |   24 +
 .../sql/impl/rel/BeamAggregationRel.java        |  182 +
 .../extensions/sql/impl/rel/BeamFilterRel.java  |   69 +
 .../extensions/sql/impl/rel/BeamIOSinkRel.java  |   75 +
 .../sql/impl/rel/BeamIOSourceRel.java           |   62 +
 .../sql/impl/rel/BeamIntersectRel.java          |   58 +
 .../extensions/sql/impl/rel/BeamJoinRel.java    |  298 ++
 .../sql/impl/rel/BeamLogicalConvention.java     |   72 +
 .../extensions/sql/impl/rel/BeamMinusRel.java   |   56 +
 .../extensions/sql/impl/rel/BeamProjectRel.java |   80 +
 .../extensions/sql/impl/rel/BeamRelNode.java    |   39 +
 .../sql/impl/rel/BeamSetOperatorRelBase.java    |   98 +
 .../extensions/sql/impl/rel/BeamSortRel.java    |  235 ++
 .../sql/impl/rel/BeamSqlRelUtils.java           |   72 +
 .../extensions/sql/impl/rel/BeamUnionRel.java   |   88 +
 .../extensions/sql/impl/rel/BeamValuesRel.java  |   79 +
 .../extensions/sql/impl/rel/package-info.java   |   23 +
 .../sql/impl/rule/BeamAggregationRule.java      |  162 +
 .../sql/impl/rule/BeamFilterRule.java           |   49 +
 .../sql/impl/rule/BeamIOSinkRule.java           |   81 +
 .../sql/impl/rule/BeamIOSourceRule.java         |   49 +
 .../sql/impl/rule/BeamIntersectRule.java        |   50 +
 .../extensions/sql/impl/rule/BeamJoinRule.java  |   53 +
 .../extensions/sql/impl/rule/BeamMinusRule.java |   50 +
 .../sql/impl/rule/BeamProjectRule.java          |   50 +
 .../extensions/sql/impl/rule/BeamSortRule.java  |   51 +
 .../extensions/sql/impl/rule/BeamUnionRule.java |   50 +
 .../sql/impl/rule/BeamValuesRule.java           |   48 +
 .../extensions/sql/impl/rule/package-info.java  |   23 +
 .../sql/impl/schema/BaseBeamTable.java          |   35 +
 .../extensions/sql/impl/schema/BeamIOType.java  |   28 +
 .../sql/impl/schema/BeamPCollectionTable.java   |   63 +
 .../sql/impl/schema/BeamSqlTable.java           |   54 +
 .../sql/impl/schema/BeamTableUtils.java         |  118 +
 .../impl/schema/kafka/BeamKafkaCSVTable.java    |  109 +
 .../sql/impl/schema/kafka/BeamKafkaTable.java   |  109 +
 .../sql/impl/schema/kafka/package-info.java     |   22 +
 .../sql/impl/schema/package-info.java           |   22 +
 .../sql/impl/schema/text/BeamTextCSVTable.java  |   70 +
 .../schema/text/BeamTextCSVTableIOReader.java   |   58 +
 .../schema/text/BeamTextCSVTableIOWriter.java   |   58 +
 .../sql/impl/schema/text/BeamTextTable.java     |   41 +
 .../sql/impl/schema/text/package-info.java      |   22 +
 .../transform/BeamAggregationTransforms.java    |  311 ++
 .../impl/transform/BeamBuiltinAggregations.java |  557 +++
 .../sql/impl/transform/BeamJoinTransforms.java  |  161 +
 .../transform/BeamSetOperatorsTransforms.java   |  111 +
 .../sql/impl/transform/BeamSqlFilterFn.java     |   63 +
 .../transform/BeamSqlOutputToConsoleFn.java     |   41 +
 .../sql/impl/transform/BeamSqlProjectFn.java    |   72 +
 .../sql/impl/transform/package-info.java        |   22 +
 .../extensions/sql/impl/utils/CalciteUtils.java |  113 +
 .../extensions/sql/impl/utils/SqlTypeUtils.java |   63 +
 .../extensions/sql/impl/utils/package-info.java |   22 +
 .../beam/sdk/extensions/sql/package-info.java   |   22 +
 .../sql/src/main/resources/log4j.properties     |   23 +
 ...ged.org.codehaus.commons.compiler.properties |   18 +
 .../extensions/sql/BeamSqlApiSurfaceTest.java   |   57 +
 .../sql/BeamSqlDslAggregationTest.java          |  400 ++
 .../beam/sdk/extensions/sql/BeamSqlDslBase.java |  136 +
 .../extensions/sql/BeamSqlDslFilterTest.java    |  155 +
 .../sdk/extensions/sql/BeamSqlDslJoinTest.java  |  188 +
 .../extensions/sql/BeamSqlDslProjectTest.java   |  227 ++
 .../extensions/sql/BeamSqlDslUdfUdafTest.java   |  139 +
 .../beam/sdk/extensions/sql/TestUtils.java      |  190 +
 .../impl/interpreter/BeamSqlFnExecutorTest.java |  446 +++
 .../interpreter/BeamSqlFnExecutorTestBase.java  |   88 +
 .../operator/BeamNullExperssionTest.java        |   55 +
 .../operator/BeamSqlAndOrExpressionTest.java    |   61 +
 .../operator/BeamSqlCaseExpressionTest.java     |   93 +
 .../operator/BeamSqlCastExpressionTest.java     |  129 +
 .../operator/BeamSqlCompareExpressionTest.java  |  115 +
 .../operator/BeamSqlInputRefExpressionTest.java |   57 +
 .../operator/BeamSqlPrimitiveTest.java          |   59 +
 .../BeamSqlReinterpretExpressionTest.java       |   75 +
 .../operator/BeamSqlUdfExpressionTest.java      |   51 +
 .../BeamSqlArithmeticExpressionTest.java        |  237 ++
 .../date/BeamSqlCurrentDateExpressionTest.java  |   38 +
 .../date/BeamSqlCurrentTimeExpressionTest.java  |   39 +
 .../BeamSqlCurrentTimestampExpressionTest.java  |   39 +
 .../date/BeamSqlDateCeilExpressionTest.java     |   50 +
 .../date/BeamSqlDateExpressionTestBase.java     |   52 +
 .../date/BeamSqlDateFloorExpressionTest.java    |   49 +
 .../date/BeamSqlDatetimePlusExpressionTest.java |  155 +
 .../date/BeamSqlExtractExpressionTest.java      |  103 +
 .../BeamSqlIntervalMultiplyExpressionTest.java  |  107 +
 .../operator/date/TimeUnitUtilsTest.java        |   54 +
 .../logical/BeamSqlNotExpressionTest.java       |   47 +
 .../math/BeamSqlMathBinaryExpressionTest.java   |  215 ++
 .../math/BeamSqlMathUnaryExpressionTest.java    |  312 ++
 .../string/BeamSqlCharLengthExpressionTest.java |   44 +
 .../string/BeamSqlConcatExpressionTest.java     |   66 +
 .../string/BeamSqlInitCapExpressionTest.java    |   54 +
 .../string/BeamSqlLowerExpressionTest.java      |   44 +
 .../string/BeamSqlOverlayExpressionTest.java    |   87 +
 .../string/BeamSqlPositionExpressionTest.java   |   84 +
 .../BeamSqlStringUnaryExpressionTest.java       |   52 +
 .../string/BeamSqlSubstringExpressionTest.java  |  101 +
 .../string/BeamSqlTrimExpressionTest.java       |  103 +
 .../string/BeamSqlUpperExpressionTest.java      |   44 +
 .../extensions/sql/impl/rel/BaseRelTest.java    |   34 +
 .../sql/impl/rel/BeamIntersectRelTest.java      |  118 +
 .../rel/BeamJoinRelBoundedVsBoundedTest.java    |  203 +
 .../rel/BeamJoinRelUnboundedVsBoundedTest.java  |  240 ++
 .../BeamJoinRelUnboundedVsUnboundedTest.java    |  218 ++
 .../sql/impl/rel/BeamMinusRelTest.java          |  117 +
 .../impl/rel/BeamSetOperatorRelBaseTest.java    |  105 +
 .../sql/impl/rel/BeamSortRelTest.java           |  257 ++
 .../sql/impl/rel/BeamUnionRelTest.java          |  103 +
 .../sql/impl/rel/BeamValuesRelTest.java         |  104 +
 .../sdk/extensions/sql/impl/rel/CheckSize.java  |   41 +
 .../sql/impl/schema/BeamSqlRowCoderTest.java    |   77 +
 .../schema/kafka/BeamKafkaCSVTableTest.java     |  107 +
 .../impl/schema/text/BeamTextCSVTableTest.java  |  176 +
 .../transform/BeamAggregationTransformTest.java |  453 +++
 .../schema/transform/BeamTransformBaseTest.java |   97 +
 .../sql/impl/utils/SqlTypeUtilsTest.java        |   76 +
 ...amSqlArithmeticOperatorsIntegrationTest.java |  165 +
 ...mSqlBuiltinFunctionsIntegrationTestBase.java |  168 +
 ...amSqlComparisonOperatorsIntegrationTest.java |  329 ++
 ...mSqlConditionalFunctionsIntegrationTest.java |   60 +
 .../BeamSqlDateFunctionsIntegrationTest.java    |  125 +
 .../BeamSqlLogicalFunctionsIntegrationTest.java |   43 +
 .../BeamSqlMathFunctionsIntegrationTest.java    |  351 ++
 .../BeamSqlStringFunctionsIntegrationTest.java  |   51 +
 .../extensions/sql/mock/MockedBoundedTable.java |  134 +
 .../sdk/extensions/sql/mock/MockedTable.java    |   42 +
 .../sql/mock/MockedUnboundedTable.java          |  110 +
 sdks/java/fn-execution/pom.xml                  |   82 +
 .../harness/channel/ManagedChannelFactory.java  |   82 +
 .../harness/channel/SocketAddressFactory.java   |   64 +
 .../beam/harness/channel/package-info.java      |   22 +
 .../channel/ManagedChannelFactoryTest.java      |   71 +
 .../channel/SocketAddressFactoryTest.java       |   56 +
 .../org/apache/beam/harness/test/Consumer.java  |   26 +
 .../org/apache/beam/harness/test/Supplier.java  |   26 +
 .../apache/beam/harness/test/TestExecutors.java |   93 +
 .../beam/harness/test/TestExecutorsTest.java    |  175 +
 .../apache/beam/harness/test/TestStreams.java   |  185 +
 .../beam/harness/test/TestStreamsTest.java      |  109 +
 sdks/java/harness/pom.xml                       |  128 +-
 .../beam/fn/harness/BeamFnDataReadRunner.java   |   15 +-
 .../beam/fn/harness/BeamFnDataWriteRunner.java  |   15 +-
 .../beam/fn/harness/BoundedSourceRunner.java    |   14 +-
 .../apache/beam/fn/harness/FnApiDoFnRunner.java |  398 +-
 .../org/apache/beam/fn/harness/FnHarness.java   |   64 +-
 .../org/apache/beam/fn/harness/IdGenerator.java |   33 +
 .../fn/harness/PTransformRunnerFactory.java     |    7 +-
 .../harness/channel/ManagedChannelFactory.java  |   86 -
 .../harness/channel/SocketAddressFactory.java   |   64 -
 .../beam/fn/harness/channel/package-info.java   |   22 -
 .../fn/harness/control/BeamFnControlClient.java |   15 +-
 .../harness/control/ProcessBundleHandler.java   |  154 +-
 .../fn/harness/control/RegisterHandler.java     |    8 +-
 .../BeamFnDataBufferingOutboundObserver.java    |    8 +-
 .../beam/fn/harness/data/BeamFnDataClient.java  |    7 +-
 .../fn/harness/data/BeamFnDataGrpcClient.java   |   19 +-
 .../harness/data/BeamFnDataGrpcMultiplexer.java |   24 +-
 .../harness/data/BeamFnDataInboundObserver.java |    4 +-
 .../beam/fn/harness/fake/FakeStepContext.java   |   39 -
 .../beam/fn/harness/fake/package-info.java      |   22 -
 .../beam/fn/harness/fn/ThrowingBiConsumer.java  |   33 +
 .../fn/harness/logging/BeamFnLoggingClient.java |  202 +-
 .../beam/fn/harness/state/BagUserState.java     |  121 +
 .../fn/harness/state/BeamFnStateClient.java     |   39 +
 .../state/BeamFnStateGrpcClientCache.java       |  173 +
 .../state/LazyCachingIteratorToIterable.java    |   72 +
 .../harness/state/StateFetchingIterators.java   |  126 +
 .../beam/fn/harness/state/package-info.java     |   22 +
 .../harness/stream/BufferingStreamObserver.java |   16 +-
 .../beam/fn/harness/stream/DataStreams.java     |   73 +-
 .../harness/stream/StreamObserverFactory.java   |    4 +-
 .../fn/harness/BeamFnDataReadRunnerTest.java    |   19 +-
 .../fn/harness/BeamFnDataWriteRunnerTest.java   |   15 +-
 .../fn/harness/BoundedSourceRunnerTest.java     |   21 +-
 .../beam/fn/harness/FnApiDoFnRunnerTest.java    |  247 +-
 .../apache/beam/fn/harness/FnHarnessTest.java   |   25 +-
 .../apache/beam/fn/harness/IdGeneratorTest.java |   40 +
 .../channel/ManagedChannelFactoryTest.java      |   74 -
 .../channel/SocketAddressFactoryTest.java       |   56 -
 .../control/BeamFnControlClientTest.java        |   13 +-
 .../control/ProcessBundleHandlerTest.java       |  160 +-
 .../fn/harness/control/RegisterHandlerTest.java |   10 +-
 ...BeamFnDataBufferingOutboundObserverTest.java |    4 +-
 .../harness/data/BeamFnDataGrpcClientTest.java  |   30 +-
 .../data/BeamFnDataGrpcMultiplexerTest.java     |    9 +-
 .../data/BeamFnDataInboundObserverTest.java     |    2 +-
 .../logging/BeamFnLoggingClientTest.java        |  136 +-
 .../beam/fn/harness/state/BagUserStateTest.java |  106 +
 .../state/BeamFnStateGrpcClientCacheTest.java   |  234 ++
 .../fn/harness/state/FakeBeamFnStateClient.java |  110 +
 .../LazyCachingIteratorToIterableTest.java      |   76 +
 .../state/StateFetchingIteratorsTest.java       |   99 +
 .../stream/BufferingStreamObserverTest.java     |   12 +-
 .../beam/fn/harness/stream/DataStreamsTest.java |  165 +-
 .../stream/DirectStreamObserverTest.java        |    8 +-
 .../beam/fn/harness/test/TestExecutors.java     |   85 -
 .../beam/fn/harness/test/TestExecutorsTest.java |  160 -
 .../beam/fn/harness/test/TestStreams.java       |  162 -
 .../beam/fn/harness/test/TestStreamsTest.java   |   84 -
 sdks/java/io/amqp/pom.xml                       |   24 +-
 .../org/apache/beam/sdk/io/amqp/AmqpIO.java     |   33 +-
 .../org/apache/beam/sdk/io/amqp/AmqpIOTest.java |  112 +-
 sdks/java/io/cassandra/pom.xml                  |    2 +-
 .../beam/sdk/io/cassandra/CassandraIO.java      |   64 +-
 sdks/java/io/common/pom.xml                     |    2 +-
 .../sdk/io/common/IOTestPipelineOptions.java    |    6 +
 .../elasticsearch-tests-2/pom.xml               |   60 +
 .../src/test/contrib/create_elk_container.sh    |   24 +
 .../sdk/io/elasticsearch/ElasticsearchIOIT.java |  123 +
 .../io/elasticsearch/ElasticsearchIOTest.java   |  185 +
 .../elasticsearch-tests-5/pom.xml               |  124 +
 .../src/test/contrib/create_elk_container.sh    |   24 +
 .../sdk/io/elasticsearch/ElasticsearchIOIT.java |  122 +
 .../io/elasticsearch/ElasticsearchIOTest.java   |  185 +
 .../org/elasticsearch/bootstrap/JarHell.java    |   39 +
 .../elasticsearch-tests-common/pom.xml          |   77 +
 .../elasticsearch/ElasticSearchIOTestUtils.java |  141 +
 .../elasticsearch/ElasticsearchIOITCommon.java  |   92 +
 .../ElasticsearchIOTestCommon.java              |  306 ++
 sdks/java/io/elasticsearch-tests/pom.xml        |  144 +
 sdks/java/io/elasticsearch/pom.xml              |  228 +-
 .../sdk/io/elasticsearch/ElasticsearchIO.java   |  448 ++-
 .../beam/sdk/io/elasticsearch/package-info.java |    1 -
 .../src/test/contrib/create_elk_container.sh    |   24 -
 .../elasticsearch/ElasticSearchIOTestUtils.java |  138 -
 .../sdk/io/elasticsearch/ElasticsearchIOIT.java |  155 -
 .../io/elasticsearch/ElasticsearchIOTest.java   |  355 --
 .../elasticsearch/ElasticsearchTestDataSet.java |   97 -
 sdks/java/io/google-cloud-platform/pom.xml      |    9 +-
 .../beam/sdk/io/gcp/bigquery/BatchLoads.java    |  473 ++-
 .../sdk/io/gcp/bigquery/BigQueryHelpers.java    |   28 +-
 .../beam/sdk/io/gcp/bigquery/BigQueryIO.java    |  755 +++-
 .../io/gcp/bigquery/BigQueryQuerySource.java    |   49 +-
 .../sdk/io/gcp/bigquery/BigQueryServices.java   |   53 +-
 .../io/gcp/bigquery/BigQueryServicesImpl.java   |   82 +-
 .../sdk/io/gcp/bigquery/BigQuerySourceBase.java |  131 +-
 .../gcp/bigquery/BigQueryTableRowIterator.java  |  501 ---
 .../io/gcp/bigquery/BigQueryTableSource.java    |   40 +-
 .../sdk/io/gcp/bigquery/CalculateSchemas.java   |   78 -
 .../beam/sdk/io/gcp/bigquery/CreateTables.java  |   23 +-
 .../io/gcp/bigquery/DynamicDestinations.java    |   13 +-
 .../bigquery/DynamicDestinationsHelpers.java    |   27 +-
 .../sdk/io/gcp/bigquery/ReifyAsIterable.java    |   51 +
 .../sdk/io/gcp/bigquery/SchemaAndRecord.java    |   43 +
 .../io/gcp/bigquery/StreamingWriteTables.java   |   10 +-
 .../sdk/io/gcp/bigquery/TableDestination.java   |   43 +-
 .../io/gcp/bigquery/TableDestinationCoder.java  |    2 +
 .../gcp/bigquery/TableDestinationCoderV2.java   |   59 +
 .../io/gcp/bigquery/WriteBundlesToFiles.java    |   43 +-
 .../bigquery/WriteGroupedRecordsToFiles.java    |    7 +-
 .../sdk/io/gcp/bigquery/WritePartition.java     |   13 +-
 .../beam/sdk/io/gcp/bigquery/WriteRename.java   |  112 +-
 .../beam/sdk/io/gcp/bigquery/WriteTables.java   |  189 +-
 .../beam/sdk/io/gcp/bigtable/BigtableIO.java    |  135 +-
 .../io/gcp/bigtable/BigtableServiceImpl.java    |   16 +-
 .../beam/sdk/io/gcp/datastore/DatastoreV1.java  |  111 +-
 .../apache/beam/sdk/io/gcp/pubsub/PubsubIO.java |   35 +-
 .../sdk/io/gcp/pubsub/PubsubUnboundedSink.java  |    6 +-
 .../io/gcp/pubsub/PubsubUnboundedSource.java    |   41 +-
 .../sdk/io/gcp/spanner/AbstractSpannerFn.java   |   71 -
 .../sdk/io/gcp/spanner/CreateTransactionFn.java |   22 +-
 .../io/gcp/spanner/MutationGroupEncoder.java    |  660 ++++
 .../io/gcp/spanner/MutationSizeEstimator.java   |   48 +
 .../sdk/io/gcp/spanner/NaiveSpannerReadFn.java  |   19 +-
 .../beam/sdk/io/gcp/spanner/OrderedCode.java    |  764 ++++
 .../sdk/io/gcp/spanner/ReadSpannerSchema.java   |   94 +
 .../sdk/io/gcp/spanner/SerializedMutation.java  |   35 +
 .../io/gcp/spanner/SerializedMutationCoder.java |   60 +
 .../sdk/io/gcp/spanner/SpannerAccessor.java     |   43 +
 .../beam/sdk/io/gcp/spanner/SpannerConfig.java  |   41 +-
 .../beam/sdk/io/gcp/spanner/SpannerIO.java      |  384 +-
 .../beam/sdk/io/gcp/spanner/SpannerSchema.java  |  144 +
 .../sdk/io/gcp/spanner/SpannerWriteGroupFn.java |  125 -
 .../beam/sdk/io/gcp/GcpApiSurfaceTest.java      |    2 +
 .../sdk/io/gcp/bigquery/BigQueryIOTest.java     |  811 ++--
 .../bigquery/BigQueryTableRowIteratorTest.java  |  358 --
 .../sdk/io/gcp/bigquery/BigQueryUtilTest.java   |  187 -
 .../io/gcp/bigquery/FakeBigQueryServices.java   |   78 -
 .../sdk/io/gcp/bigquery/FakeDatasetService.java |   21 +-
 .../sdk/io/gcp/bigquery/FakeJobService.java     |   46 +-
 .../sdk/io/gcp/bigquery/TableContainer.java     |    2 +
 .../sdk/io/gcp/bigtable/BigtableIOTest.java     |   30 +-
 .../sdk/io/gcp/datastore/DatastoreV1Test.java   |  167 +-
 .../sdk/io/gcp/datastore/SplitQueryFnIT.java    |    5 +-
 .../beam/sdk/io/gcp/pubsub/PubsubIOTest.java    |   55 +
 .../gcp/spanner/MutationGroupEncoderTest.java   |  636 ++++
 .../sdk/io/gcp/spanner/OrderedCodeTest.java     |  890 +++++
 .../io/gcp/spanner/ReadSpannerSchemaTest.java   |  134 +
 .../sdk/io/gcp/spanner/SpannerIOReadTest.java   |   69 -
 .../sdk/io/gcp/spanner/SpannerIOWriteTest.java  |  453 ++-
 .../sdk/io/gcp/spanner/SpannerSchemaTest.java   |   61 +
 .../beam/sdk/io/gcp/spanner/SpannerWriteIT.java |    5 +-
 sdks/java/io/hadoop-common/pom.xml              |    2 +-
 sdks/java/io/hadoop-file-system/pom.xml         |    2 +-
 .../beam/sdk/io/hdfs/HadoopFileSystem.java      |   20 +-
 .../beam/sdk/io/hdfs/HadoopFileSystemTest.java  |   66 +-
 sdks/java/io/hadoop/input-format/pom.xml        |    7 +-
 .../hadoop/inputformat/HadoopInputFormatIO.java |   48 +-
 .../inputformat/HadoopInputFormatIOTest.java    |   14 +-
 sdks/java/io/hadoop/jdk1.8-tests/pom.xml        |    2 +-
 sdks/java/io/hadoop/pom.xml                     |    2 +-
 sdks/java/io/hbase/pom.xml                      |    2 +-
 .../io/hbase/HBaseCoderProviderRegistrar.java   |    8 +-
 .../org/apache/beam/sdk/io/hbase/HBaseIO.java   | 1001 ++---
 .../beam/sdk/io/hbase/HBaseMutationCoder.java   |   27 +-
 .../beam/sdk/io/hbase/HBaseResultCoder.java     |    6 +-
 .../beam/sdk/io/hbase/SerializableScan.java     |   37 +-
 .../hbase/HBaseCoderProviderRegistrarTest.java  |    4 +-
 .../apache/beam/sdk/io/hbase/HBaseIOTest.java   |  754 ++--
 .../sdk/io/hbase/HBaseMutationCoderTest.java    |    4 +-
 .../beam/sdk/io/hbase/HBaseResultCoderTest.java |    4 +-
 .../beam/sdk/io/hbase/SerializableScanTest.java |    6 +-
 sdks/java/io/hcatalog/pom.xml                   |    2 +-
 .../apache/beam/sdk/io/hcatalog/HCatalogIO.java |   23 +-
 .../beam/sdk/io/hcatalog/HCatalogIOTest.java    |   24 +-
 sdks/java/io/jdbc/pom.xml                       |    2 +-
 .../org/apache/beam/sdk/io/jdbc/JdbcIO.java     |  313 +-
 .../org/apache/beam/sdk/io/jdbc/JdbcIOTest.java |    6 -
 sdks/java/io/jms/pom.xml                        |    2 +-
 .../java/org/apache/beam/sdk/io/jms/JmsIO.java  |   71 +-
 sdks/java/io/kafka/README.md                    |   36 +
 sdks/java/io/kafka/pom.xml                      |   19 +-
 .../org/apache/beam/sdk/io/kafka/KafkaIO.java   |  790 +++-
 .../apache/beam/sdk/io/kafka/ProducerSpEL.java  |  135 +
 .../apache/beam/sdk/io/kafka/KafkaIOTest.java   |  233 +-
 sdks/java/io/kinesis/pom.xml                    |    8 +-
 .../beam/sdk/io/kinesis/AWSClientsProvider.java |   36 +
 .../sdk/io/kinesis/GetKinesisRecordsResult.java |    8 +-
 .../sdk/io/kinesis/KinesisClientProvider.java   |   33 -
 .../apache/beam/sdk/io/kinesis/KinesisIO.java   |  125 +-
 .../beam/sdk/io/kinesis/KinesisReader.java      |  146 +-
 .../sdk/io/kinesis/KinesisReaderCheckpoint.java |   18 -
 .../beam/sdk/io/kinesis/KinesisSource.java      |   40 +-
 .../apache/beam/sdk/io/kinesis/RoundRobin.java  |   54 -
 .../beam/sdk/io/kinesis/ShardCheckpoint.java    |    8 +-
 .../beam/sdk/io/kinesis/ShardReadersPool.java   |  162 +
 .../sdk/io/kinesis/ShardRecordsIterator.java    |   88 +-
 .../sdk/io/kinesis/SimplifiedKinesisClient.java |   98 +-
 .../io/kinesis/TransientKinesisException.java   |    4 +-
 .../beam/sdk/io/kinesis/AmazonKinesisMock.java  |   19 +-
 .../sdk/io/kinesis/KinesisMockReadTest.java     |    5 +-
 .../beam/sdk/io/kinesis/KinesisReaderIT.java    |    5 +-
 .../beam/sdk/io/kinesis/KinesisReaderTest.java  |  152 +-
 .../beam/sdk/io/kinesis/RoundRobinTest.java     |   59 -
 .../sdk/io/kinesis/ShardReadersPoolTest.java    |  185 +
 .../io/kinesis/ShardRecordsIteratorTest.java    |   35 +-
 .../io/kinesis/SimplifiedKinesisClientTest.java |  107 +
 sdks/java/io/mongodb/pom.xml                    |    2 +-
 .../beam/sdk/io/mongodb/MongoDbGridFSIO.java    |    4 -
 .../apache/beam/sdk/io/mongodb/MongoDbIO.java   |   62 +-
 .../sdk/io/mongodb/MongoDBGridFSIOTest.java     |    2 +-
 .../beam/sdk/io/mongodb/MongoDbIOTest.java      |   11 +
 sdks/java/io/mqtt/pom.xml                       |    2 +-
 .../org/apache/beam/sdk/io/mqtt/MqttIO.java     |   49 +-
 sdks/java/io/pom.xml                            |   12 +-
 sdks/java/io/redis/pom.xml                      |   90 +
 .../io/redis/RedisConnectionConfiguration.java  |  122 +
 .../org/apache/beam/sdk/io/redis/RedisIO.java   |  451 +++
 .../apache/beam/sdk/io/redis/package-info.java  |   22 +
 .../apache/beam/sdk/io/redis/RedisIOTest.java   |  109 +
 sdks/java/io/solr/pom.xml                       |  145 +
 .../beam/sdk/io/solr/AuthorizedSolrClient.java  |   91 +
 .../beam/sdk/io/solr/JavaBinCodecCoder.java     |   98 +
 .../org/apache/beam/sdk/io/solr/SolrIO.java     |  705 ++++
 .../apache/beam/sdk/io/solr/package-info.java   |   20 +
 .../beam/sdk/io/solr/JavaBinCodecCoderTest.java |   81 +
 .../org/apache/beam/sdk/io/solr/SolrIOTest.java |  269 ++
 .../beam/sdk/io/solr/SolrIOTestUtils.java       |  132 +
 .../resources/cloud-minimal/conf/schema.xml     |   29 +
 .../resources/cloud-minimal/conf/solrconfig.xml |   48 +
 sdks/java/io/tika/pom.xml                       |  113 +
 .../apache/beam/sdk/io/tika/ParseResult.java    |  144 +
 .../org/apache/beam/sdk/io/tika/TikaIO.java     |  284 ++
 .../apache/beam/sdk/io/tika/package-info.java   |   22 +
 .../beam/sdk/io/tika/ParseResultTest.java       |   83 +
 .../org/apache/beam/sdk/io/tika/TikaIOTest.java |  149 +
 .../java/io/tika/src/test/resources/damaged.pdf |    2 +
 .../resources/valid/apache-beam-tika-pdf.zip    |  Bin 0 -> 11685 bytes
 .../test/resources/valid/apache-beam-tika.odt   |  Bin 0 -> 12540 bytes
 sdks/java/io/xml/pom.xml                        |    2 +-
 .../java/org/apache/beam/sdk/io/xml/XmlIO.java  |  375 +-
 .../org/apache/beam/sdk/io/xml/XmlSink.java     |   10 -
 .../org/apache/beam/sdk/io/xml/XmlSource.java   |   63 +-
 .../org/apache/beam/sdk/io/xml/XmlSinkTest.java |   12 +-
 .../apache/beam/sdk/io/xml/XmlSourceTest.java   |   74 +-
 sdks/java/java8tests/pom.xml                    |    2 +-
 sdks/java/javadoc/ant.xml                       |    1 +
 sdks/java/javadoc/pom.xml                       |   22 +-
 .../maven-archetypes/examples-java8/pom.xml     |    2 +-
 .../main/resources/archetype-resources/pom.xml  |   13 +
 sdks/java/maven-archetypes/examples/pom.xml     |    2 +-
 sdks/java/maven-archetypes/pom.xml              |    2 +-
 sdks/java/maven-archetypes/starter/pom.xml      |    2 +-
 sdks/java/nexmark/README.md                     |  340 ++
 sdks/java/nexmark/pom.xml                       |  265 ++
 .../java/org/apache/beam/sdk/nexmark/Main.java  |  303 ++
 .../org/apache/beam/sdk/nexmark/Monitor.java    |   78 +
 .../beam/sdk/nexmark/NexmarkConfiguration.java  |  721 ++++
 .../beam/sdk/nexmark/NexmarkLauncher.java       | 1157 ++++++
 .../apache/beam/sdk/nexmark/NexmarkOptions.java |  403 ++
 .../apache/beam/sdk/nexmark/NexmarkPerf.java    |  207 ++
 .../apache/beam/sdk/nexmark/NexmarkSuite.java   |  112 +
 .../apache/beam/sdk/nexmark/NexmarkUtils.java   |  674 ++++
 .../apache/beam/sdk/nexmark/model/Auction.java  |  187 +
 .../beam/sdk/nexmark/model/AuctionBid.java      |   85 +
 .../beam/sdk/nexmark/model/AuctionCount.java    |   84 +
 .../beam/sdk/nexmark/model/AuctionPrice.java    |   88 +
 .../org/apache/beam/sdk/nexmark/model/Bid.java  |  177 +
 .../beam/sdk/nexmark/model/BidsPerSession.java  |   87 +
 .../beam/sdk/nexmark/model/CategoryPrice.java   |   97 +
 .../org/apache/beam/sdk/nexmark/model/Done.java |   80 +
 .../apache/beam/sdk/nexmark/model/Event.java    |  171 +
 .../beam/sdk/nexmark/model/IdNameReserve.java   |   98 +
 .../beam/sdk/nexmark/model/KnownSize.java       |   26 +
 .../beam/sdk/nexmark/model/NameCityStateId.java |  103 +
 .../apache/beam/sdk/nexmark/model/Person.java   |  163 +
 .../beam/sdk/nexmark/model/SellerPrice.java     |   89 +
 .../beam/sdk/nexmark/model/package-info.java    |   22 +
 .../apache/beam/sdk/nexmark/package-info.java   |   21 +
 .../sdk/nexmark/queries/AbstractSimulator.java  |  211 ++
 .../beam/sdk/nexmark/queries/NexmarkQuery.java  |  270 ++
 .../sdk/nexmark/queries/NexmarkQueryModel.java  |  117 +
 .../apache/beam/sdk/nexmark/queries/Query0.java |   70 +
 .../beam/sdk/nexmark/queries/Query0Model.java   |   64 +
 .../apache/beam/sdk/nexmark/queries/Query1.java |   67 +
 .../beam/sdk/nexmark/queries/Query10.java       |  367 ++
 .../beam/sdk/nexmark/queries/Query11.java       |   79 +
 .../beam/sdk/nexmark/queries/Query12.java       |   80 +
 .../beam/sdk/nexmark/queries/Query1Model.java   |   76 +
 .../apache/beam/sdk/nexmark/queries/Query2.java |   79 +
 .../beam/sdk/nexmark/queries/Query2Model.java   |   80 +
 .../apache/beam/sdk/nexmark/queries/Query3.java |  301 ++
 .../beam/sdk/nexmark/queries/Query3Model.java   |  124 +
 .../apache/beam/sdk/nexmark/queries/Query4.java |  116 +
 .../beam/sdk/nexmark/queries/Query4Model.java   |  186 +
 .../apache/beam/sdk/nexmark/queries/Query5.java |  138 +
 .../beam/sdk/nexmark/queries/Query5Model.java   |  176 +
 .../apache/beam/sdk/nexmark/queries/Query6.java |  155 +
 .../beam/sdk/nexmark/queries/Query6Model.java   |  133 +
 .../apache/beam/sdk/nexmark/queries/Query7.java |   90 +
 .../beam/sdk/nexmark/queries/Query7Model.java   |  130 +
 .../apache/beam/sdk/nexmark/queries/Query8.java |   98 +
 .../beam/sdk/nexmark/queries/Query8Model.java   |  148 +
 .../apache/beam/sdk/nexmark/queries/Query9.java |   44 +
 .../beam/sdk/nexmark/queries/Query9Model.java   |   44 +
 .../beam/sdk/nexmark/queries/WinningBids.java   |  418 +++
 .../nexmark/queries/WinningBidsSimulator.java   |  206 ++
 .../beam/sdk/nexmark/queries/package-info.java  |   22 +
 .../sdk/nexmark/sources/BoundedEventSource.java |  190 +
 .../beam/sdk/nexmark/sources/Generator.java     |  609 +++
 .../sdk/nexmark/sources/GeneratorConfig.java    |  298 ++
 .../nexmark/sources/UnboundedEventSource.java   |  329 ++
 .../beam/sdk/nexmark/sources/package-info.java  |   22 +
 .../nexmark/src/main/resources/log4j.properties |   55 +
 .../beam/sdk/nexmark/queries/QueryTest.java     |  185 +
 .../nexmark/sources/BoundedEventSourceTest.java |   70 +
 .../beam/sdk/nexmark/sources/GeneratorTest.java |  110 +
 .../sources/UnboundedEventSourceTest.java       |  105 +
 sdks/java/pom.xml                               |    5 +-
 sdks/pom.xml                                    |    4 +-
 sdks/python/apache_beam/__init__.py             |   47 +-
 sdks/python/apache_beam/coders/coder_impl.py    |   22 +-
 sdks/python/apache_beam/coders/coders.py        |  118 +-
 sdks/python/apache_beam/coders/coders_test.py   |    2 +-
 .../apache_beam/coders/coders_test_common.py    |   17 +-
 .../apache_beam/coders/observable_test.py       |    1 -
 .../coders/proto2_coder_test_messages_pb2.py    |    6 +-
 .../apache_beam/coders/standard_coders_test.py  |   11 +-
 sdks/python/apache_beam/coders/stream_test.py   |    1 -
 sdks/python/apache_beam/coders/typecoders.py    |    1 -
 .../examples/complete/autocomplete.py           |    6 +-
 .../examples/complete/autocomplete_test.py      |    2 +-
 .../examples/complete/estimate_pi.py            |    5 +-
 .../examples/complete/estimate_pi_test.py       |    2 +-
 .../examples/complete/game/game_stats.py        |  393 ++
 .../examples/complete/game/game_stats_test.py   |   81 +
 .../examples/complete/game/hourly_team_score.py |  247 +-
 .../examples/complete/game/leader_board.py      |  349 ++
 .../examples/complete/game/leader_board_test.py |   69 +
 .../examples/complete/game/user_score.py        |  186 +-
 .../complete/juliaset/juliaset/juliaset.py      |    8 +-
 .../complete/juliaset/juliaset/juliaset_test.py |    1 -
 .../examples/complete/juliaset/juliaset_main.py |    2 -
 .../examples/complete/juliaset/setup.py         |    7 +-
 .../apache_beam/examples/complete/tfidf.py      |   17 +-
 .../apache_beam/examples/complete/tfidf_test.py |    7 +-
 .../examples/complete/top_wikipedia_sessions.py |    5 +-
 .../complete/top_wikipedia_sessions_test.py     |    1 -
 .../examples/cookbook/bigquery_side_input.py    |    5 +-
 .../examples/cookbook/bigquery_tornadoes.py     |    2 +-
 .../cookbook/bigquery_tornadoes_it_test.py      |    2 +-
 .../examples/cookbook/custom_ptransform.py      |    2 +-
 .../examples/cookbook/datastore_wordcount.py    |   39 +-
 .../examples/cookbook/group_with_coder.py       |    6 +-
 .../examples/cookbook/group_with_coder_test.py  |    1 -
 .../examples/cookbook/mergecontacts.py          |   36 +-
 .../examples/cookbook/multiple_output_pardo.py  |   14 +-
 .../apache_beam/examples/snippets/snippets.py   |   82 +-
 .../examples/snippets/snippets_test.py          |   70 +-
 .../apache_beam/examples/streaming_wordcount.py |    9 +-
 .../apache_beam/examples/windowed_wordcount.py  |    7 +-
 sdks/python/apache_beam/examples/wordcount.py   |   12 +-
 .../apache_beam/examples/wordcount_debugging.py |   12 +-
 .../apache_beam/examples/wordcount_fnapi.py     |  151 +
 .../apache_beam/examples/wordcount_it_test.py   |   16 +
 .../apache_beam/examples/wordcount_minimal.py   |    6 +-
 sdks/python/apache_beam/internal/gcp/auth.py    |    1 -
 .../apache_beam/internal/gcp/json_value.py      |   46 +-
 .../apache_beam/internal/gcp/json_value_test.py |    3 +-
 sdks/python/apache_beam/internal/pickler.py     |    2 +-
 sdks/python/apache_beam/internal/util.py        |    6 +-
 sdks/python/apache_beam/io/avroio.py            |  164 +-
 sdks/python/apache_beam/io/avroio_test.py       |   54 +-
 .../python/apache_beam/io/concat_source_test.py |    1 -
 sdks/python/apache_beam/io/filebasedsink.py     |   22 +-
 .../python/apache_beam/io/filebasedsink_test.py |    3 +-
 sdks/python/apache_beam/io/filebasedsource.py   |  234 +-
 .../apache_beam/io/filebasedsource_test.py      |    8 +-
 sdks/python/apache_beam/io/filesystem.py        |   31 +-
 sdks/python/apache_beam/io/filesystem_test.py   |    5 +-
 sdks/python/apache_beam/io/filesystems.py       |    1 -
 sdks/python/apache_beam/io/filesystems_test.py  |    4 +-
 sdks/python/apache_beam/io/gcp/bigquery.py      |  264 +-
 sdks/python/apache_beam/io/gcp/bigquery_test.py |    4 +-
 .../io/gcp/datastore/v1/adaptive_throttler.py   |   94 +
 .../gcp/datastore/v1/adaptive_throttler_test.py |   95 +
 .../io/gcp/datastore/v1/datastoreio.py          |   55 +-
 .../io/gcp/datastore/v1/datastoreio_test.py     |   11 +-
 .../apache_beam/io/gcp/datastore/v1/helper.py   |   31 +-
 .../io/gcp/datastore/v1/helper_test.py          |    7 +-
 .../apache_beam/io/gcp/gcsfilesystem_test.py    |    1 +
 sdks/python/apache_beam/io/gcp/gcsio.py         |   17 +-
 .../clients/bigquery/bigquery_v2_messages.py    |    3 -
 .../clients/storage/storage_v1_messages.py      |    3 -
 sdks/python/apache_beam/io/gcp/pubsub.py        |    5 +-
 sdks/python/apache_beam/io/gcp/pubsub_test.py   |    5 +-
 .../io/gcp/tests/bigquery_matcher_test.py       |    3 +-
 sdks/python/apache_beam/io/gcp/tests/utils.py   |    8 +-
 .../apache_beam/io/gcp/tests/utils_test.py      |   70 +-
 sdks/python/apache_beam/io/iobase.py            |   83 +-
 .../apache_beam/io/localfilesystem_test.py      |    4 +-
 sdks/python/apache_beam/io/range_trackers.py    |   54 +-
 .../apache_beam/io/range_trackers_test.py       |   37 +
 sdks/python/apache_beam/io/source_test_utils.py |   92 +-
 .../apache_beam/io/source_test_utils_test.py    |    2 +-
 sdks/python/apache_beam/io/sources_test.py      |    1 -
 sdks/python/apache_beam/io/textio.py            |  178 +-
 sdks/python/apache_beam/io/textio_test.py       |  109 +-
 sdks/python/apache_beam/io/tfrecordio.py        |    5 +-
 sdks/python/apache_beam/io/tfrecordio_test.py   |    8 +-
 sdks/python/apache_beam/metrics/cells_test.py   |    2 +-
 sdks/python/apache_beam/metrics/execution.py    |    5 +-
 .../apache_beam/metrics/execution_test.py       |    8 +-
 sdks/python/apache_beam/metrics/metric.py       |    3 +-
 sdks/python/apache_beam/metrics/metric_test.py  |   18 +-
 sdks/python/apache_beam/metrics/metricbase.py   |    4 +
 .../apache_beam/options/pipeline_options.py     |   32 +-
 .../options/pipeline_options_test.py            |    7 +-
 .../options/pipeline_options_validator_test.py  |    3 +-
 .../apache_beam/options/value_provider.py       |    1 -
 sdks/python/apache_beam/pipeline.py             |  136 +-
 sdks/python/apache_beam/pipeline_test.py        |   92 +-
 sdks/python/apache_beam/pvalue.py               |  111 +-
 sdks/python/apache_beam/pvalue_test.py          |    8 +
 sdks/python/apache_beam/runners/common.py       |    6 +-
 sdks/python/apache_beam/runners/common_test.py  |    2 +-
 .../runners/dataflow/dataflow_metrics.py        |    2 +-
 .../runners/dataflow/dataflow_runner.py         |  122 +-
 .../runners/dataflow/dataflow_runner_test.py    |   89 +-
 .../runners/dataflow/internal/apiclient.py      |  106 +-
 .../runners/dataflow/internal/apiclient_test.py |  112 +-
 .../clients/dataflow/dataflow_v1b3_client.py    |  259 +-
 .../clients/dataflow/dataflow_v1b3_messages.py  |   92 +-
 .../clients/dataflow/message_matchers.py        |    1 -
 .../clients/dataflow/message_matchers_test.py   |    3 +-
 .../runners/dataflow/internal/dependency.py     |   58 +-
 .../dataflow/internal/dependency_test.py        |   10 +-
 .../runners/dataflow/internal/names.py          |    2 +
 .../runners/dataflow/native_io/iobase_test.py   |   22 +-
 .../dataflow/native_io/streaming_create.py      |    2 +-
 .../runners/dataflow/template_runner_test.py    |    5 +-
 .../runners/dataflow/test_dataflow_runner.py    |    5 +-
 .../consumer_tracking_pipeline_visitor_test.py  |    2 +-
 .../runners/direct/direct_metrics.py            |    2 +-
 .../runners/direct/direct_metrics_test.py       |    8 +-
 .../apache_beam/runners/direct/direct_runner.py |   19 +-
 .../runners/direct/direct_runner_test.py        |   41 +
 .../runners/direct/evaluation_context.py        |   31 +-
 .../apache_beam/runners/direct/executor.py      |  127 +-
 .../runners/direct/helper_transforms.py         |    2 +-
 .../runners/direct/transform_evaluator.py       |   67 +-
 sdks/python/apache_beam/runners/direct/util.py  |    7 +-
 .../runners/experimental/__init__.py            |   16 +
 .../experimental/python_rpc_direct/__init__.py  |   22 +
 .../python_rpc_direct_runner.py                 |  110 +
 .../experimental/python_rpc_direct/server.py    |  111 +
 sdks/python/apache_beam/runners/job/__init__.py |   16 +
 sdks/python/apache_beam/runners/job/manager.py  |   52 +
 sdks/python/apache_beam/runners/job/utils.py    |   32 +
 .../apache_beam/runners/pipeline_context.py     |    2 +-
 .../runners/portability/fn_api_runner.py        |  393 +-
 .../runners/portability/fn_api_runner_test.py   |  115 +-
 .../portability/maptask_executor_runner.py      |   19 +-
 .../portability/maptask_executor_runner_test.py |   12 +-
 .../portability/universal_local_runner.py       |  409 ++
 .../portability/universal_local_runner_main.py  |   44 +
 .../portability/universal_local_runner_test.py  |   85 +
 sdks/python/apache_beam/runners/runner.py       |   61 +-
 sdks/python/apache_beam/runners/runner_test.py  |    2 +-
 .../runners/worker/bundle_processor.py          |  235 +-
 .../apache_beam/runners/worker/data_plane.py    |   35 +-
 .../runners/worker/data_plane_test.py           |    7 +-
 .../apache_beam/runners/worker/log_handler.py   |   17 +-
 .../runners/worker/log_handler_test.py          |   20 +-
 .../apache_beam/runners/worker/opcounters.py    |   10 +-
 .../runners/worker/opcounters_test.py           |    1 -
 .../apache_beam/runners/worker/operations.pxd   |    4 +-
 .../apache_beam/runners/worker/operations.py    |   84 +-
 .../apache_beam/runners/worker/sdk_worker.py    |  206 +-
 .../runners/worker/sdk_worker_main.py           |   30 +-
 .../runners/worker/sdk_worker_test.py           |   11 +-
 .../apache_beam/runners/worker/statesampler.pyx |   68 +-
 .../runners/worker/statesampler_fake.py         |   17 +-
 .../runners/worker/statesampler_test.py         |    5 +-
 .../apache_beam/testing/pipeline_verifiers.py   |    1 -
 .../testing/pipeline_verifiers_test.py          |    5 +-
 .../python/apache_beam/testing/test_pipeline.py |   54 +-
 .../apache_beam/testing/test_pipeline_test.py   |    4 +-
 sdks/python/apache_beam/testing/test_stream.py  |    1 -
 .../apache_beam/testing/test_stream_test.py     |    3 +-
 sdks/python/apache_beam/testing/test_utils.py   |   23 +-
 .../apache_beam/testing/test_utils_test.py      |   62 +
 sdks/python/apache_beam/testing/util.py         |    5 +-
 sdks/python/apache_beam/testing/util_test.py    |    4 +-
 sdks/python/apache_beam/transforms/combiners.py |    9 +-
 .../apache_beam/transforms/combiners_test.py    |    5 +-
 sdks/python/apache_beam/transforms/core.py      |  397 +-
 .../apache_beam/transforms/create_test.py       |    6 +-
 sdks/python/apache_beam/transforms/display.py   |   91 +-
 .../apache_beam/transforms/display_test.py      |    6 +-
 .../python/apache_beam/transforms/ptransform.py |  147 +-
 .../apache_beam/transforms/ptransform_test.py   |   78 +-
 .../apache_beam/transforms/sideinputs_test.py   |    5 +-
 sdks/python/apache_beam/transforms/timeutil.py  |    1 -
 sdks/python/apache_beam/transforms/trigger.py   |   32 +-
 .../apache_beam/transforms/trigger_test.py      |   19 +-
 sdks/python/apache_beam/transforms/util.py      |  273 +-
 sdks/python/apache_beam/transforms/util_test.py |  108 +
 sdks/python/apache_beam/transforms/window.py    |    9 +-
 .../apache_beam/transforms/window_test.py       |   17 +-
 .../transforms/write_ptransform_test.py         |    4 +-
 sdks/python/apache_beam/typehints/decorators.py |  129 +-
 .../typehints/native_type_compatibility.py      |  166 +
 .../typehints/native_type_compatibility_test.py |   92 +
 sdks/python/apache_beam/typehints/opcodes.py    |   19 +-
 .../apache_beam/typehints/trivial_inference.py  |   80 +-
 .../typehints/trivial_inference_test.py         |    9 +-
 sdks/python/apache_beam/typehints/typecheck.py  |    6 +-
 .../typehints/typed_pipeline_test.py            |   33 +-
 sdks/python/apache_beam/typehints/typehints.py  |   79 +-
 .../apache_beam/typehints/typehints_test.py     |    3 +-
 .../apache_beam/utils/annotations_test.py       |    1 +
 sdks/python/apache_beam/utils/counters.py       |  107 +-
 sdks/python/apache_beam/utils/counters_test.py  |   78 +
 sdks/python/apache_beam/utils/processes_test.py |    1 -
 sdks/python/apache_beam/utils/proto_utils.py    |   11 +
 sdks/python/apache_beam/utils/retry.py          |    9 +-
 sdks/python/apache_beam/utils/retry_test.py     |    5 +-
 sdks/python/apache_beam/utils/urns.py           |   22 +-
 sdks/python/apache_beam/version.py              |    2 +-
 sdks/python/container/Dockerfile                |   27 +
 sdks/python/container/boot.go                   |  123 +
 sdks/python/container/pom.xml                   |  154 +
 sdks/python/gen_protos.py                       |   30 +-
 sdks/python/generate_pydoc.sh                   |  134 +-
 sdks/python/pom.xml                             |    9 +-
 sdks/python/run_postcommit.sh                   |   20 -
 sdks/python/run_pylint.sh                       |   45 +-
 sdks/python/run_validatesrunner.sh              |   71 +
 sdks/python/setup.py                            |   10 +-
 sdks/python/tox.ini                             |   21 +-
 1246 files changed, 92588 insertions(+), 17745 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/7181d0c3/pom.xml
----------------------------------------------------------------------
diff --cc pom.xml
index ae86a9c,0b7b323..5f997a3
--- a/pom.xml
+++ b/pom.xml
@@@ -183,9 -196,10 +196,10 @@@
      <!-- sdks/java/build-tools has project-wide configuration. To make these available
        in all modules, link it directly to the parent pom.xml. -->
      <module>sdks/java/build-tools</module>
+     <module>model</module>
      <module>sdks</module>
      <module>runners</module>
 -    <module>examples</module>
 +    <!--<module>examples</module>-->
      <!-- sdks/java/javadoc builds project-wide Javadoc. It has to run last. -->
      <module>sdks/java/javadoc</module>
    </modules>

http://git-wip-us.apache.org/repos/asf/beam/blob/7181d0c3/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryTimerInternals.java
----------------------------------------------------------------------
diff --cc runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryTimerInternals.java
index c1d42d6,c7b4ac6..719f75f
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryTimerInternals.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryTimerInternals.java
@@@ -23,9 -23,9 +23,10 @@@ import static com.google.common.base.Pr
  
  import com.google.common.base.MoreObjects;
  import com.google.common.collect.HashBasedTable;
 +import com.google.common.collect.ImmutableList;
  import com.google.common.collect.Table;
  import java.util.NavigableSet;
+ import java.util.NoSuchElementException;
  import java.util.TreeSet;
  import javax.annotation.Nullable;
  import org.apache.beam.sdk.state.TimeDomain;

http://git-wip-us.apache.org/repos/asf/beam/blob/7181d0c3/runners/pom.xml
----------------------------------------------------------------------
diff --cc runners/pom.xml
index 39a9811,47f3c0e..aed0c10
--- a/runners/pom.xml
+++ b/runners/pom.xml
@@@ -35,7 -35,9 +35,10 @@@
    <modules>
      <module>core-construction-java</module>
      <module>core-java</module>
+     <module>java-fn-execution</module>
+     <module>local-artifact-service-java</module>
+     <module>reference</module>
 +    <module>map-reduce</module>
      <module>direct-java</module>
      <module>flink</module>
      <module>google-cloud-dataflow-java</module>

http://git-wip-us.apache.org/repos/asf/beam/blob/7181d0c3/sdks/java/pom.xml
----------------------------------------------------------------------
diff --cc sdks/java/pom.xml
index 75b2043,c6ab234..77a7de4
--- a/sdks/java/pom.xml
+++ b/sdks/java/pom.xml
@@@ -37,10 -37,10 +37,11 @@@
           and other project configuration to be used in all modules.
      <module>build-tools</module> -->
      <module>core</module>
 -    <module>io</module>
 +    <!--module>io</module-->
 +    <module>io/hadoop-file-system</module>
      <module>maven-archetypes</module>
      <module>extensions</module>
+     <module>fn-execution</module>
      <!-- javadoc runs directly from the root parent as the last module
           in the build to be able to capture runner-specific javadoc.
      <module>javadoc</module> -->

http://git-wip-us.apache.org/repos/asf/beam/blob/7181d0c3/sdks/pom.xml
----------------------------------------------------------------------
diff --cc sdks/pom.xml
index c06f764,7c85489..f0afdad
--- a/sdks/pom.xml
+++ b/sdks/pom.xml
@@@ -33,9 -33,9 +33,9 @@@
    <name>Apache Beam :: SDKs</name>
  
    <modules>
-     <module>common</module>
+     <module>go</module>
      <module>java</module>
 -    <module>python</module>
 +    <!--<module>python</module>-->
    </modules>
  
    <profiles>


[24/50] [abbrv] beam git commit: Add sdks/java/fn-execution

Posted by ke...@apache.org.
Add sdks/java/fn-execution

This module contains java code shared by both the SDK harness and
Runners. It does not depend on the Java SDK. Runner harness
libraries will depend on this module, and are not permitted to interact
with SDK constructs.


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

Branch: refs/heads/mr-runner
Commit: fdd5971d95df3219954023c6806f09dce87a1f8c
Parents: fbcb0ba
Author: Thomas Groh <tg...@google.com>
Authored: Wed Oct 11 18:46:27 2017 -0700
Committer: Thomas Groh <tg...@google.com>
Committed: Thu Nov 2 18:25:12 2017 -0700

----------------------------------------------------------------------
 ...job_beam_PostCommit_Java_MavenInstall.groovy |   2 +-
 .../job_beam_PreCommit_Java_MavenInstall.groovy |   4 +-
 pom.xml                                         |  61 ++++++-
 sdks/java/fn-execution/pom.xml                  |  96 +++++++++++
 .../harness/channel/ManagedChannelFactory.java  |  82 ++++++++++
 .../harness/channel/SocketAddressFactory.java   |  64 ++++++++
 .../beam/harness/channel/package-info.java      |  22 +++
 .../channel/ManagedChannelFactoryTest.java      |  71 ++++++++
 .../channel/SocketAddressFactoryTest.java       |  56 +++++++
 .../apache/beam/harness/test/TestExecutors.java |  85 ++++++++++
 .../beam/harness/test/TestExecutorsTest.java    | 160 ++++++++++++++++++
 .../apache/beam/harness/test/TestStreams.java   | 162 +++++++++++++++++++
 .../beam/harness/test/TestStreamsTest.java      |  84 ++++++++++
 sdks/java/harness/pom.xml                       |  26 +++
 .../org/apache/beam/fn/harness/FnHarness.java   |  19 ++-
 .../harness/channel/ManagedChannelFactory.java  |  86 ----------
 .../harness/channel/SocketAddressFactory.java   |  64 --------
 .../beam/fn/harness/channel/package-info.java   |  22 ---
 .../fn/harness/BeamFnDataReadRunnerTest.java    |   4 +-
 .../apache/beam/fn/harness/FnHarnessTest.java   |   2 +-
 .../channel/ManagedChannelFactoryTest.java      |  73 ---------
 .../channel/SocketAddressFactoryTest.java       |  56 -------
 .../control/BeamFnControlClientTest.java        |   2 +-
 .../fn/harness/control/RegisterHandlerTest.java |   4 +-
 ...BeamFnDataBufferingOutboundObserverTest.java |   2 +-
 .../harness/data/BeamFnDataGrpcClientTest.java  |   2 +-
 .../data/BeamFnDataGrpcMultiplexerTest.java     |   2 +-
 .../logging/BeamFnLoggingClientTest.java        |   2 +-
 .../state/BeamFnStateGrpcClientCacheTest.java   |   2 +-
 .../stream/BufferingStreamObserverTest.java     |   6 +-
 .../stream/DirectStreamObserverTest.java        |   6 +-
 .../beam/fn/harness/test/TestExecutors.java     |  85 ----------
 .../beam/fn/harness/test/TestExecutorsTest.java | 160 ------------------
 .../beam/fn/harness/test/TestStreams.java       | 162 -------------------
 .../beam/fn/harness/test/TestStreamsTest.java   |  84 ----------
 sdks/java/pom.xml                               |   1 +
 36 files changed, 1003 insertions(+), 818 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/fdd5971d/.test-infra/jenkins/job_beam_PostCommit_Java_MavenInstall.groovy
----------------------------------------------------------------------
diff --git a/.test-infra/jenkins/job_beam_PostCommit_Java_MavenInstall.groovy b/.test-infra/jenkins/job_beam_PostCommit_Java_MavenInstall.groovy
index 5d67e6d..0dda772 100644
--- a/.test-infra/jenkins/job_beam_PostCommit_Java_MavenInstall.groovy
+++ b/.test-infra/jenkins/job_beam_PostCommit_Java_MavenInstall.groovy
@@ -47,7 +47,7 @@ mavenJob('beam_PostCommit_Java_MavenInstall') {
   goals([
       'clean',
       'install',
-      '--projects sdks/java/core,runners/direct-java',
+      '--projects sdks/java/core,runners/direct-java,sdks/java/fn-execution',
       ' --also-make',
       '--also-make-dependents',
       '--batch-mode',

http://git-wip-us.apache.org/repos/asf/beam/blob/fdd5971d/.test-infra/jenkins/job_beam_PreCommit_Java_MavenInstall.groovy
----------------------------------------------------------------------
diff --git a/.test-infra/jenkins/job_beam_PreCommit_Java_MavenInstall.groovy b/.test-infra/jenkins/job_beam_PreCommit_Java_MavenInstall.groovy
index 52423e0..0775e2f 100644
--- a/.test-infra/jenkins/job_beam_PreCommit_Java_MavenInstall.groovy
+++ b/.test-infra/jenkins/job_beam_PreCommit_Java_MavenInstall.groovy
@@ -38,14 +38,14 @@ mavenJob('beam_PreCommit_Java_MavenInstall') {
   common_job_properties.setMavenConfig(delegate)
 
   // Sets that this is a PreCommit job.
-  common_job_properties.setPreCommit(delegate, 'mvn clean install -pl sdks/java/core,runners/direct-java -am -amd', 'Run Java PreCommit')
+  common_job_properties.setPreCommit(delegate, 'mvn clean install -pl sdks/java/core,runners/direct-java,sdks/java/fn-execution -am -amd', 'Run Java PreCommit')
 
   // Maven goals for this job: The Java SDK, its dependencies, and things that depend on it.
   goals([
     '--batch-mode',
     '--errors',
     '--activate-profiles release,jenkins-precommit,direct-runner,dataflow-runner,spark-runner,flink-runner,apex-runner',
-    '--projects sdks/java/core,runners/direct-java',
+    '--projects sdks/java/core,runners/direct-java,sdks/java/fn-execution',
     '--also-make',
     '--also-make-dependents',
     '-D pullRequest=$ghprbPullId',

http://git-wip-us.apache.org/repos/asf/beam/blob/fdd5971d/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index fcd0339..f1eee91 100644
--- a/pom.xml
+++ b/pom.xml
@@ -289,6 +289,52 @@
     </profile>
 
     <profile>
+      <id>java8-enable-like-dependencies</id>
+      <activation>
+        <jdk>[1.8,)</jdk>
+      </activation>
+      <build>
+        <plugins>
+          <!-- Override Beam parent to allow Java8 dependencies -->
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-enforcer-plugin</artifactId>
+            <version>${maven-enforcer-plugin.version}</version>
+            <executions>
+              <execution>
+                <id>enforce</id>
+                <goals>
+                  <goal>enforce</goal>
+                </goals>
+                <configuration>
+                  <rules>
+                    <enforceBytecodeVersion>
+                      <maxJdkVersion>1.8</maxJdkVersion>
+                      <excludes>
+                        <!--
+                          Supplied by the user JDK and compiled with matching
+                          version. Is not shaded, so safe to ignore.
+                        -->
+                        <exclude>jdk.tools:jdk.tools</exclude>
+                      </excludes>
+                    </enforceBytecodeVersion>
+                    <requireJavaVersion>
+                      <version>[1.7,)</version>
+                    </requireJavaVersion>
+                    <requireMavenVersion>
+                      <!-- Keep aligned with preqrequisite section below. -->
+                      <version>[3.2,)</version>
+                    </requireMavenVersion>
+                  </rules>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+
+    <profile>
       <id>doclint-java8-disable</id>
       <activation>
         <jdk>[1.8,)</jdk>
@@ -462,6 +508,19 @@
 
       <dependency>
         <groupId>org.apache.beam</groupId>
+        <artifactId>beam-sdks-java-fn-execution</artifactId>
+        <version>${project.version}</version>
+      </dependency>
+
+      <dependency>
+        <groupId>org.apache.beam</groupId>
+        <artifactId>beam-sdks-java-fn-execution</artifactId>
+        <version>${project.version}</version>
+        <type>test-jar</type>
+      </dependency>
+
+      <dependency>
+        <groupId>org.apache.beam</groupId>
         <artifactId>beam-sdks-java-harness</artifactId>
         <version>${project.version}</version>
       </dependency>
@@ -601,7 +660,7 @@
       <dependency>
         <groupId>org.apache.beam</groupId>
         <artifactId>beam-sdks-java-io-hadoop-input-format</artifactId>
-	    <version>${project.version}</version>
+        <version>${project.version}</version>
       </dependency>
 
       <dependency>

http://git-wip-us.apache.org/repos/asf/beam/blob/fdd5971d/sdks/java/fn-execution/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/fn-execution/pom.xml b/sdks/java/fn-execution/pom.xml
new file mode 100644
index 0000000..9929c29
--- /dev/null
+++ b/sdks/java/fn-execution/pom.xml
@@ -0,0 +1,96 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+    Licensed to the Apache Software Foundation (ASF) under one or more
+    contributor license agreements.  See the NOTICE file distributed with
+    this work for additional information regarding copyright ownership.
+    The ASF licenses this file to You under the Apache License, Version 2.0
+    (the "License"); you may not use this file except in compliance with
+    the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing, software
+    distributed under the License is distributed on an "AS IS" BASIS,
+    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+    See the License for the specific language governing permissions and
+    limitations under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+
+  <modelVersion>4.0.0</modelVersion>
+
+  <parent>
+    <groupId>org.apache.beam</groupId>
+    <artifactId>beam-sdks-java-parent</artifactId>
+    <version>2.3.0-SNAPSHOT</version>
+    <relativePath>../pom.xml</relativePath>
+  </parent>
+
+  <artifactId>beam-sdks-java-fn-execution</artifactId>
+  <name>Apache Beam :: SDKs :: Java :: Harness Core</name>
+  <description>Contains code shared across the Beam Java SDK Harness and the Java Runner Harness
+    libraries.
+  </description>
+
+  <packaging>jar</packaging>
+
+  <build>
+    <plugins>
+      <plugin>
+        <!--  Override Beam parent to allow Java8 -->
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-compiler-plugin</artifactId>
+        <configuration>
+          <source>1.8</source>
+          <target>1.8</target>
+        </configuration>
+      </plugin>
+    </plugins>
+  </build>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>beam-model-pipeline</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-core</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-stub</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-netty</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>io.netty</groupId>
+      <artifactId>netty-transport-native-epoll</artifactId>
+      <classifier>linux-x86_64</classifier>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+
+    <!-- Test dependencies -->
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.hamcrest</groupId>
+      <artifactId>hamcrest-all</artifactId>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/beam/blob/fdd5971d/sdks/java/fn-execution/src/main/java/org/apache/beam/harness/channel/ManagedChannelFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/harness/channel/ManagedChannelFactory.java b/sdks/java/fn-execution/src/main/java/org/apache/beam/harness/channel/ManagedChannelFactory.java
new file mode 100644
index 0000000..187cfdb
--- /dev/null
+++ b/sdks/java/fn-execution/src/main/java/org/apache/beam/harness/channel/ManagedChannelFactory.java
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.harness.channel;
+
+import io.grpc.ManagedChannel;
+import io.grpc.ManagedChannelBuilder;
+import io.grpc.netty.NettyChannelBuilder;
+import io.netty.channel.epoll.EpollDomainSocketChannel;
+import io.netty.channel.epoll.EpollEventLoopGroup;
+import io.netty.channel.epoll.EpollSocketChannel;
+import io.netty.channel.unix.DomainSocketAddress;
+import java.net.SocketAddress;
+import org.apache.beam.model.pipeline.v1.Endpoints.ApiServiceDescriptor;
+
+/**
+ * A Factory which creates an underlying {@link ManagedChannel} implementation.
+ */
+public abstract class ManagedChannelFactory {
+  public static ManagedChannelFactory createDefault() {
+    return new Default();
+  }
+
+  public static ManagedChannelFactory createEpoll() {
+    io.netty.channel.epoll.Epoll.ensureAvailability();
+    return new Epoll();
+  }
+
+  public abstract ManagedChannel forDescriptor(ApiServiceDescriptor apiServiceDescriptor);
+
+  /**
+   * Creates a {@link ManagedChannel} backed by an {@link EpollDomainSocketChannel} if the address
+   * is a {@link DomainSocketAddress}. Otherwise creates a {@link ManagedChannel} backed by an
+   * {@link EpollSocketChannel}.
+   */
+  private static class Epoll extends ManagedChannelFactory {
+    @Override
+    public ManagedChannel forDescriptor(ApiServiceDescriptor apiServiceDescriptor) {
+      SocketAddress address = SocketAddressFactory.createFrom(apiServiceDescriptor.getUrl());
+      return NettyChannelBuilder.forAddress(address)
+          .channelType(address instanceof DomainSocketAddress
+              ? EpollDomainSocketChannel.class : EpollSocketChannel.class)
+          .eventLoopGroup(new EpollEventLoopGroup())
+          .usePlaintext(true)
+          // Set the message size to max value here. The actual size is governed by the
+          // buffer size in the layers above.
+          .maxInboundMessageSize(Integer.MAX_VALUE)
+          .build();
+    }
+  }
+
+  /**
+   * Creates a {@link ManagedChannel} relying on the {@link ManagedChannelBuilder} to create
+   * instances.
+   */
+  private static class Default extends ManagedChannelFactory {
+    @Override
+    public ManagedChannel forDescriptor(ApiServiceDescriptor apiServiceDescriptor) {
+      return ManagedChannelBuilder.forTarget(apiServiceDescriptor.getUrl())
+          .usePlaintext(true)
+          // Set the message size to max value here. The actual size is governed by the
+          // buffer size in the layers above.
+          .maxInboundMessageSize(Integer.MAX_VALUE)
+          .build();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/fdd5971d/sdks/java/fn-execution/src/main/java/org/apache/beam/harness/channel/SocketAddressFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/harness/channel/SocketAddressFactory.java b/sdks/java/fn-execution/src/main/java/org/apache/beam/harness/channel/SocketAddressFactory.java
new file mode 100644
index 0000000..5253291
--- /dev/null
+++ b/sdks/java/fn-execution/src/main/java/org/apache/beam/harness/channel/SocketAddressFactory.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.harness.channel;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import com.google.common.net.HostAndPort;
+import io.netty.channel.unix.DomainSocketAddress;
+import java.io.File;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+
+/** Creates a {@link SocketAddress} based upon a supplied string. */
+public class SocketAddressFactory {
+  private static final String UNIX_DOMAIN_SOCKET_PREFIX = "unix://";
+
+  /**
+   * Parse a {@link SocketAddress} from the given string.
+   */
+  public static SocketAddress createFrom(String value) {
+    if (value.startsWith(UNIX_DOMAIN_SOCKET_PREFIX)) {
+      // Unix Domain Socket address.
+      // Create the underlying file for the Unix Domain Socket.
+      String filePath = value.substring(UNIX_DOMAIN_SOCKET_PREFIX.length());
+      File file = new File(filePath);
+      if (!file.isAbsolute()) {
+        throw new IllegalArgumentException("File path must be absolute: " + filePath);
+      }
+      try {
+        if (file.createNewFile()) {
+          // If this application created the file, delete it when the application exits.
+          file.deleteOnExit();
+        }
+      } catch (IOException ex) {
+        throw new RuntimeException(ex);
+      }
+      // Create the SocketAddress referencing the file.
+      return new DomainSocketAddress(file);
+    } else {
+      // Standard TCP/IP address.
+      HostAndPort hostAndPort = HostAndPort.fromString(value);
+      checkArgument(hostAndPort.hasPort(),
+          "Address must be a unix:// path or be in the form host:port. Got: %s", value);
+      return new InetSocketAddress(hostAndPort.getHostText(), hostAndPort.getPort());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/fdd5971d/sdks/java/fn-execution/src/main/java/org/apache/beam/harness/channel/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/harness/channel/package-info.java b/sdks/java/fn-execution/src/main/java/org/apache/beam/harness/channel/package-info.java
new file mode 100644
index 0000000..2a33445
--- /dev/null
+++ b/sdks/java/fn-execution/src/main/java/org/apache/beam/harness/channel/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * gRPC channel management.
+ */
+package org.apache.beam.harness.channel;

http://git-wip-us.apache.org/repos/asf/beam/blob/fdd5971d/sdks/java/fn-execution/src/test/java/org/apache/beam/harness/channel/ManagedChannelFactoryTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/harness/channel/ManagedChannelFactoryTest.java b/sdks/java/fn-execution/src/test/java/org/apache/beam/harness/channel/ManagedChannelFactoryTest.java
new file mode 100644
index 0000000..f73ed80
--- /dev/null
+++ b/sdks/java/fn-execution/src/test/java/org/apache/beam/harness/channel/ManagedChannelFactoryTest.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.harness.channel;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assume.assumeTrue;
+
+import io.grpc.ManagedChannel;
+import org.apache.beam.model.pipeline.v1.Endpoints;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+
+/** Tests for {@link ManagedChannelFactory}. */
+@RunWith(JUnit4.class)
+public class ManagedChannelFactoryTest {
+  @Rule public TemporaryFolder tmpFolder = new TemporaryFolder();
+
+  @Test
+  public void testDefaultChannel() {
+    Endpoints.ApiServiceDescriptor apiServiceDescriptor =
+        Endpoints.ApiServiceDescriptor.newBuilder().setUrl("localhost:123").build();
+    ManagedChannel channel =
+        ManagedChannelFactory.createDefault().forDescriptor(apiServiceDescriptor);
+    assertEquals("localhost:123", channel.authority());
+    channel.shutdownNow();
+  }
+
+  @Test
+  public void testEpollHostPortChannel() {
+    assumeTrue(io.netty.channel.epoll.Epoll.isAvailable());
+    Endpoints.ApiServiceDescriptor apiServiceDescriptor =
+        Endpoints.ApiServiceDescriptor.newBuilder().setUrl("localhost:123").build();
+    ManagedChannel channel =
+        ManagedChannelFactory.createEpoll().forDescriptor(apiServiceDescriptor);
+    assertEquals("localhost:123", channel.authority());
+    channel.shutdownNow();
+  }
+
+  @Test
+  public void testEpollDomainSocketChannel() throws Exception {
+    assumeTrue(io.netty.channel.epoll.Epoll.isAvailable());
+    Endpoints.ApiServiceDescriptor apiServiceDescriptor =
+        Endpoints.ApiServiceDescriptor.newBuilder()
+            .setUrl("unix://" + tmpFolder.newFile().getAbsolutePath())
+            .build();
+    ManagedChannel channel =
+        ManagedChannelFactory.createEpoll().forDescriptor(apiServiceDescriptor);
+    assertEquals(apiServiceDescriptor.getUrl().substring("unix://".length()), channel.authority());
+    channel.shutdownNow();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/fdd5971d/sdks/java/fn-execution/src/test/java/org/apache/beam/harness/channel/SocketAddressFactoryTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/harness/channel/SocketAddressFactoryTest.java b/sdks/java/fn-execution/src/test/java/org/apache/beam/harness/channel/SocketAddressFactoryTest.java
new file mode 100644
index 0000000..95a7d67
--- /dev/null
+++ b/sdks/java/fn-execution/src/test/java/org/apache/beam/harness/channel/SocketAddressFactoryTest.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.harness.channel;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+
+import io.netty.channel.unix.DomainSocketAddress;
+import java.io.File;
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+import org.hamcrest.Matchers;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Tests for {@link SocketAddressFactory}. */
+@RunWith(JUnit4.class)
+public class SocketAddressFactoryTest {
+  @Rule public TemporaryFolder tmpFolder = new TemporaryFolder();
+
+  @Test
+  public void testHostPortSocket() {
+    SocketAddress socketAddress = SocketAddressFactory.createFrom("localhost:123");
+    assertThat(socketAddress, Matchers.instanceOf(InetSocketAddress.class));
+    assertEquals("localhost", ((InetSocketAddress) socketAddress).getHostString());
+    assertEquals(123, ((InetSocketAddress) socketAddress).getPort());
+  }
+
+  @Test
+  public void testDomainSocket() throws Exception {
+    File tmpFile = tmpFolder.newFile();
+    SocketAddress socketAddress = SocketAddressFactory.createFrom(
+        "unix://" + tmpFile.getAbsolutePath());
+    assertThat(socketAddress, Matchers.instanceOf(DomainSocketAddress.class));
+    assertEquals(tmpFile.getAbsolutePath(), ((DomainSocketAddress) socketAddress).path());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/fdd5971d/sdks/java/fn-execution/src/test/java/org/apache/beam/harness/test/TestExecutors.java
----------------------------------------------------------------------
diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/harness/test/TestExecutors.java b/sdks/java/fn-execution/src/test/java/org/apache/beam/harness/test/TestExecutors.java
new file mode 100644
index 0000000..d818a61
--- /dev/null
+++ b/sdks/java/fn-execution/src/test/java/org/apache/beam/harness/test/TestExecutors.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.harness.test;
+
+import com.google.common.util.concurrent.ForwardingExecutorService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+import org.junit.rules.TestRule;
+import org.junit.runner.Description;
+import org.junit.runners.model.Statement;
+
+/**
+ * A {@link TestRule} that validates that all submitted tasks finished and were completed. This
+ * allows for testing that tasks have exercised the appropriate shutdown logic.
+ */
+public class TestExecutors {
+  public static TestExecutorService from(Supplier<ExecutorService> executorServiceSuppler) {
+    return new FromSupplier(executorServiceSuppler);
+  }
+
+  /** A union of the {@link ExecutorService} and {@link TestRule} interfaces. */
+  public interface TestExecutorService extends ExecutorService, TestRule {}
+
+  private static class FromSupplier extends ForwardingExecutorService
+      implements TestExecutorService {
+    private final Supplier<ExecutorService> executorServiceSupplier;
+    private ExecutorService delegate;
+
+    private FromSupplier(Supplier<ExecutorService> executorServiceSupplier) {
+      this.executorServiceSupplier = executorServiceSupplier;
+    }
+
+    @Override
+    public Statement apply(Statement statement, Description arg1) {
+      return new Statement() {
+        @Override
+        public void evaluate() throws Throwable {
+          Throwable thrown = null;
+          delegate = executorServiceSupplier.get();
+          try {
+            statement.evaluate();
+          } catch (Throwable t) {
+            thrown = t;
+          }
+          shutdown();
+          if (!awaitTermination(5, TimeUnit.SECONDS)) {
+            shutdownNow();
+            IllegalStateException e =
+                new IllegalStateException("Test executor failed to shutdown cleanly.");
+            if (thrown != null) {
+              thrown.addSuppressed(e);
+            } else {
+              thrown = e;
+            }
+          }
+          if (thrown != null) {
+            throw thrown;
+          }
+        }
+      };
+    }
+
+    @Override
+    protected ExecutorService delegate() {
+      return delegate;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/fdd5971d/sdks/java/fn-execution/src/test/java/org/apache/beam/harness/test/TestExecutorsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/harness/test/TestExecutorsTest.java b/sdks/java/fn-execution/src/test/java/org/apache/beam/harness/test/TestExecutorsTest.java
new file mode 100644
index 0000000..1381b55
--- /dev/null
+++ b/sdks/java/fn-execution/src/test/java/org/apache/beam/harness/test/TestExecutorsTest.java
@@ -0,0 +1,160 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.harness.test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.beam.harness.test.TestExecutors.TestExecutorService;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+import org.junit.runners.model.Statement;
+
+/** Tests for {@link TestExecutors}. */
+@RunWith(JUnit4.class)
+public class TestExecutorsTest {
+  @Test
+  public void testSuccessfulTermination() throws Throwable {
+    ExecutorService service = Executors.newSingleThreadExecutor();
+    final TestExecutorService testService = TestExecutors.from(() -> service);
+    final AtomicBoolean taskRan = new AtomicBoolean();
+    testService
+        .apply(
+            new Statement() {
+              @Override
+              public void evaluate() throws Throwable {
+                testService.submit(() -> taskRan.set(true));
+              }
+            },
+            null)
+        .evaluate();
+    assertTrue(service.isTerminated());
+    assertTrue(taskRan.get());
+  }
+
+  @Test
+  public void testTaskBlocksForeverCausesFailure() throws Throwable {
+    ExecutorService service = Executors.newSingleThreadExecutor();
+    final TestExecutorService testService = TestExecutors.from(() -> service);
+    final AtomicBoolean taskStarted = new AtomicBoolean();
+    final AtomicBoolean taskWasInterrupted = new AtomicBoolean();
+    try {
+      testService
+          .apply(
+              new Statement() {
+                @Override
+                public void evaluate() throws Throwable {
+                  testService.submit(this::taskToRun);
+                }
+
+                private void taskToRun() {
+                  taskStarted.set(true);
+                  try {
+                    while (true) {
+                      Thread.sleep(10000);
+                    }
+                  } catch (InterruptedException e) {
+                    taskWasInterrupted.set(true);
+                    return;
+                  }
+                }
+              },
+              null)
+          .evaluate();
+      fail();
+    } catch (IllegalStateException e) {
+      assertEquals(IllegalStateException.class, e.getClass());
+      assertEquals("Test executor failed to shutdown cleanly.", e.getMessage());
+    }
+    assertTrue(service.isShutdown());
+  }
+
+  @Test
+  public void testStatementFailurePropagatedCleanly() throws Throwable {
+    ExecutorService service = Executors.newSingleThreadExecutor();
+    final TestExecutorService testService = TestExecutors.from(() -> service);
+    final RuntimeException exceptionToThrow = new RuntimeException();
+    try {
+      testService
+          .apply(
+              new Statement() {
+                @Override
+                public void evaluate() throws Throwable {
+                  throw exceptionToThrow;
+                }
+              },
+              null)
+          .evaluate();
+      fail();
+    } catch (RuntimeException thrownException) {
+      assertSame(exceptionToThrow, thrownException);
+    }
+    assertTrue(service.isShutdown());
+  }
+
+  @Test
+  public void testStatementFailurePropagatedWhenExecutorServiceFailingToTerminate()
+      throws Throwable {
+    ExecutorService service = Executors.newSingleThreadExecutor();
+    final TestExecutorService testService = TestExecutors.from(() -> service);
+    final AtomicBoolean taskStarted = new AtomicBoolean();
+    final AtomicBoolean taskWasInterrupted = new AtomicBoolean();
+    final RuntimeException exceptionToThrow = new RuntimeException();
+    try {
+      testService
+          .apply(
+              new Statement() {
+                @Override
+                public void evaluate() throws Throwable {
+                  testService.submit(this::taskToRun);
+                  throw exceptionToThrow;
+                }
+
+                private void taskToRun() {
+                  taskStarted.set(true);
+                  try {
+                    while (true) {
+                      Thread.sleep(10000);
+                    }
+                  } catch (InterruptedException e) {
+                    taskWasInterrupted.set(true);
+                    return;
+                  }
+                }
+              },
+              null)
+          .evaluate();
+      fail();
+    } catch (RuntimeException thrownException) {
+      assertSame(exceptionToThrow, thrownException);
+      assertEquals(1, exceptionToThrow.getSuppressed().length);
+      assertEquals(IllegalStateException.class, exceptionToThrow.getSuppressed()[0].getClass());
+      assertEquals(
+          "Test executor failed to shutdown cleanly.",
+          exceptionToThrow.getSuppressed()[0].getMessage());
+    }
+    assertTrue(service.isShutdown());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/fdd5971d/sdks/java/fn-execution/src/test/java/org/apache/beam/harness/test/TestStreams.java
----------------------------------------------------------------------
diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/harness/test/TestStreams.java b/sdks/java/fn-execution/src/test/java/org/apache/beam/harness/test/TestStreams.java
new file mode 100644
index 0000000..a7b362d
--- /dev/null
+++ b/sdks/java/fn-execution/src/test/java/org/apache/beam/harness/test/TestStreams.java
@@ -0,0 +1,162 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.harness.test;
+
+import io.grpc.stub.CallStreamObserver;
+import io.grpc.stub.StreamObserver;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+
+/** Utility methods which enable testing of {@link StreamObserver}s. */
+public class TestStreams {
+  /**
+   * Creates a test {@link CallStreamObserver}  {@link Builder} that forwards
+   * {@link StreamObserver#onNext} calls to the supplied {@link Consumer}.
+   */
+  public static <T> Builder<T> withOnNext(Consumer<T> onNext) {
+    return new Builder<>(new ForwardingCallStreamObserver<>(
+        onNext,
+        TestStreams::noop,
+        TestStreams::noop,
+        TestStreams::returnTrue));
+  }
+
+  /** A builder for a test {@link CallStreamObserver} that performs various callbacks. */
+  public static class Builder<T> {
+    private final ForwardingCallStreamObserver<T> observer;
+    private Builder(ForwardingCallStreamObserver<T> observer) {
+      this.observer = observer;
+    }
+
+    /**
+     * Returns a new {@link Builder} like this one with the specified
+     * {@link CallStreamObserver#isReady} callback.
+     */
+    public Builder<T> withIsReady(Supplier<Boolean> isReady) {
+      return new Builder<>(new ForwardingCallStreamObserver<>(
+          observer.onNext,
+          observer.onError,
+          observer.onCompleted,
+          isReady));
+    }
+
+    /**
+     * Returns a new {@link Builder} like this one with the specified
+     * {@link StreamObserver#onCompleted} callback.
+     */
+    public Builder<T> withOnCompleted(Runnable onCompleted) {
+      return new Builder<>(new ForwardingCallStreamObserver<>(
+          observer.onNext,
+          observer.onError,
+          onCompleted,
+          observer.isReady));
+    }
+
+    /**
+     * Returns a new {@link Builder} like this one with the specified
+     * {@link StreamObserver#onError} callback.
+     */
+    public Builder<T> withOnError(Runnable onError) {
+      return new Builder<>(new ForwardingCallStreamObserver<>(
+          observer.onNext,
+          new Consumer<Throwable>() {
+            @Override
+            public void accept(Throwable t) {
+              onError.run();
+            }
+          },
+          observer.onCompleted,
+          observer.isReady));
+    }
+
+    /**
+     * Returns a new {@link Builder} like this one with the specified
+     * {@link StreamObserver#onError} consumer.
+     */
+    public Builder<T> withOnError(Consumer<Throwable> onError) {
+      return new Builder<>(new ForwardingCallStreamObserver<>(
+          observer.onNext, onError, observer.onCompleted, observer.isReady));
+    }
+
+    public CallStreamObserver<T> build() {
+      return observer;
+    }
+  }
+
+  private static void noop() {
+  }
+
+  private static void noop(Throwable t) {
+  }
+
+  private static boolean returnTrue() {
+    return true;
+  }
+
+  /** A {@link CallStreamObserver} which executes the supplied callbacks. */
+  private static class ForwardingCallStreamObserver<T> extends CallStreamObserver<T> {
+    private final Consumer<T> onNext;
+    private final Supplier<Boolean> isReady;
+    private final Consumer<Throwable> onError;
+    private final Runnable onCompleted;
+
+    public ForwardingCallStreamObserver(
+        Consumer<T> onNext,
+        Consumer<Throwable> onError,
+        Runnable onCompleted,
+        Supplier<Boolean> isReady) {
+      this.onNext = onNext;
+      this.onError = onError;
+      this.onCompleted = onCompleted;
+      this.isReady = isReady;
+    }
+
+    @Override
+    public void onNext(T value) {
+      onNext.accept(value);
+    }
+
+    @Override
+    public void onError(Throwable t) {
+      onError.accept(t);
+    }
+
+    @Override
+    public void onCompleted() {
+      onCompleted.run();
+    }
+
+    @Override
+    public boolean isReady() {
+      return isReady.get();
+    }
+
+    @Override
+    public void setOnReadyHandler(Runnable onReadyHandler) {}
+
+    @Override
+    public void disableAutoInboundFlowControl() {}
+
+    @Override
+    public void request(int count) {}
+
+    @Override
+    public void setMessageCompression(boolean enable) {}
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/fdd5971d/sdks/java/fn-execution/src/test/java/org/apache/beam/harness/test/TestStreamsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/harness/test/TestStreamsTest.java b/sdks/java/fn-execution/src/test/java/org/apache/beam/harness/test/TestStreamsTest.java
new file mode 100644
index 0000000..f5741ae
--- /dev/null
+++ b/sdks/java/fn-execution/src/test/java/org/apache/beam/harness/test/TestStreamsTest.java
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.harness.test;
+
+import static org.hamcrest.Matchers.contains;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Tests for {@link TestStreams}. */
+@RunWith(JUnit4.class)
+public class TestStreamsTest {
+  @Test
+  public void testOnNextIsCalled() {
+    AtomicBoolean onNextWasCalled = new AtomicBoolean();
+    TestStreams.withOnNext(onNextWasCalled::set).build().onNext(true);
+    assertTrue(onNextWasCalled.get());
+  }
+
+  @Test
+  public void testIsReadyIsCalled() {
+    final AtomicBoolean isReadyWasCalled = new AtomicBoolean();
+    assertFalse(TestStreams.withOnNext(null)
+        .withIsReady(() -> isReadyWasCalled.getAndSet(true))
+        .build()
+        .isReady());
+    assertTrue(isReadyWasCalled.get());
+  }
+
+  @Test
+  public void testOnCompletedIsCalled() {
+    AtomicBoolean onCompletedWasCalled = new AtomicBoolean();
+    TestStreams.withOnNext(null)
+        .withOnCompleted(() -> onCompletedWasCalled.set(true))
+        .build()
+        .onCompleted();
+    assertTrue(onCompletedWasCalled.get());
+  }
+
+  @Test
+  public void testOnErrorRunnableIsCalled() {
+    RuntimeException throwable = new RuntimeException();
+    AtomicBoolean onErrorWasCalled = new AtomicBoolean();
+    TestStreams.withOnNext(null)
+        .withOnError(() -> onErrorWasCalled.set(true))
+        .build()
+        .onError(throwable);
+    assertTrue(onErrorWasCalled.get());
+  }
+
+  @Test
+  public void testOnErrorConsumerIsCalled() {
+    RuntimeException throwable = new RuntimeException();
+    Collection<Throwable> onErrorWasCalled = new ArrayList<>();
+    TestStreams.withOnNext(null)
+        .withOnError(onErrorWasCalled::add)
+        .build()
+        .onError(throwable);
+    assertThat(onErrorWasCalled, contains(throwable));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/fdd5971d/sdks/java/harness/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/harness/pom.xml b/sdks/java/harness/pom.xml
index e60c8a4..3707730 100644
--- a/sdks/java/harness/pom.xml
+++ b/sdks/java/harness/pom.xml
@@ -36,6 +36,17 @@
       <plugins>
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
+          <artifactId>maven-dependency-plugin</artifactId>
+          <configuration>
+            <ignoredUnusedDeclaredDependencies>
+              <ignoredUnusedDeclaredDependency>
+                com.google.protobuf:protobuf-java
+              </ignoredUnusedDeclaredDependency>
+            </ignoredUnusedDeclaredDependencies>
+          </configuration>
+        </plugin>
+        <plugin>
+          <groupId>org.apache.maven.plugins</groupId>
           <artifactId>maven-shade-plugin</artifactId>
           <executions>
             <execution>
@@ -139,6 +150,11 @@
 
     <dependency>
       <groupId>org.apache.beam</groupId>
+      <artifactId>beam-sdks-java-fn-execution</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.beam</groupId>
       <artifactId>beam-sdks-java-core</artifactId>
     </dependency>
 
@@ -155,6 +171,14 @@
       <artifactId>beam-sdks-java-extensions-google-cloud-platform-core</artifactId>
     </dependency>
 
+    <!-- test dependencies -->
+    <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>beam-sdks-java-fn-execution</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+
     <dependency>
       <groupId>org.apache.beam</groupId>
       <artifactId>beam-runners-core-java</artifactId>
@@ -205,6 +229,7 @@
     <dependency>
       <groupId>io.grpc</groupId>
       <artifactId>grpc-netty</artifactId>
+      <scope>runtime</scope>
     </dependency>
 
     <dependency>
@@ -216,6 +241,7 @@
       <groupId>io.netty</groupId>
       <artifactId>netty-transport-native-epoll</artifactId>
       <classifier>linux-x86_64</classifier>
+      <scope>runtime</scope>
     </dependency>
 
     <dependency>

http://git-wip-us.apache.org/repos/asf/beam/blob/fdd5971d/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java
index 7d78856..e1790fa 100644
--- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java
@@ -21,7 +21,7 @@ package org.apache.beam.fn.harness;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.protobuf.TextFormat;
 import java.util.EnumMap;
-import org.apache.beam.fn.harness.channel.ManagedChannelFactory;
+import java.util.List;
 import org.apache.beam.fn.harness.control.BeamFnControlClient;
 import org.apache.beam.fn.harness.control.ProcessBundleHandler;
 import org.apache.beam.fn.harness.control.RegisterHandler;
@@ -30,9 +30,13 @@ import org.apache.beam.fn.harness.fn.ThrowingFunction;
 import org.apache.beam.fn.harness.logging.BeamFnLoggingClient;
 import org.apache.beam.fn.harness.state.BeamFnStateGrpcClientCache;
 import org.apache.beam.fn.harness.stream.StreamObserverFactory;
+import org.apache.beam.harness.channel.ManagedChannelFactory;
 import org.apache.beam.model.fnexecution.v1.BeamFnApi;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi.InstructionRequest;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi.InstructionResponse.Builder;
 import org.apache.beam.model.pipeline.v1.Endpoints;
 import org.apache.beam.sdk.extensions.gcp.options.GcsOptions;
+import org.apache.beam.sdk.options.ExperimentalOptions;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.util.common.ReflectHelpers;
 import org.slf4j.Logger;
@@ -90,7 +94,13 @@ public class FnHarness {
   public static void main(PipelineOptions options,
       Endpoints.ApiServiceDescriptor loggingApiServiceDescriptor,
       Endpoints.ApiServiceDescriptor controlApiServiceDescriptor) throws Exception {
-    ManagedChannelFactory channelFactory = ManagedChannelFactory.from(options);
+    ManagedChannelFactory channelFactory;
+    List<String> experiments = options.as(ExperimentalOptions.class).getExperiments();
+    if (experiments != null && experiments.contains("beam_fn_api_epoll")) {
+      channelFactory = ManagedChannelFactory.createEpoll();
+    } else {
+      channelFactory = ManagedChannelFactory.createDefault();
+    }
     StreamObserverFactory streamObserverFactory = StreamObserverFactory.fromOptions(options);
     try (BeamFnLoggingClient logging = new BeamFnLoggingClient(
         options,
@@ -99,9 +109,8 @@ public class FnHarness {
 
       LOG.info("Fn Harness started");
       EnumMap<BeamFnApi.InstructionRequest.RequestCase,
-              ThrowingFunction<BeamFnApi.InstructionRequest,
-                               BeamFnApi.InstructionResponse.Builder>> handlers =
-          new EnumMap<>(BeamFnApi.InstructionRequest.RequestCase.class);
+              ThrowingFunction<InstructionRequest, Builder>>
+          handlers = new EnumMap<>(BeamFnApi.InstructionRequest.RequestCase.class);
 
       RegisterHandler fnApiRegistry = new RegisterHandler();
       BeamFnDataGrpcClient beamFnDataMultiplexer = new BeamFnDataGrpcClient(

http://git-wip-us.apache.org/repos/asf/beam/blob/fdd5971d/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/channel/ManagedChannelFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/channel/ManagedChannelFactory.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/channel/ManagedChannelFactory.java
deleted file mode 100644
index 0c615a9..0000000
--- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/channel/ManagedChannelFactory.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.beam.fn.harness.channel;
-
-import io.grpc.ManagedChannel;
-import io.grpc.ManagedChannelBuilder;
-import io.grpc.netty.NettyChannelBuilder;
-import io.netty.channel.epoll.EpollDomainSocketChannel;
-import io.netty.channel.epoll.EpollEventLoopGroup;
-import io.netty.channel.epoll.EpollSocketChannel;
-import io.netty.channel.unix.DomainSocketAddress;
-import java.net.SocketAddress;
-import java.util.List;
-import org.apache.beam.model.pipeline.v1.Endpoints.ApiServiceDescriptor;
-import org.apache.beam.sdk.options.ExperimentalOptions;
-import org.apache.beam.sdk.options.PipelineOptions;
-
-/**
- * Uses {@link PipelineOptions} to configure which underlying {@link ManagedChannel} implementation
- * to use.
- */
-public abstract class ManagedChannelFactory {
-  public static ManagedChannelFactory from(PipelineOptions options) {
-    List<String> experiments = options.as(ExperimentalOptions.class).getExperiments();
-    if (experiments != null && experiments.contains("beam_fn_api_epoll")) {
-      io.netty.channel.epoll.Epoll.ensureAvailability();
-      return new Epoll();
-    }
-    return new Default();
-  }
-
-  public abstract ManagedChannel forDescriptor(ApiServiceDescriptor apiServiceDescriptor);
-
-  /**
-   * Creates a {@link ManagedChannel} backed by an {@link EpollDomainSocketChannel} if the address
-   * is a {@link DomainSocketAddress}. Otherwise creates a {@link ManagedChannel} backed by an
-   * {@link EpollSocketChannel}.
-   */
-  private static class Epoll extends ManagedChannelFactory {
-    @Override
-    public ManagedChannel forDescriptor(ApiServiceDescriptor apiServiceDescriptor) {
-      SocketAddress address = SocketAddressFactory.createFrom(apiServiceDescriptor.getUrl());
-      return NettyChannelBuilder.forAddress(address)
-          .channelType(address instanceof DomainSocketAddress
-              ? EpollDomainSocketChannel.class : EpollSocketChannel.class)
-          .eventLoopGroup(new EpollEventLoopGroup())
-          .usePlaintext(true)
-          // Set the message size to max value here. The actual size is governed by the
-          // buffer size in the layers above.
-          .maxInboundMessageSize(Integer.MAX_VALUE)
-          .build();
-    }
-  }
-
-  /**
-   * Creates a {@link ManagedChannel} relying on the {@link ManagedChannelBuilder} to create
-   * instances.
-   */
-  private static class Default extends ManagedChannelFactory {
-    @Override
-    public ManagedChannel forDescriptor(ApiServiceDescriptor apiServiceDescriptor) {
-      return ManagedChannelBuilder.forTarget(apiServiceDescriptor.getUrl())
-          .usePlaintext(true)
-          // Set the message size to max value here. The actual size is governed by the
-          // buffer size in the layers above.
-          .maxInboundMessageSize(Integer.MAX_VALUE)
-          .build();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/fdd5971d/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/channel/SocketAddressFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/channel/SocketAddressFactory.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/channel/SocketAddressFactory.java
deleted file mode 100644
index a27d542..0000000
--- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/channel/SocketAddressFactory.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.beam.fn.harness.channel;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import com.google.common.net.HostAndPort;
-import io.netty.channel.unix.DomainSocketAddress;
-import java.io.File;
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.net.SocketAddress;
-
-/** Creates a {@link SocketAddress} based upon a supplied string. */
-public class SocketAddressFactory {
-  private static final String UNIX_DOMAIN_SOCKET_PREFIX = "unix://";
-
-  /**
-   * Parse a {@link SocketAddress} from the given string.
-   */
-  public static SocketAddress createFrom(String value) {
-    if (value.startsWith(UNIX_DOMAIN_SOCKET_PREFIX)) {
-      // Unix Domain Socket address.
-      // Create the underlying file for the Unix Domain Socket.
-      String filePath = value.substring(UNIX_DOMAIN_SOCKET_PREFIX.length());
-      File file = new File(filePath);
-      if (!file.isAbsolute()) {
-        throw new IllegalArgumentException("File path must be absolute: " + filePath);
-      }
-      try {
-        if (file.createNewFile()) {
-          // If this application created the file, delete it when the application exits.
-          file.deleteOnExit();
-        }
-      } catch (IOException ex) {
-        throw new RuntimeException(ex);
-      }
-      // Create the SocketAddress referencing the file.
-      return new DomainSocketAddress(file);
-    } else {
-      // Standard TCP/IP address.
-      HostAndPort hostAndPort = HostAndPort.fromString(value);
-      checkArgument(hostAndPort.hasPort(),
-          "Address must be a unix:// path or be in the form host:port. Got: %s", value);
-      return new InetSocketAddress(hostAndPort.getHostText(), hostAndPort.getPort());
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/fdd5971d/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/channel/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/channel/package-info.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/channel/package-info.java
deleted file mode 100644
index 6323166..0000000
--- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/channel/package-info.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-/**
- * gRPC channel management.
- */
-package org.apache.beam.fn.harness.channel;

http://git-wip-us.apache.org/repos/asf/beam/blob/fdd5971d/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataReadRunnerTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataReadRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataReadRunnerTest.java
index de68d41..f00346d 100644
--- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataReadRunnerTest.java
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataReadRunnerTest.java
@@ -49,8 +49,8 @@ import org.apache.beam.fn.harness.PTransformRunnerFactory.Registrar;
 import org.apache.beam.fn.harness.data.BeamFnDataClient;
 import org.apache.beam.fn.harness.fn.ThrowingConsumer;
 import org.apache.beam.fn.harness.fn.ThrowingRunnable;
-import org.apache.beam.fn.harness.test.TestExecutors;
-import org.apache.beam.fn.harness.test.TestExecutors.TestExecutorService;
+import org.apache.beam.harness.test.TestExecutors;
+import org.apache.beam.harness.test.TestExecutors.TestExecutorService;
 import org.apache.beam.model.fnexecution.v1.BeamFnApi;
 import org.apache.beam.model.pipeline.v1.Endpoints;
 import org.apache.beam.model.pipeline.v1.RunnerApi;

http://git-wip-us.apache.org/repos/asf/beam/blob/fdd5971d/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnHarnessTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnHarnessTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnHarnessTest.java
index fc89acf..66c31a8 100644
--- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnHarnessTest.java
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnHarnessTest.java
@@ -29,7 +29,7 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.CountDownLatch;
 import java.util.function.Consumer;
-import org.apache.beam.fn.harness.test.TestStreams;
+import org.apache.beam.harness.test.TestStreams;
 import org.apache.beam.model.fnexecution.v1.BeamFnApi;
 import org.apache.beam.model.fnexecution.v1.BeamFnApi.InstructionRequest;
 import org.apache.beam.model.fnexecution.v1.BeamFnApi.InstructionResponse;

http://git-wip-us.apache.org/repos/asf/beam/blob/fdd5971d/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/channel/ManagedChannelFactoryTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/channel/ManagedChannelFactoryTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/channel/ManagedChannelFactoryTest.java
deleted file mode 100644
index 6f27e21..0000000
--- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/channel/ManagedChannelFactoryTest.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.beam.fn.harness.channel;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assume.assumeTrue;
-
-import io.grpc.ManagedChannel;
-import org.apache.beam.model.pipeline.v1.Endpoints;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/** Tests for {@link ManagedChannelFactory}. */
-@RunWith(JUnit4.class)
-public class ManagedChannelFactoryTest {
-  @Rule public TemporaryFolder tmpFolder = new TemporaryFolder();
-
-  @Test
-  public void testDefaultChannel() {
-    Endpoints.ApiServiceDescriptor apiServiceDescriptor =
-        Endpoints.ApiServiceDescriptor.newBuilder().setUrl("localhost:123").build();
-    ManagedChannel channel = ManagedChannelFactory.from(PipelineOptionsFactory.create())
-        .forDescriptor(apiServiceDescriptor);
-    assertEquals("localhost:123", channel.authority());
-    channel.shutdownNow();
-  }
-
-  @Test
-  public void testEpollHostPortChannel() {
-    assumeTrue(io.netty.channel.epoll.Epoll.isAvailable());
-    Endpoints.ApiServiceDescriptor apiServiceDescriptor =
-        Endpoints.ApiServiceDescriptor.newBuilder().setUrl("localhost:123").build();
-    ManagedChannel channel = ManagedChannelFactory.from(
-        PipelineOptionsFactory.fromArgs(new String[]{ "--experiments=beam_fn_api_epoll" }).create())
-        .forDescriptor(apiServiceDescriptor);
-    assertEquals("localhost:123", channel.authority());
-    channel.shutdownNow();
-  }
-
-  @Test
-  public void testEpollDomainSocketChannel() throws Exception {
-    assumeTrue(io.netty.channel.epoll.Epoll.isAvailable());
-    Endpoints.ApiServiceDescriptor apiServiceDescriptor =
-        Endpoints.ApiServiceDescriptor.newBuilder()
-            .setUrl("unix://" + tmpFolder.newFile().getAbsolutePath())
-            .build();
-    ManagedChannel channel = ManagedChannelFactory.from(
-        PipelineOptionsFactory.fromArgs(new String[]{ "--experiments=beam_fn_api_epoll" }).create())
-        .forDescriptor(apiServiceDescriptor);
-    assertEquals(apiServiceDescriptor.getUrl().substring("unix://".length()), channel.authority());
-    channel.shutdownNow();
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/fdd5971d/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/channel/SocketAddressFactoryTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/channel/SocketAddressFactoryTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/channel/SocketAddressFactoryTest.java
deleted file mode 100644
index 610a8ea..0000000
--- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/channel/SocketAddressFactoryTest.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.beam.fn.harness.channel;
-
-import static org.hamcrest.Matchers.instanceOf;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertThat;
-
-import io.netty.channel.unix.DomainSocketAddress;
-import java.io.File;
-import java.net.InetSocketAddress;
-import java.net.SocketAddress;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/** Tests for {@link SocketAddressFactory}. */
-@RunWith(JUnit4.class)
-public class SocketAddressFactoryTest {
-  @Rule public TemporaryFolder tmpFolder = new TemporaryFolder();
-
-  @Test
-  public void testHostPortSocket() {
-    SocketAddress socketAddress = SocketAddressFactory.createFrom("localhost:123");
-    assertThat(socketAddress, instanceOf(InetSocketAddress.class));
-    assertEquals("localhost", ((InetSocketAddress) socketAddress).getHostString());
-    assertEquals(123, ((InetSocketAddress) socketAddress).getPort());
-  }
-
-  @Test
-  public void testDomainSocket() throws Exception {
-    File tmpFile = tmpFolder.newFile();
-    SocketAddress socketAddress = SocketAddressFactory.createFrom(
-        "unix://" + tmpFile.getAbsolutePath());
-    assertThat(socketAddress, instanceOf(DomainSocketAddress.class));
-    assertEquals(tmpFile.getAbsolutePath(), ((DomainSocketAddress) socketAddress).path());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/fdd5971d/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/BeamFnControlClientTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/BeamFnControlClientTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/BeamFnControlClientTest.java
index 8dc62b3..56ae7ed 100644
--- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/BeamFnControlClientTest.java
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/BeamFnControlClientTest.java
@@ -39,7 +39,7 @@ import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.function.Function;
 import org.apache.beam.fn.harness.fn.ThrowingFunction;
-import org.apache.beam.fn.harness.test.TestStreams;
+import org.apache.beam.harness.test.TestStreams;
 import org.apache.beam.model.fnexecution.v1.BeamFnApi;
 import org.apache.beam.model.fnexecution.v1.BeamFnControlGrpc;
 import org.apache.beam.model.pipeline.v1.Endpoints;

http://git-wip-us.apache.org/repos/asf/beam/blob/fdd5971d/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/RegisterHandlerTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/RegisterHandlerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/RegisterHandlerTest.java
index 40b2145..aa1a504 100644
--- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/RegisterHandlerTest.java
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/RegisterHandlerTest.java
@@ -23,8 +23,8 @@ import static org.junit.Assert.assertEquals;
 import java.util.concurrent.Callable;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
-import org.apache.beam.fn.harness.test.TestExecutors;
-import org.apache.beam.fn.harness.test.TestExecutors.TestExecutorService;
+import org.apache.beam.harness.test.TestExecutors;
+import org.apache.beam.harness.test.TestExecutors.TestExecutorService;
 import org.apache.beam.model.fnexecution.v1.BeamFnApi;
 import org.apache.beam.model.fnexecution.v1.BeamFnApi.RegisterResponse;
 import org.apache.beam.model.pipeline.v1.RunnerApi;

http://git-wip-us.apache.org/repos/asf/beam/blob/fdd5971d/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataBufferingOutboundObserverTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataBufferingOutboundObserverTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataBufferingOutboundObserverTest.java
index 4898b90..81b1aa4 100644
--- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataBufferingOutboundObserverTest.java
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataBufferingOutboundObserverTest.java
@@ -29,7 +29,7 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.concurrent.atomic.AtomicBoolean;
 import org.apache.beam.fn.harness.fn.CloseableThrowingConsumer;
-import org.apache.beam.fn.harness.test.TestStreams;
+import org.apache.beam.harness.test.TestStreams;
 import org.apache.beam.model.fnexecution.v1.BeamFnApi;
 import org.apache.beam.sdk.coders.ByteArrayCoder;
 import org.apache.beam.sdk.coders.Coder;

http://git-wip-us.apache.org/repos/asf/beam/blob/fdd5971d/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataGrpcClientTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataGrpcClientTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataGrpcClientTest.java
index 5e2545d..7df8925 100644
--- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataGrpcClientTest.java
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataGrpcClientTest.java
@@ -45,7 +45,7 @@ import java.util.function.Consumer;
 import java.util.function.Function;
 import org.apache.beam.fn.harness.fn.CloseableThrowingConsumer;
 import org.apache.beam.fn.harness.fn.ThrowingConsumer;
-import org.apache.beam.fn.harness.test.TestStreams;
+import org.apache.beam.harness.test.TestStreams;
 import org.apache.beam.model.fnexecution.v1.BeamFnApi;
 import org.apache.beam.model.fnexecution.v1.BeamFnDataGrpc;
 import org.apache.beam.model.pipeline.v1.Endpoints;

http://git-wip-us.apache.org/repos/asf/beam/blob/fdd5971d/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataGrpcMultiplexerTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataGrpcMultiplexerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataGrpcMultiplexerTest.java
index c63dd62..6a12ed0 100644
--- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataGrpcMultiplexerTest.java
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataGrpcMultiplexerTest.java
@@ -30,7 +30,7 @@ import java.util.Collection;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
-import org.apache.beam.fn.harness.test.TestStreams;
+import org.apache.beam.harness.test.TestStreams;
 import org.apache.beam.model.fnexecution.v1.BeamFnApi;
 import org.apache.beam.model.pipeline.v1.Endpoints;
 import org.apache.beam.sdk.values.KV;

http://git-wip-us.apache.org/repos/asf/beam/blob/fdd5971d/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/logging/BeamFnLoggingClientTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/logging/BeamFnLoggingClientTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/logging/BeamFnLoggingClientTest.java
index c9057ea..1e68b18 100644
--- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/logging/BeamFnLoggingClientTest.java
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/logging/BeamFnLoggingClientTest.java
@@ -41,7 +41,7 @@ import java.util.concurrent.atomic.AtomicReference;
 import java.util.logging.Level;
 import java.util.logging.LogManager;
 import java.util.logging.LogRecord;
-import org.apache.beam.fn.harness.test.TestStreams;
+import org.apache.beam.harness.test.TestStreams;
 import org.apache.beam.model.fnexecution.v1.BeamFnApi;
 import org.apache.beam.model.fnexecution.v1.BeamFnLoggingGrpc;
 import org.apache.beam.model.pipeline.v1.Endpoints;

http://git-wip-us.apache.org/repos/asf/beam/blob/fdd5971d/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/BeamFnStateGrpcClientCacheTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/BeamFnStateGrpcClientCacheTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/BeamFnStateGrpcClientCacheTest.java
index e8c616d..12c9c43 100644
--- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/BeamFnStateGrpcClientCacheTest.java
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/BeamFnStateGrpcClientCacheTest.java
@@ -40,7 +40,7 @@ import java.util.concurrent.ExecutionException;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.function.Function;
 import org.apache.beam.fn.harness.IdGenerator;
-import org.apache.beam.fn.harness.test.TestStreams;
+import org.apache.beam.harness.test.TestStreams;
 import org.apache.beam.model.fnexecution.v1.BeamFnApi.StateRequest;
 import org.apache.beam.model.fnexecution.v1.BeamFnApi.StateResponse;
 import org.apache.beam.model.fnexecution.v1.BeamFnStateGrpc;

http://git-wip-us.apache.org/repos/asf/beam/blob/fdd5971d/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/stream/BufferingStreamObserverTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/stream/BufferingStreamObserverTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/stream/BufferingStreamObserverTest.java
index b26e8e1..3f66c4c 100644
--- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/stream/BufferingStreamObserverTest.java
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/stream/BufferingStreamObserverTest.java
@@ -32,9 +32,9 @@ import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.function.Consumer;
-import org.apache.beam.fn.harness.test.TestExecutors;
-import org.apache.beam.fn.harness.test.TestExecutors.TestExecutorService;
-import org.apache.beam.fn.harness.test.TestStreams;
+import org.apache.beam.harness.test.TestExecutors;
+import org.apache.beam.harness.test.TestExecutors.TestExecutorService;
+import org.apache.beam.harness.test.TestStreams;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.runner.RunWith;

http://git-wip-us.apache.org/repos/asf/beam/blob/fdd5971d/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/stream/DirectStreamObserverTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/stream/DirectStreamObserverTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/stream/DirectStreamObserverTest.java
index b5d3ec1..120a73d 100644
--- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/stream/DirectStreamObserverTest.java
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/stream/DirectStreamObserverTest.java
@@ -32,9 +32,9 @@ import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.function.Consumer;
-import org.apache.beam.fn.harness.test.TestExecutors;
-import org.apache.beam.fn.harness.test.TestExecutors.TestExecutorService;
-import org.apache.beam.fn.harness.test.TestStreams;
+import org.apache.beam.harness.test.TestExecutors;
+import org.apache.beam.harness.test.TestExecutors.TestExecutorService;
+import org.apache.beam.harness.test.TestStreams;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.runner.RunWith;

http://git-wip-us.apache.org/repos/asf/beam/blob/fdd5971d/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/test/TestExecutors.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/test/TestExecutors.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/test/TestExecutors.java
deleted file mode 100644
index f846466..0000000
--- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/test/TestExecutors.java
+++ /dev/null
@@ -1,85 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.beam.fn.harness.test;
-
-import com.google.common.util.concurrent.ForwardingExecutorService;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.TimeUnit;
-import java.util.function.Supplier;
-import org.junit.rules.TestRule;
-import org.junit.runner.Description;
-import org.junit.runners.model.Statement;
-
-/**
- * A {@link TestRule} that validates that all submitted tasks finished and were completed. This
- * allows for testing that tasks have exercised the appropriate shutdown logic.
- */
-public class TestExecutors {
-  public static TestExecutorService from(Supplier<ExecutorService> executorServiceSuppler) {
-    return new FromSupplier(executorServiceSuppler);
-  }
-
-  /** A union of the {@link ExecutorService} and {@link TestRule} interfaces. */
-  public interface TestExecutorService extends ExecutorService, TestRule {}
-
-  private static class FromSupplier extends ForwardingExecutorService
-      implements TestExecutorService {
-    private final Supplier<ExecutorService> executorServiceSupplier;
-    private ExecutorService delegate;
-
-    private FromSupplier(Supplier<ExecutorService> executorServiceSupplier) {
-      this.executorServiceSupplier = executorServiceSupplier;
-    }
-
-    @Override
-    public Statement apply(Statement statement, Description arg1) {
-      return new Statement() {
-        @Override
-        public void evaluate() throws Throwable {
-          Throwable thrown = null;
-          delegate = executorServiceSupplier.get();
-          try {
-            statement.evaluate();
-          } catch (Throwable t) {
-            thrown = t;
-          }
-          shutdown();
-          if (!awaitTermination(5, TimeUnit.SECONDS)) {
-            shutdownNow();
-            IllegalStateException e =
-                new IllegalStateException("Test executor failed to shutdown cleanly.");
-            if (thrown != null) {
-              thrown.addSuppressed(e);
-            } else {
-              thrown = e;
-            }
-          }
-          if (thrown != null) {
-            throw thrown;
-          }
-        }
-      };
-    }
-
-    @Override
-    protected ExecutorService delegate() {
-      return delegate;
-    }
-  }
-}


[09/50] [abbrv] beam git commit: Do not relocate generated Model Classes

Posted by ke...@apache.org.
Do not relocate generated Model Classes

Doing so with this pattern breaks the clients, as the shade plugin will
rewrite the gRPC method name as well, which servers do not implement.
All calls will then be broken.


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

Branch: refs/heads/mr-runner
Commit: 6ea8afd783db33e8ad6437be276d9d817f5a6261
Parents: 8247b53
Author: Thomas Groh <tg...@google.com>
Authored: Thu Oct 26 15:00:12 2017 -0700
Committer: Thomas Groh <tg...@google.com>
Committed: Wed Nov 1 13:57:26 2017 -0700

----------------------------------------------------------------------
 sdks/java/harness/pom.xml | 7 -------
 1 file changed, 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/6ea8afd7/sdks/java/harness/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/harness/pom.xml b/sdks/java/harness/pom.xml
index 6343b3e..e60c8a4 100644
--- a/sdks/java/harness/pom.xml
+++ b/sdks/java/harness/pom.xml
@@ -91,13 +91,6 @@
                     </shadedPattern>
                   </relocation>
                   <relocation>
-                    <pattern>org.apache.beam.model</pattern>
-                    <!--suppress MavenModelInspection -->
-                    <shadedPattern>
-                      org.apache.beam.fn.harness.private.org.apache.beam.model
-                    </shadedPattern>
-                  </relocation>
-                  <relocation>
                     <pattern>org.apache.beam.runners</pattern>
                     <!--suppress MavenModelInspection -->
                     <shadedPattern>


[04/50] [abbrv] beam git commit: Remove obsolete extra parameter

Posted by ke...@apache.org.
Remove obsolete extra parameter


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

Branch: refs/heads/mr-runner
Commit: 64437549540d72dd11c10944f0175df4efa0a81b
Parents: b743ab1
Author: Rafal Wojdyla <ra...@spotify.com>
Authored: Mon Oct 30 21:58:24 2017 -0400
Committer: Luke Cwik <lc...@google.com>
Committed: Wed Nov 1 10:46:24 2017 -0700

----------------------------------------------------------------------
 .../org/apache/beam/sdk/transforms/join/CoGbkResult.java     | 8 +++-----
 1 file changed, 3 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/64437549/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResult.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResult.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResult.java
index 16a0bae..1dad9f4 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResult.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResult.java
@@ -117,15 +117,13 @@ public class CoGbkResult {
       // against a given tag would not match anything.
       final Boolean[] containsTag = new Boolean[schema.size()];
       for (int unionTag = 0; unionTag < schema.size(); unionTag++) {
-        final int unionTag0 = unionTag;
-        updateUnionTag(tail, containsTag, unionTag, unionTag0);
+        updateUnionTag(tail, containsTag, unionTag);
       }
     }
   }
 
   private <T> void updateUnionTag(
-      final Reiterator<RawUnionValue> tail, final Boolean[] containsTag,
-      int unionTag, final int unionTag0) {
+      final Reiterator<RawUnionValue> tail, final Boolean[] containsTag, final int unionTag) {
     @SuppressWarnings("unchecked")
     final Iterable<T> head = (Iterable<T>) valueMap.get(unionTag);
     valueMap.set(
@@ -135,7 +133,7 @@ public class CoGbkResult {
           public Iterator<T> iterator() {
             return Iterators.concat(
                 head.iterator(),
-                new UnionValueIterator<T>(unionTag0, tail.copy(), containsTag));
+                new UnionValueIterator<T>(unionTag, tail.copy(), containsTag));
           }
         });
   }


[30/50] [abbrv] beam git commit: Add all portability protos to Go

Posted by ke...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/42100456/sdks/go/pkg/beam/model/pipeline_v1/endpoints.pb.go
----------------------------------------------------------------------
diff --git a/sdks/go/pkg/beam/model/pipeline_v1/endpoints.pb.go b/sdks/go/pkg/beam/model/pipeline_v1/endpoints.pb.go
new file mode 100644
index 0000000..72a230d
--- /dev/null
+++ b/sdks/go/pkg/beam/model/pipeline_v1/endpoints.pb.go
@@ -0,0 +1,160 @@
+// Code generated by protoc-gen-go. DO NOT EDIT.
+// source: endpoints.proto
+
+package pipeline_v1
+
+import proto "github.com/golang/protobuf/proto"
+import fmt "fmt"
+import math "math"
+
+// Reference imports to suppress errors if they are not otherwise used.
+var _ = proto.Marshal
+var _ = fmt.Errorf
+var _ = math.Inf
+
+type ApiServiceDescriptor struct {
+	// (Required) The URL to connect to.
+	Url string `protobuf:"bytes,2,opt,name=url" json:"url,omitempty"`
+	// (Optional) The method for authentication. If unspecified, access to the
+	// url is already being performed in a trusted context (e.g. localhost,
+	// private network).
+	//
+	// Types that are valid to be assigned to Authentication:
+	//	*ApiServiceDescriptor_Oauth2ClientCredentialsGrant
+	Authentication isApiServiceDescriptor_Authentication `protobuf_oneof:"authentication"`
+}
+
+func (m *ApiServiceDescriptor) Reset()                    { *m = ApiServiceDescriptor{} }
+func (m *ApiServiceDescriptor) String() string            { return proto.CompactTextString(m) }
+func (*ApiServiceDescriptor) ProtoMessage()               {}
+func (*ApiServiceDescriptor) Descriptor() ([]byte, []int) { return fileDescriptor1, []int{0} }
+
+type isApiServiceDescriptor_Authentication interface {
+	isApiServiceDescriptor_Authentication()
+}
+
+type ApiServiceDescriptor_Oauth2ClientCredentialsGrant struct {
+	Oauth2ClientCredentialsGrant *OAuth2ClientCredentialsGrant `protobuf:"bytes,3,opt,name=oauth2_client_credentials_grant,json=oauth2ClientCredentialsGrant,oneof"`
+}
+
+func (*ApiServiceDescriptor_Oauth2ClientCredentialsGrant) isApiServiceDescriptor_Authentication() {}
+
+func (m *ApiServiceDescriptor) GetAuthentication() isApiServiceDescriptor_Authentication {
+	if m != nil {
+		return m.Authentication
+	}
+	return nil
+}
+
+func (m *ApiServiceDescriptor) GetUrl() string {
+	if m != nil {
+		return m.Url
+	}
+	return ""
+}
+
+func (m *ApiServiceDescriptor) GetOauth2ClientCredentialsGrant() *OAuth2ClientCredentialsGrant {
+	if x, ok := m.GetAuthentication().(*ApiServiceDescriptor_Oauth2ClientCredentialsGrant); ok {
+		return x.Oauth2ClientCredentialsGrant
+	}
+	return nil
+}
+
+// XXX_OneofFuncs is for the internal use of the proto package.
+func (*ApiServiceDescriptor) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) error, func(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error), func(msg proto.Message) (n int), []interface{}) {
+	return _ApiServiceDescriptor_OneofMarshaler, _ApiServiceDescriptor_OneofUnmarshaler, _ApiServiceDescriptor_OneofSizer, []interface{}{
+		(*ApiServiceDescriptor_Oauth2ClientCredentialsGrant)(nil),
+	}
+}
+
+func _ApiServiceDescriptor_OneofMarshaler(msg proto.Message, b *proto.Buffer) error {
+	m := msg.(*ApiServiceDescriptor)
+	// authentication
+	switch x := m.Authentication.(type) {
+	case *ApiServiceDescriptor_Oauth2ClientCredentialsGrant:
+		b.EncodeVarint(3<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.Oauth2ClientCredentialsGrant); err != nil {
+			return err
+		}
+	case nil:
+	default:
+		return fmt.Errorf("ApiServiceDescriptor.Authentication has unexpected type %T", x)
+	}
+	return nil
+}
+
+func _ApiServiceDescriptor_OneofUnmarshaler(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error) {
+	m := msg.(*ApiServiceDescriptor)
+	switch tag {
+	case 3: // authentication.oauth2_client_credentials_grant
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(OAuth2ClientCredentialsGrant)
+		err := b.DecodeMessage(msg)
+		m.Authentication = &ApiServiceDescriptor_Oauth2ClientCredentialsGrant{msg}
+		return true, err
+	default:
+		return false, nil
+	}
+}
+
+func _ApiServiceDescriptor_OneofSizer(msg proto.Message) (n int) {
+	m := msg.(*ApiServiceDescriptor)
+	// authentication
+	switch x := m.Authentication.(type) {
+	case *ApiServiceDescriptor_Oauth2ClientCredentialsGrant:
+		s := proto.Size(x.Oauth2ClientCredentialsGrant)
+		n += proto.SizeVarint(3<<3 | proto.WireBytes)
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case nil:
+	default:
+		panic(fmt.Sprintf("proto: unexpected type %T in oneof", x))
+	}
+	return n
+}
+
+type OAuth2ClientCredentialsGrant struct {
+	// (Required) The URL to submit a "client_credentials" grant type request for
+	// an OAuth access token which will be used as a bearer token for requests.
+	Url string `protobuf:"bytes,1,opt,name=url" json:"url,omitempty"`
+}
+
+func (m *OAuth2ClientCredentialsGrant) Reset()                    { *m = OAuth2ClientCredentialsGrant{} }
+func (m *OAuth2ClientCredentialsGrant) String() string            { return proto.CompactTextString(m) }
+func (*OAuth2ClientCredentialsGrant) ProtoMessage()               {}
+func (*OAuth2ClientCredentialsGrant) Descriptor() ([]byte, []int) { return fileDescriptor1, []int{1} }
+
+func (m *OAuth2ClientCredentialsGrant) GetUrl() string {
+	if m != nil {
+		return m.Url
+	}
+	return ""
+}
+
+func init() {
+	proto.RegisterType((*ApiServiceDescriptor)(nil), "org.apache.beam.model.pipeline.v1.ApiServiceDescriptor")
+	proto.RegisterType((*OAuth2ClientCredentialsGrant)(nil), "org.apache.beam.model.pipeline.v1.OAuth2ClientCredentialsGrant")
+}
+
+func init() { proto.RegisterFile("endpoints.proto", fileDescriptor1) }
+
+var fileDescriptor1 = []byte{
+	// 235 bytes of a gzipped FileDescriptorProto
+	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x84, 0x90, 0xb1, 0x4a, 0x03, 0x41,
+	0x10, 0x86, 0x5d, 0x03, 0x42, 0x36, 0xa0, 0xe1, 0xb0, 0x48, 0x11, 0x30, 0xa6, 0x4a, 0xb5, 0x98,
+	0x58, 0x5a, 0x48, 0x2e, 0x8a, 0x76, 0x42, 0xec, 0x6c, 0x96, 0xcd, 0xde, 0x90, 0x0c, 0x6c, 0x76,
+	0x96, 0xb9, 0xc9, 0x3d, 0x83, 0x2f, 0xe6, 0x7b, 0xc9, 0x46, 0x4e, 0x1b, 0xc9, 0x75, 0xc3, 0xfc,
+	0xf0, 0xfd, 0xfc, 0x9f, 0xbe, 0x82, 0x58, 0x25, 0xc2, 0x28, 0xb5, 0x49, 0x4c, 0x42, 0xc5, 0x2d,
+	0xf1, 0xd6, 0xb8, 0xe4, 0xfc, 0x0e, 0xcc, 0x06, 0xdc, 0xde, 0xec, 0xa9, 0x82, 0x60, 0x12, 0x26,
+	0x08, 0x18, 0xc1, 0x34, 0xf3, 0xe9, 0x97, 0xd2, 0xd7, 0xcb, 0x84, 0xef, 0xc0, 0x0d, 0x7a, 0x78,
+	0x82, 0xda, 0x33, 0x26, 0x21, 0x2e, 0x86, 0xba, 0x77, 0xe0, 0x30, 0x3a, 0x9f, 0xa8, 0x59, 0x7f,
+	0x9d, 0xcf, 0xe2, 0x53, 0xe9, 0x1b, 0x72, 0x07, 0xd9, 0x2d, 0xac, 0x0f, 0x08, 0x51, 0xac, 0x67,
+	0xa8, 0x20, 0x0a, 0xba, 0x50, 0xdb, 0x2d, 0xbb, 0x28, 0xa3, 0xde, 0x44, 0xcd, 0x06, 0x8b, 0x47,
+	0xd3, 0x59, 0x6c, 0xde, 0x96, 0x99, 0xb4, 0x3a, 0x82, 0x56, 0x7f, 0x9c, 0x97, 0x8c, 0x79, 0x3d,
+	0x5b, 0x8f, 0x7f, 0x9a, 0xfe, 0xcf, 0xcb, 0xa1, 0xbe, 0xcc, 0x71, 0xfe, 0x79, 0x27, 0x48, 0x71,
+	0x7a, 0xa7, 0xc7, 0xa7, 0x88, 0xed, 0x1c, 0xf5, 0x3b, 0xa7, 0x7c, 0xd0, 0xdd, 0x7a, 0xca, 0xfe,
+	0x73, 0xab, 0xf4, 0x63, 0xd0, 0xfe, 0x6d, 0x33, 0xdf, 0x5c, 0x1c, 0x05, 0xdf, 0x7f, 0x07, 0x00,
+	0x00, 0xff, 0xff, 0x98, 0xdf, 0xf8, 0x2f, 0x73, 0x01, 0x00, 0x00,
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/42100456/sdks/go/pkg/beam/model/pipeline_v1/standard_window_fns.pb.go
----------------------------------------------------------------------
diff --git a/sdks/go/pkg/beam/model/pipeline_v1/standard_window_fns.pb.go b/sdks/go/pkg/beam/model/pipeline_v1/standard_window_fns.pb.go
new file mode 100644
index 0000000..082724d
--- /dev/null
+++ b/sdks/go/pkg/beam/model/pipeline_v1/standard_window_fns.pb.go
@@ -0,0 +1,120 @@
+// Code generated by protoc-gen-go. DO NOT EDIT.
+// source: standard_window_fns.proto
+
+package pipeline_v1
+
+import proto "github.com/golang/protobuf/proto"
+import fmt "fmt"
+import math "math"
+import google_protobuf1 "github.com/golang/protobuf/ptypes/duration"
+import google_protobuf2 "github.com/golang/protobuf/ptypes/timestamp"
+
+// Reference imports to suppress errors if they are not otherwise used.
+var _ = proto.Marshal
+var _ = fmt.Errorf
+var _ = math.Inf
+
+// beam:windowfn:fixed_windows:v0.1
+type FixedWindowsPayload struct {
+	Size   *google_protobuf1.Duration  `protobuf:"bytes,1,opt,name=size" json:"size,omitempty"`
+	Offset *google_protobuf2.Timestamp `protobuf:"bytes,2,opt,name=offset" json:"offset,omitempty"`
+}
+
+func (m *FixedWindowsPayload) Reset()                    { *m = FixedWindowsPayload{} }
+func (m *FixedWindowsPayload) String() string            { return proto.CompactTextString(m) }
+func (*FixedWindowsPayload) ProtoMessage()               {}
+func (*FixedWindowsPayload) Descriptor() ([]byte, []int) { return fileDescriptor2, []int{0} }
+
+func (m *FixedWindowsPayload) GetSize() *google_protobuf1.Duration {
+	if m != nil {
+		return m.Size
+	}
+	return nil
+}
+
+func (m *FixedWindowsPayload) GetOffset() *google_protobuf2.Timestamp {
+	if m != nil {
+		return m.Offset
+	}
+	return nil
+}
+
+// beam:windowfn:sliding_windows:v0.1
+type SlidingWindowsPayload struct {
+	Size   *google_protobuf1.Duration  `protobuf:"bytes,1,opt,name=size" json:"size,omitempty"`
+	Offset *google_protobuf2.Timestamp `protobuf:"bytes,2,opt,name=offset" json:"offset,omitempty"`
+	Period *google_protobuf1.Duration  `protobuf:"bytes,3,opt,name=period" json:"period,omitempty"`
+}
+
+func (m *SlidingWindowsPayload) Reset()                    { *m = SlidingWindowsPayload{} }
+func (m *SlidingWindowsPayload) String() string            { return proto.CompactTextString(m) }
+func (*SlidingWindowsPayload) ProtoMessage()               {}
+func (*SlidingWindowsPayload) Descriptor() ([]byte, []int) { return fileDescriptor2, []int{1} }
+
+func (m *SlidingWindowsPayload) GetSize() *google_protobuf1.Duration {
+	if m != nil {
+		return m.Size
+	}
+	return nil
+}
+
+func (m *SlidingWindowsPayload) GetOffset() *google_protobuf2.Timestamp {
+	if m != nil {
+		return m.Offset
+	}
+	return nil
+}
+
+func (m *SlidingWindowsPayload) GetPeriod() *google_protobuf1.Duration {
+	if m != nil {
+		return m.Period
+	}
+	return nil
+}
+
+// beam:windowfn:session_windows:v0.1
+type SessionsPayload struct {
+	GapSize *google_protobuf1.Duration `protobuf:"bytes,1,opt,name=gap_size,json=gapSize" json:"gap_size,omitempty"`
+}
+
+func (m *SessionsPayload) Reset()                    { *m = SessionsPayload{} }
+func (m *SessionsPayload) String() string            { return proto.CompactTextString(m) }
+func (*SessionsPayload) ProtoMessage()               {}
+func (*SessionsPayload) Descriptor() ([]byte, []int) { return fileDescriptor2, []int{2} }
+
+func (m *SessionsPayload) GetGapSize() *google_protobuf1.Duration {
+	if m != nil {
+		return m.GapSize
+	}
+	return nil
+}
+
+func init() {
+	proto.RegisterType((*FixedWindowsPayload)(nil), "org.apache.beam.model.pipeline.v1.FixedWindowsPayload")
+	proto.RegisterType((*SlidingWindowsPayload)(nil), "org.apache.beam.model.pipeline.v1.SlidingWindowsPayload")
+	proto.RegisterType((*SessionsPayload)(nil), "org.apache.beam.model.pipeline.v1.SessionsPayload")
+}
+
+func init() { proto.RegisterFile("standard_window_fns.proto", fileDescriptor2) }
+
+var fileDescriptor2 = []byte{
+	// 277 bytes of a gzipped FileDescriptorProto
+	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x92, 0x4d, 0x4b, 0xc3, 0x40,
+	0x10, 0x86, 0xa9, 0x4a, 0x95, 0xed, 0x41, 0x8c, 0x08, 0x69, 0x0e, 0x7e, 0xe4, 0xe4, 0xc5, 0x2d,
+	0xa9, 0xfe, 0x82, 0x2a, 0xf5, 0x2a, 0x8d, 0x20, 0x78, 0x09, 0x13, 0x77, 0xb2, 0x0e, 0x24, 0x3b,
+	0x4b, 0x76, 0xfb, 0x61, 0xff, 0x93, 0xff, 0x51, 0xc8, 0x87, 0x07, 0x3d, 0xd4, 0x93, 0xe7, 0x79,
+	0xde, 0xf7, 0x19, 0x76, 0x47, 0x8c, 0x9d, 0x07, 0xa3, 0xa0, 0x56, 0xd9, 0x9a, 0x8c, 0xe2, 0x75,
+	0x56, 0x18, 0x27, 0x6d, 0xcd, 0x9e, 0x83, 0x2b, 0xae, 0xb5, 0x04, 0x0b, 0x6f, 0xef, 0x28, 0x73,
+	0x84, 0x4a, 0x56, 0xac, 0xb0, 0x94, 0x96, 0x2c, 0x96, 0x64, 0x50, 0xae, 0x92, 0xe8, 0x5c, 0x33,
+	0xeb, 0x12, 0x27, 0x4d, 0x20, 0x5f, 0x16, 0x13, 0xb5, 0xac, 0xc1, 0x13, 0x9b, 0xb6, 0x22, 0xba,
+	0xf8, 0x39, 0xf7, 0x54, 0xa1, 0xf3, 0x50, 0xd9, 0x16, 0x88, 0x37, 0xe2, 0x74, 0x4e, 0x1b, 0x54,
+	0x2f, 0x8d, 0xdc, 0x3d, 0xc1, 0x47, 0xc9, 0xa0, 0x82, 0x1b, 0x71, 0xe0, 0x68, 0x8b, 0xe1, 0xe0,
+	0x72, 0x70, 0x3d, 0x9a, 0x8e, 0x65, 0x5b, 0x23, 0xfb, 0x1a, 0xf9, 0xd0, 0x69, 0x16, 0x0d, 0x16,
+	0x4c, 0xc5, 0x90, 0x8b, 0xc2, 0xa1, 0x0f, 0xf7, 0x9a, 0x40, 0xf4, 0x2b, 0xf0, 0xdc, 0x7b, 0x17,
+	0x1d, 0x19, 0x7f, 0x0e, 0xc4, 0x59, 0x5a, 0x92, 0x22, 0xa3, 0xff, 0x5d, 0x1e, 0x24, 0x62, 0x68,
+	0xb1, 0x26, 0x56, 0xe1, 0xfe, 0x2e, 0x49, 0x07, 0xc6, 0x8f, 0xe2, 0x38, 0x45, 0xe7, 0x88, 0xcd,
+	0xf7, 0xa2, 0x77, 0xe2, 0x48, 0x83, 0xcd, 0xfe, 0xb6, 0xec, 0xa1, 0x06, 0x9b, 0xd2, 0x16, 0x67,
+	0xf7, 0x62, 0xf7, 0xc7, 0xce, 0x4e, 0xd2, 0xee, 0x2c, 0xda, 0xb7, 0x99, 0x1b, 0xf7, 0x3a, 0xea,
+	0xe7, 0xd9, 0x2a, 0xc9, 0x87, 0x8d, 0xe0, 0xf6, 0x2b, 0x00, 0x00, 0xff, 0xff, 0x82, 0x58, 0x88,
+	0x91, 0x3f, 0x02, 0x00, 0x00,
+}


[11/50] [abbrv] beam git commit: This closes #4069: Fix working dir in website precommits

Posted by ke...@apache.org.
This closes #4069: Fix working dir in website precommits


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

Branch: refs/heads/mr-runner
Commit: a33c717dbb9b25485218b135a78b5cdb3f8239c9
Parents: 1f37c29 7f9cfdc
Author: Kenneth Knowles <ke...@apache.org>
Authored: Wed Nov 1 14:47:42 2017 -0700
Committer: Kenneth Knowles <ke...@apache.org>
Committed: Wed Nov 1 14:47:42 2017 -0700

----------------------------------------------------------------------
 .test-infra/jenkins/job_beam_PreCommit_Website_Merge.groovy | 3 +++
 .test-infra/jenkins/job_beam_PreCommit_Website_Stage.groovy | 3 +++
 .test-infra/jenkins/job_beam_PreCommit_Website_Test.groovy  | 3 +++
 3 files changed, 9 insertions(+)
----------------------------------------------------------------------



[40/50] [abbrv] beam git commit: This closes #4066

Posted by ke...@apache.org.
This closes #4066


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

Branch: refs/heads/mr-runner
Commit: 6e2889226bf41a2085280a79b430042a80f69e9c
Parents: 7550a58 bd1586f
Author: Thomas Groh <tg...@google.com>
Authored: Mon Nov 6 16:48:08 2017 -0800
Committer: Thomas Groh <tg...@google.com>
Committed: Mon Nov 6 16:48:08 2017 -0800

----------------------------------------------------------------------
 runners/direct-java/pom.xml | 18 ++----------------
 1 file changed, 2 insertions(+), 16 deletions(-)
----------------------------------------------------------------------



[31/50] [abbrv] beam git commit: Add all portability protos to Go

Posted by ke...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/42100456/sdks/go/pkg/beam/model/pipeline_v1/beam_runner_api.pb.go
----------------------------------------------------------------------
diff --git a/sdks/go/pkg/beam/model/pipeline_v1/beam_runner_api.pb.go b/sdks/go/pkg/beam/model/pipeline_v1/beam_runner_api.pb.go
new file mode 100644
index 0000000..31dc53e
--- /dev/null
+++ b/sdks/go/pkg/beam/model/pipeline_v1/beam_runner_api.pb.go
@@ -0,0 +1,3491 @@
+// Code generated by protoc-gen-go. DO NOT EDIT.
+// source: beam_runner_api.proto
+
+/*
+Package pipeline_v1 is a generated protocol buffer package.
+
+It is generated from these files:
+	beam_runner_api.proto
+	endpoints.proto
+	standard_window_fns.proto
+
+It has these top-level messages:
+	Components
+	MessageWithComponents
+	Pipeline
+	PTransform
+	PCollection
+	ParDoPayload
+	Parameter
+	StateSpec
+	ValueStateSpec
+	BagStateSpec
+	CombiningStateSpec
+	MapStateSpec
+	SetStateSpec
+	TimerSpec
+	IsBounded
+	ReadPayload
+	WindowIntoPayload
+	CombinePayload
+	TestStreamPayload
+	WriteFilesPayload
+	Coder
+	WindowingStrategy
+	MergeStatus
+	AccumulationMode
+	ClosingBehavior
+	OnTimeBehavior
+	OutputTime
+	TimeDomain
+	Trigger
+	TimestampTransform
+	SideInput
+	Environment
+	SdkFunctionSpec
+	FunctionSpec
+	DisplayData
+	ApiServiceDescriptor
+	OAuth2ClientCredentialsGrant
+	FixedWindowsPayload
+	SlidingWindowsPayload
+	SessionsPayload
+*/
+package pipeline_v1
+
+import proto "github.com/golang/protobuf/proto"
+import fmt "fmt"
+import math "math"
+import google_protobuf "github.com/golang/protobuf/ptypes/any"
+
+// Reference imports to suppress errors if they are not otherwise used.
+var _ = proto.Marshal
+var _ = fmt.Errorf
+var _ = math.Inf
+
+// This is a compile-time assertion to ensure that this generated file
+// is compatible with the proto package it is being compiled against.
+// A compilation error at this line likely means your copy of the
+// proto package needs to be updated.
+const _ = proto.ProtoPackageIsVersion2 // please upgrade the proto package
+
+type Parameter_Type_Enum int32
+
+const (
+	Parameter_Type_UNSPECIFIED         Parameter_Type_Enum = 0
+	Parameter_Type_WINDOW              Parameter_Type_Enum = 1
+	Parameter_Type_PIPELINE_OPTIONS    Parameter_Type_Enum = 2
+	Parameter_Type_RESTRICTION_TRACKER Parameter_Type_Enum = 3
+)
+
+var Parameter_Type_Enum_name = map[int32]string{
+	0: "UNSPECIFIED",
+	1: "WINDOW",
+	2: "PIPELINE_OPTIONS",
+	3: "RESTRICTION_TRACKER",
+}
+var Parameter_Type_Enum_value = map[string]int32{
+	"UNSPECIFIED":         0,
+	"WINDOW":              1,
+	"PIPELINE_OPTIONS":    2,
+	"RESTRICTION_TRACKER": 3,
+}
+
+func (x Parameter_Type_Enum) String() string {
+	return proto.EnumName(Parameter_Type_Enum_name, int32(x))
+}
+func (Parameter_Type_Enum) EnumDescriptor() ([]byte, []int) { return fileDescriptor0, []int{6, 0, 0} }
+
+type IsBounded_Enum int32
+
+const (
+	IsBounded_UNSPECIFIED IsBounded_Enum = 0
+	IsBounded_UNBOUNDED   IsBounded_Enum = 1
+	IsBounded_BOUNDED     IsBounded_Enum = 2
+)
+
+var IsBounded_Enum_name = map[int32]string{
+	0: "UNSPECIFIED",
+	1: "UNBOUNDED",
+	2: "BOUNDED",
+}
+var IsBounded_Enum_value = map[string]int32{
+	"UNSPECIFIED": 0,
+	"UNBOUNDED":   1,
+	"BOUNDED":     2,
+}
+
+func (x IsBounded_Enum) String() string {
+	return proto.EnumName(IsBounded_Enum_name, int32(x))
+}
+func (IsBounded_Enum) EnumDescriptor() ([]byte, []int) { return fileDescriptor0, []int{14, 0} }
+
+type MergeStatus_Enum int32
+
+const (
+	MergeStatus_UNSPECIFIED MergeStatus_Enum = 0
+	// The WindowFn does not require merging.
+	// Examples: global window, FixedWindows, SlidingWindows
+	MergeStatus_NON_MERGING MergeStatus_Enum = 1
+	// The WindowFn is merging and the PCollection has not had merging
+	// performed.
+	// Example: Sessions prior to a GroupByKey
+	MergeStatus_NEEDS_MERGE MergeStatus_Enum = 2
+	// The WindowFn is merging and the PCollection has had merging occur
+	// already.
+	// Example: Sessions after a GroupByKey
+	MergeStatus_ALREADY_MERGED MergeStatus_Enum = 3
+)
+
+var MergeStatus_Enum_name = map[int32]string{
+	0: "UNSPECIFIED",
+	1: "NON_MERGING",
+	2: "NEEDS_MERGE",
+	3: "ALREADY_MERGED",
+}
+var MergeStatus_Enum_value = map[string]int32{
+	"UNSPECIFIED":    0,
+	"NON_MERGING":    1,
+	"NEEDS_MERGE":    2,
+	"ALREADY_MERGED": 3,
+}
+
+func (x MergeStatus_Enum) String() string {
+	return proto.EnumName(MergeStatus_Enum_name, int32(x))
+}
+func (MergeStatus_Enum) EnumDescriptor() ([]byte, []int) { return fileDescriptor0, []int{22, 0} }
+
+type AccumulationMode_Enum int32
+
+const (
+	AccumulationMode_UNSPECIFIED AccumulationMode_Enum = 0
+	// The aggregation is discarded when it is output
+	AccumulationMode_DISCARDING AccumulationMode_Enum = 1
+	// The aggregation is accumulated across outputs
+	AccumulationMode_ACCUMULATING AccumulationMode_Enum = 2
+)
+
+var AccumulationMode_Enum_name = map[int32]string{
+	0: "UNSPECIFIED",
+	1: "DISCARDING",
+	2: "ACCUMULATING",
+}
+var AccumulationMode_Enum_value = map[string]int32{
+	"UNSPECIFIED":  0,
+	"DISCARDING":   1,
+	"ACCUMULATING": 2,
+}
+
+func (x AccumulationMode_Enum) String() string {
+	return proto.EnumName(AccumulationMode_Enum_name, int32(x))
+}
+func (AccumulationMode_Enum) EnumDescriptor() ([]byte, []int) { return fileDescriptor0, []int{23, 0} }
+
+type ClosingBehavior_Enum int32
+
+const (
+	ClosingBehavior_UNSPECIFIED ClosingBehavior_Enum = 0
+	// Emit output when a window expires, whether or not there has been
+	// any new data since the last output.
+	ClosingBehavior_EMIT_ALWAYS ClosingBehavior_Enum = 1
+	// Only emit output when new data has arrives since the last output
+	ClosingBehavior_EMIT_IF_NONEMPTY ClosingBehavior_Enum = 2
+)
+
+var ClosingBehavior_Enum_name = map[int32]string{
+	0: "UNSPECIFIED",
+	1: "EMIT_ALWAYS",
+	2: "EMIT_IF_NONEMPTY",
+}
+var ClosingBehavior_Enum_value = map[string]int32{
+	"UNSPECIFIED":      0,
+	"EMIT_ALWAYS":      1,
+	"EMIT_IF_NONEMPTY": 2,
+}
+
+func (x ClosingBehavior_Enum) String() string {
+	return proto.EnumName(ClosingBehavior_Enum_name, int32(x))
+}
+func (ClosingBehavior_Enum) EnumDescriptor() ([]byte, []int) { return fileDescriptor0, []int{24, 0} }
+
+type OnTimeBehavior_Enum int32
+
+const (
+	OnTimeBehavior_UNSPECIFIED OnTimeBehavior_Enum = 0
+	// Always fire the on-time pane. Even if there is no new data since
+	// the previous firing, an element will be produced.
+	OnTimeBehavior_FIRE_ALWAYS OnTimeBehavior_Enum = 1
+	// Only fire the on-time pane if there is new data since the previous firing.
+	OnTimeBehavior_FIRE_IF_NONEMPTY OnTimeBehavior_Enum = 2
+)
+
+var OnTimeBehavior_Enum_name = map[int32]string{
+	0: "UNSPECIFIED",
+	1: "FIRE_ALWAYS",
+	2: "FIRE_IF_NONEMPTY",
+}
+var OnTimeBehavior_Enum_value = map[string]int32{
+	"UNSPECIFIED":      0,
+	"FIRE_ALWAYS":      1,
+	"FIRE_IF_NONEMPTY": 2,
+}
+
+func (x OnTimeBehavior_Enum) String() string {
+	return proto.EnumName(OnTimeBehavior_Enum_name, int32(x))
+}
+func (OnTimeBehavior_Enum) EnumDescriptor() ([]byte, []int) { return fileDescriptor0, []int{25, 0} }
+
+type OutputTime_Enum int32
+
+const (
+	OutputTime_UNSPECIFIED OutputTime_Enum = 0
+	// The output has the timestamp of the end of the window.
+	OutputTime_END_OF_WINDOW OutputTime_Enum = 1
+	// The output has the latest timestamp of the input elements since
+	// the last output.
+	OutputTime_LATEST_IN_PANE OutputTime_Enum = 2
+	// The output has the earliest timestamp of the input elements since
+	// the last output.
+	OutputTime_EARLIEST_IN_PANE OutputTime_Enum = 3
+)
+
+var OutputTime_Enum_name = map[int32]string{
+	0: "UNSPECIFIED",
+	1: "END_OF_WINDOW",
+	2: "LATEST_IN_PANE",
+	3: "EARLIEST_IN_PANE",
+}
+var OutputTime_Enum_value = map[string]int32{
+	"UNSPECIFIED":      0,
+	"END_OF_WINDOW":    1,
+	"LATEST_IN_PANE":   2,
+	"EARLIEST_IN_PANE": 3,
+}
+
+func (x OutputTime_Enum) String() string {
+	return proto.EnumName(OutputTime_Enum_name, int32(x))
+}
+func (OutputTime_Enum) EnumDescriptor() ([]byte, []int) { return fileDescriptor0, []int{26, 0} }
+
+type TimeDomain_Enum int32
+
+const (
+	TimeDomain_UNSPECIFIED TimeDomain_Enum = 0
+	// Event time is time from the perspective of the data
+	TimeDomain_EVENT_TIME TimeDomain_Enum = 1
+	// Processing time is time from the perspective of the
+	// execution of your pipeline
+	TimeDomain_PROCESSING_TIME TimeDomain_Enum = 2
+	// Synchronized processing time is the minimum of the
+	// processing time of all pending elements.
+	//
+	// The "processing time" of an element refers to
+	// the local processing time at which it was emitted
+	TimeDomain_SYNCHRONIZED_PROCESSING_TIME TimeDomain_Enum = 3
+)
+
+var TimeDomain_Enum_name = map[int32]string{
+	0: "UNSPECIFIED",
+	1: "EVENT_TIME",
+	2: "PROCESSING_TIME",
+	3: "SYNCHRONIZED_PROCESSING_TIME",
+}
+var TimeDomain_Enum_value = map[string]int32{
+	"UNSPECIFIED":                  0,
+	"EVENT_TIME":                   1,
+	"PROCESSING_TIME":              2,
+	"SYNCHRONIZED_PROCESSING_TIME": 3,
+}
+
+func (x TimeDomain_Enum) String() string {
+	return proto.EnumName(TimeDomain_Enum_name, int32(x))
+}
+func (TimeDomain_Enum) EnumDescriptor() ([]byte, []int) { return fileDescriptor0, []int{27, 0} }
+
+type DisplayData_Type_Enum int32
+
+const (
+	DisplayData_Type_UNSPECIFIED DisplayData_Type_Enum = 0
+	DisplayData_Type_STRING      DisplayData_Type_Enum = 1
+	DisplayData_Type_INTEGER     DisplayData_Type_Enum = 2
+	DisplayData_Type_FLOAT       DisplayData_Type_Enum = 3
+	DisplayData_Type_BOOLEAN     DisplayData_Type_Enum = 4
+	DisplayData_Type_TIMESTAMP   DisplayData_Type_Enum = 5
+	DisplayData_Type_DURATION    DisplayData_Type_Enum = 6
+	DisplayData_Type_JAVA_CLASS  DisplayData_Type_Enum = 7
+)
+
+var DisplayData_Type_Enum_name = map[int32]string{
+	0: "UNSPECIFIED",
+	1: "STRING",
+	2: "INTEGER",
+	3: "FLOAT",
+	4: "BOOLEAN",
+	5: "TIMESTAMP",
+	6: "DURATION",
+	7: "JAVA_CLASS",
+}
+var DisplayData_Type_Enum_value = map[string]int32{
+	"UNSPECIFIED": 0,
+	"STRING":      1,
+	"INTEGER":     2,
+	"FLOAT":       3,
+	"BOOLEAN":     4,
+	"TIMESTAMP":   5,
+	"DURATION":    6,
+	"JAVA_CLASS":  7,
+}
+
+func (x DisplayData_Type_Enum) String() string {
+	return proto.EnumName(DisplayData_Type_Enum_name, int32(x))
+}
+func (DisplayData_Type_Enum) EnumDescriptor() ([]byte, []int) { return fileDescriptor0, []int{34, 2, 0} }
+
+// A set of mappings from id to message. This is included as an optional field
+// on any proto message that may contain references needing resolution.
+type Components struct {
+	// (Required) A map from pipeline-scoped id to PTransform.
+	Transforms map[string]*PTransform `protobuf:"bytes,1,rep,name=transforms" json:"transforms,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"`
+	// (Required) A map from pipeline-scoped id to PCollection.
+	Pcollections map[string]*PCollection `protobuf:"bytes,2,rep,name=pcollections" json:"pcollections,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"`
+	// (Required) A map from pipeline-scoped id to WindowingStrategy.
+	WindowingStrategies map[string]*WindowingStrategy `protobuf:"bytes,3,rep,name=windowing_strategies,json=windowingStrategies" json:"windowing_strategies,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"`
+	// (Required) A map from pipeline-scoped id to Coder.
+	Coders map[string]*Coder `protobuf:"bytes,4,rep,name=coders" json:"coders,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"`
+	// (Required) A map from pipeline-scoped id to Environment.
+	Environments map[string]*Environment `protobuf:"bytes,5,rep,name=environments" json:"environments,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"`
+}
+
+func (m *Components) Reset()                    { *m = Components{} }
+func (m *Components) String() string            { return proto.CompactTextString(m) }
+func (*Components) ProtoMessage()               {}
+func (*Components) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{0} }
+
+func (m *Components) GetTransforms() map[string]*PTransform {
+	if m != nil {
+		return m.Transforms
+	}
+	return nil
+}
+
+func (m *Components) GetPcollections() map[string]*PCollection {
+	if m != nil {
+		return m.Pcollections
+	}
+	return nil
+}
+
+func (m *Components) GetWindowingStrategies() map[string]*WindowingStrategy {
+	if m != nil {
+		return m.WindowingStrategies
+	}
+	return nil
+}
+
+func (m *Components) GetCoders() map[string]*Coder {
+	if m != nil {
+		return m.Coders
+	}
+	return nil
+}
+
+func (m *Components) GetEnvironments() map[string]*Environment {
+	if m != nil {
+		return m.Environments
+	}
+	return nil
+}
+
+// A disjoint union of all the things that may contain references
+// that require Components to resolve.
+type MessageWithComponents struct {
+	// (Optional) The by-reference components of the root message,
+	// enabling a standalone message.
+	//
+	// If this is absent, it is expected that there are no
+	// references.
+	Components *Components `protobuf:"bytes,1,opt,name=components" json:"components,omitempty"`
+	// (Required) The root message that may contain pointers
+	// that should be resolved by looking inside components.
+	//
+	// Types that are valid to be assigned to Root:
+	//	*MessageWithComponents_Coder
+	//	*MessageWithComponents_CombinePayload
+	//	*MessageWithComponents_SdkFunctionSpec
+	//	*MessageWithComponents_ParDoPayload
+	//	*MessageWithComponents_Ptransform
+	//	*MessageWithComponents_Pcollection
+	//	*MessageWithComponents_ReadPayload
+	//	*MessageWithComponents_SideInput
+	//	*MessageWithComponents_WindowIntoPayload
+	//	*MessageWithComponents_WindowingStrategy
+	//	*MessageWithComponents_FunctionSpec
+	Root isMessageWithComponents_Root `protobuf_oneof:"root"`
+}
+
+func (m *MessageWithComponents) Reset()                    { *m = MessageWithComponents{} }
+func (m *MessageWithComponents) String() string            { return proto.CompactTextString(m) }
+func (*MessageWithComponents) ProtoMessage()               {}
+func (*MessageWithComponents) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{1} }
+
+type isMessageWithComponents_Root interface {
+	isMessageWithComponents_Root()
+}
+
+type MessageWithComponents_Coder struct {
+	Coder *Coder `protobuf:"bytes,2,opt,name=coder,oneof"`
+}
+type MessageWithComponents_CombinePayload struct {
+	CombinePayload *CombinePayload `protobuf:"bytes,3,opt,name=combine_payload,json=combinePayload,oneof"`
+}
+type MessageWithComponents_SdkFunctionSpec struct {
+	SdkFunctionSpec *SdkFunctionSpec `protobuf:"bytes,4,opt,name=sdk_function_spec,json=sdkFunctionSpec,oneof"`
+}
+type MessageWithComponents_ParDoPayload struct {
+	ParDoPayload *ParDoPayload `protobuf:"bytes,6,opt,name=par_do_payload,json=parDoPayload,oneof"`
+}
+type MessageWithComponents_Ptransform struct {
+	Ptransform *PTransform `protobuf:"bytes,7,opt,name=ptransform,oneof"`
+}
+type MessageWithComponents_Pcollection struct {
+	Pcollection *PCollection `protobuf:"bytes,8,opt,name=pcollection,oneof"`
+}
+type MessageWithComponents_ReadPayload struct {
+	ReadPayload *ReadPayload `protobuf:"bytes,9,opt,name=read_payload,json=readPayload,oneof"`
+}
+type MessageWithComponents_SideInput struct {
+	SideInput *SideInput `protobuf:"bytes,11,opt,name=side_input,json=sideInput,oneof"`
+}
+type MessageWithComponents_WindowIntoPayload struct {
+	WindowIntoPayload *WindowIntoPayload `protobuf:"bytes,12,opt,name=window_into_payload,json=windowIntoPayload,oneof"`
+}
+type MessageWithComponents_WindowingStrategy struct {
+	WindowingStrategy *WindowingStrategy `protobuf:"bytes,13,opt,name=windowing_strategy,json=windowingStrategy,oneof"`
+}
+type MessageWithComponents_FunctionSpec struct {
+	FunctionSpec *FunctionSpec `protobuf:"bytes,14,opt,name=function_spec,json=functionSpec,oneof"`
+}
+
+func (*MessageWithComponents_Coder) isMessageWithComponents_Root()             {}
+func (*MessageWithComponents_CombinePayload) isMessageWithComponents_Root()    {}
+func (*MessageWithComponents_SdkFunctionSpec) isMessageWithComponents_Root()   {}
+func (*MessageWithComponents_ParDoPayload) isMessageWithComponents_Root()      {}
+func (*MessageWithComponents_Ptransform) isMessageWithComponents_Root()        {}
+func (*MessageWithComponents_Pcollection) isMessageWithComponents_Root()       {}
+func (*MessageWithComponents_ReadPayload) isMessageWithComponents_Root()       {}
+func (*MessageWithComponents_SideInput) isMessageWithComponents_Root()         {}
+func (*MessageWithComponents_WindowIntoPayload) isMessageWithComponents_Root() {}
+func (*MessageWithComponents_WindowingStrategy) isMessageWithComponents_Root() {}
+func (*MessageWithComponents_FunctionSpec) isMessageWithComponents_Root()      {}
+
+func (m *MessageWithComponents) GetRoot() isMessageWithComponents_Root {
+	if m != nil {
+		return m.Root
+	}
+	return nil
+}
+
+func (m *MessageWithComponents) GetComponents() *Components {
+	if m != nil {
+		return m.Components
+	}
+	return nil
+}
+
+func (m *MessageWithComponents) GetCoder() *Coder {
+	if x, ok := m.GetRoot().(*MessageWithComponents_Coder); ok {
+		return x.Coder
+	}
+	return nil
+}
+
+func (m *MessageWithComponents) GetCombinePayload() *CombinePayload {
+	if x, ok := m.GetRoot().(*MessageWithComponents_CombinePayload); ok {
+		return x.CombinePayload
+	}
+	return nil
+}
+
+func (m *MessageWithComponents) GetSdkFunctionSpec() *SdkFunctionSpec {
+	if x, ok := m.GetRoot().(*MessageWithComponents_SdkFunctionSpec); ok {
+		return x.SdkFunctionSpec
+	}
+	return nil
+}
+
+func (m *MessageWithComponents) GetParDoPayload() *ParDoPayload {
+	if x, ok := m.GetRoot().(*MessageWithComponents_ParDoPayload); ok {
+		return x.ParDoPayload
+	}
+	return nil
+}
+
+func (m *MessageWithComponents) GetPtransform() *PTransform {
+	if x, ok := m.GetRoot().(*MessageWithComponents_Ptransform); ok {
+		return x.Ptransform
+	}
+	return nil
+}
+
+func (m *MessageWithComponents) GetPcollection() *PCollection {
+	if x, ok := m.GetRoot().(*MessageWithComponents_Pcollection); ok {
+		return x.Pcollection
+	}
+	return nil
+}
+
+func (m *MessageWithComponents) GetReadPayload() *ReadPayload {
+	if x, ok := m.GetRoot().(*MessageWithComponents_ReadPayload); ok {
+		return x.ReadPayload
+	}
+	return nil
+}
+
+func (m *MessageWithComponents) GetSideInput() *SideInput {
+	if x, ok := m.GetRoot().(*MessageWithComponents_SideInput); ok {
+		return x.SideInput
+	}
+	return nil
+}
+
+func (m *MessageWithComponents) GetWindowIntoPayload() *WindowIntoPayload {
+	if x, ok := m.GetRoot().(*MessageWithComponents_WindowIntoPayload); ok {
+		return x.WindowIntoPayload
+	}
+	return nil
+}
+
+func (m *MessageWithComponents) GetWindowingStrategy() *WindowingStrategy {
+	if x, ok := m.GetRoot().(*MessageWithComponents_WindowingStrategy); ok {
+		return x.WindowingStrategy
+	}
+	return nil
+}
+
+func (m *MessageWithComponents) GetFunctionSpec() *FunctionSpec {
+	if x, ok := m.GetRoot().(*MessageWithComponents_FunctionSpec); ok {
+		return x.FunctionSpec
+	}
+	return nil
+}
+
+// XXX_OneofFuncs is for the internal use of the proto package.
+func (*MessageWithComponents) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) error, func(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error), func(msg proto.Message) (n int), []interface{}) {
+	return _MessageWithComponents_OneofMarshaler, _MessageWithComponents_OneofUnmarshaler, _MessageWithComponents_OneofSizer, []interface{}{
+		(*MessageWithComponents_Coder)(nil),
+		(*MessageWithComponents_CombinePayload)(nil),
+		(*MessageWithComponents_SdkFunctionSpec)(nil),
+		(*MessageWithComponents_ParDoPayload)(nil),
+		(*MessageWithComponents_Ptransform)(nil),
+		(*MessageWithComponents_Pcollection)(nil),
+		(*MessageWithComponents_ReadPayload)(nil),
+		(*MessageWithComponents_SideInput)(nil),
+		(*MessageWithComponents_WindowIntoPayload)(nil),
+		(*MessageWithComponents_WindowingStrategy)(nil),
+		(*MessageWithComponents_FunctionSpec)(nil),
+	}
+}
+
+func _MessageWithComponents_OneofMarshaler(msg proto.Message, b *proto.Buffer) error {
+	m := msg.(*MessageWithComponents)
+	// root
+	switch x := m.Root.(type) {
+	case *MessageWithComponents_Coder:
+		b.EncodeVarint(2<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.Coder); err != nil {
+			return err
+		}
+	case *MessageWithComponents_CombinePayload:
+		b.EncodeVarint(3<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.CombinePayload); err != nil {
+			return err
+		}
+	case *MessageWithComponents_SdkFunctionSpec:
+		b.EncodeVarint(4<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.SdkFunctionSpec); err != nil {
+			return err
+		}
+	case *MessageWithComponents_ParDoPayload:
+		b.EncodeVarint(6<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.ParDoPayload); err != nil {
+			return err
+		}
+	case *MessageWithComponents_Ptransform:
+		b.EncodeVarint(7<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.Ptransform); err != nil {
+			return err
+		}
+	case *MessageWithComponents_Pcollection:
+		b.EncodeVarint(8<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.Pcollection); err != nil {
+			return err
+		}
+	case *MessageWithComponents_ReadPayload:
+		b.EncodeVarint(9<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.ReadPayload); err != nil {
+			return err
+		}
+	case *MessageWithComponents_SideInput:
+		b.EncodeVarint(11<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.SideInput); err != nil {
+			return err
+		}
+	case *MessageWithComponents_WindowIntoPayload:
+		b.EncodeVarint(12<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.WindowIntoPayload); err != nil {
+			return err
+		}
+	case *MessageWithComponents_WindowingStrategy:
+		b.EncodeVarint(13<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.WindowingStrategy); err != nil {
+			return err
+		}
+	case *MessageWithComponents_FunctionSpec:
+		b.EncodeVarint(14<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.FunctionSpec); err != nil {
+			return err
+		}
+	case nil:
+	default:
+		return fmt.Errorf("MessageWithComponents.Root has unexpected type %T", x)
+	}
+	return nil
+}
+
+func _MessageWithComponents_OneofUnmarshaler(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error) {
+	m := msg.(*MessageWithComponents)
+	switch tag {
+	case 2: // root.coder
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(Coder)
+		err := b.DecodeMessage(msg)
+		m.Root = &MessageWithComponents_Coder{msg}
+		return true, err
+	case 3: // root.combine_payload
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(CombinePayload)
+		err := b.DecodeMessage(msg)
+		m.Root = &MessageWithComponents_CombinePayload{msg}
+		return true, err
+	case 4: // root.sdk_function_spec
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(SdkFunctionSpec)
+		err := b.DecodeMessage(msg)
+		m.Root = &MessageWithComponents_SdkFunctionSpec{msg}
+		return true, err
+	case 6: // root.par_do_payload
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(ParDoPayload)
+		err := b.DecodeMessage(msg)
+		m.Root = &MessageWithComponents_ParDoPayload{msg}
+		return true, err
+	case 7: // root.ptransform
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(PTransform)
+		err := b.DecodeMessage(msg)
+		m.Root = &MessageWithComponents_Ptransform{msg}
+		return true, err
+	case 8: // root.pcollection
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(PCollection)
+		err := b.DecodeMessage(msg)
+		m.Root = &MessageWithComponents_Pcollection{msg}
+		return true, err
+	case 9: // root.read_payload
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(ReadPayload)
+		err := b.DecodeMessage(msg)
+		m.Root = &MessageWithComponents_ReadPayload{msg}
+		return true, err
+	case 11: // root.side_input
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(SideInput)
+		err := b.DecodeMessage(msg)
+		m.Root = &MessageWithComponents_SideInput{msg}
+		return true, err
+	case 12: // root.window_into_payload
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(WindowIntoPayload)
+		err := b.DecodeMessage(msg)
+		m.Root = &MessageWithComponents_WindowIntoPayload{msg}
+		return true, err
+	case 13: // root.windowing_strategy
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(WindowingStrategy)
+		err := b.DecodeMessage(msg)
+		m.Root = &MessageWithComponents_WindowingStrategy{msg}
+		return true, err
+	case 14: // root.function_spec
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(FunctionSpec)
+		err := b.DecodeMessage(msg)
+		m.Root = &MessageWithComponents_FunctionSpec{msg}
+		return true, err
+	default:
+		return false, nil
+	}
+}
+
+func _MessageWithComponents_OneofSizer(msg proto.Message) (n int) {
+	m := msg.(*MessageWithComponents)
+	// root
+	switch x := m.Root.(type) {
+	case *MessageWithComponents_Coder:
+		s := proto.Size(x.Coder)
+		n += proto.SizeVarint(2<<3 | proto.WireBytes)
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case *MessageWithComponents_CombinePayload:
+		s := proto.Size(x.CombinePayload)
+		n += proto.SizeVarint(3<<3 | proto.WireBytes)
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case *MessageWithComponents_SdkFunctionSpec:
+		s := proto.Size(x.SdkFunctionSpec)
+		n += proto.SizeVarint(4<<3 | proto.WireBytes)
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case *MessageWithComponents_ParDoPayload:
+		s := proto.Size(x.ParDoPayload)
+		n += proto.SizeVarint(6<<3 | proto.WireBytes)
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case *MessageWithComponents_Ptransform:
+		s := proto.Size(x.Ptransform)
+		n += proto.SizeVarint(7<<3 | proto.WireBytes)
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case *MessageWithComponents_Pcollection:
+		s := proto.Size(x.Pcollection)
+		n += proto.SizeVarint(8<<3 | proto.WireBytes)
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case *MessageWithComponents_ReadPayload:
+		s := proto.Size(x.ReadPayload)
+		n += proto.SizeVarint(9<<3 | proto.WireBytes)
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case *MessageWithComponents_SideInput:
+		s := proto.Size(x.SideInput)
+		n += proto.SizeVarint(11<<3 | proto.WireBytes)
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case *MessageWithComponents_WindowIntoPayload:
+		s := proto.Size(x.WindowIntoPayload)
+		n += proto.SizeVarint(12<<3 | proto.WireBytes)
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case *MessageWithComponents_WindowingStrategy:
+		s := proto.Size(x.WindowingStrategy)
+		n += proto.SizeVarint(13<<3 | proto.WireBytes)
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case *MessageWithComponents_FunctionSpec:
+		s := proto.Size(x.FunctionSpec)
+		n += proto.SizeVarint(14<<3 | proto.WireBytes)
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case nil:
+	default:
+		panic(fmt.Sprintf("proto: unexpected type %T in oneof", x))
+	}
+	return n
+}
+
+// A Pipeline is a hierarchical graph of PTransforms, linked
+// by PCollections.
+//
+// This is represented by a number of by-reference maps to nodes,
+// PCollections, SDK environments, UDF, etc., for
+// supporting compact reuse and arbitrary graph structure.
+//
+// All of the keys in the maps here are arbitrary strings that are only
+// required to be internally consistent within this proto message.
+type Pipeline struct {
+	// (Required) The coders, UDFs, graph nodes, etc, that make up
+	// this pipeline.
+	Components *Components `protobuf:"bytes,1,opt,name=components" json:"components,omitempty"`
+	// (Required) The ids of all PTransforms that are not contained within another PTransform.
+	// These must be in shallow topological order, so that traversing them recursively
+	// in this order yields a recursively topological traversal.
+	RootTransformIds []string `protobuf:"bytes,2,rep,name=root_transform_ids,json=rootTransformIds" json:"root_transform_ids,omitempty"`
+	// (Optional) Static display data for the pipeline. If there is none,
+	// it may be omitted.
+	DisplayData *DisplayData `protobuf:"bytes,3,opt,name=display_data,json=displayData" json:"display_data,omitempty"`
+}
+
+func (m *Pipeline) Reset()                    { *m = Pipeline{} }
+func (m *Pipeline) String() string            { return proto.CompactTextString(m) }
+func (*Pipeline) ProtoMessage()               {}
+func (*Pipeline) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{2} }
+
+func (m *Pipeline) GetComponents() *Components {
+	if m != nil {
+		return m.Components
+	}
+	return nil
+}
+
+func (m *Pipeline) GetRootTransformIds() []string {
+	if m != nil {
+		return m.RootTransformIds
+	}
+	return nil
+}
+
+func (m *Pipeline) GetDisplayData() *DisplayData {
+	if m != nil {
+		return m.DisplayData
+	}
+	return nil
+}
+
+// An applied PTransform! This does not contain the graph data, but only the
+// fields specific to a graph node that is a Runner API transform
+// between PCollections.
+type PTransform struct {
+	// (Required) A unique name for the application node.
+	//
+	// Ideally, this should be stable over multiple evolutions of a pipeline
+	// for the purposes of logging and associating pipeline state with a node,
+	// etc.
+	//
+	// If it is not stable, then the runner decides what will happen. But, most
+	// importantly, it must always be here and be unique, even if it is
+	// autogenerated.
+	UniqueName string `protobuf:"bytes,5,opt,name=unique_name,json=uniqueName" json:"unique_name,omitempty"`
+	// (Optional) A URN and payload that, together, fully defined the semantics
+	// of this transform.
+	//
+	// If absent, this must be an "anonymous" composite transform.
+	//
+	// For primitive transform in the Runner API, this is required, and the
+	// payloads are well-defined messages. When the URN indicates ParDo it
+	// is a ParDoPayload, and so on.
+	//
+	// TODO: document the standardized URNs and payloads
+	// TODO: separate standardized payloads into a separate proto file
+	//
+	// For some special composite transforms, the payload is also officially
+	// defined:
+	//
+	//  - when the URN is "urn:beam:transforms:combine" it is a CombinePayload
+	//
+	Spec *FunctionSpec `protobuf:"bytes,1,opt,name=spec" json:"spec,omitempty"`
+	// (Optional) if this node is a composite, a list of the ids of
+	// transforms that it contains.
+	Subtransforms []string `protobuf:"bytes,2,rep,name=subtransforms" json:"subtransforms,omitempty"`
+	// (Required) A map from local names of inputs (unique only with this map, and
+	// likely embedded in the transform payload and serialized user code) to
+	// PCollection ids.
+	//
+	// The payload for this transform may clarify the relationship of these
+	// inputs. For example:
+	//
+	//  - for a Flatten transform they are merged
+	//  - for a ParDo transform, some may be side inputs
+	//
+	// All inputs are recorded here so that the topological ordering of
+	// the graph is consistent whether or not the payload is understood.
+	//
+	Inputs map[string]string `protobuf:"bytes,3,rep,name=inputs" json:"inputs,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"`
+	// (Required) A map from local names of outputs (unique only within this map,
+	// and likely embedded in the transform payload and serialized user code)
+	// to PCollection ids.
+	//
+	// The URN or payload for this transform node may clarify the type and
+	// relationship of these outputs. For example:
+	//
+	//  - for a ParDo transform, these are tags on PCollections, which will be
+	//    embedded in the DoFn.
+	//
+	Outputs map[string]string `protobuf:"bytes,4,rep,name=outputs" json:"outputs,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"`
+	// (Optional) Static display data for this PTransform application. If
+	// there is none, or it is not relevant (such as use by the Fn API)
+	// then it may be omitted.
+	DisplayData *DisplayData `protobuf:"bytes,6,opt,name=display_data,json=displayData" json:"display_data,omitempty"`
+}
+
+func (m *PTransform) Reset()                    { *m = PTransform{} }
+func (m *PTransform) String() string            { return proto.CompactTextString(m) }
+func (*PTransform) ProtoMessage()               {}
+func (*PTransform) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{3} }
+
+func (m *PTransform) GetUniqueName() string {
+	if m != nil {
+		return m.UniqueName
+	}
+	return ""
+}
+
+func (m *PTransform) GetSpec() *FunctionSpec {
+	if m != nil {
+		return m.Spec
+	}
+	return nil
+}
+
+func (m *PTransform) GetSubtransforms() []string {
+	if m != nil {
+		return m.Subtransforms
+	}
+	return nil
+}
+
+func (m *PTransform) GetInputs() map[string]string {
+	if m != nil {
+		return m.Inputs
+	}
+	return nil
+}
+
+func (m *PTransform) GetOutputs() map[string]string {
+	if m != nil {
+		return m.Outputs
+	}
+	return nil
+}
+
+func (m *PTransform) GetDisplayData() *DisplayData {
+	if m != nil {
+		return m.DisplayData
+	}
+	return nil
+}
+
+// A PCollection!
+type PCollection struct {
+	// (Required) A unique name for the PCollection.
+	//
+	// Ideally, this should be stable over multiple evolutions of a pipeline
+	// for the purposes of logging and associating pipeline state with a node,
+	// etc.
+	//
+	// If it is not stable, then the runner decides what will happen. But, most
+	// importantly, it must always be here, even if it is autogenerated.
+	UniqueName string `protobuf:"bytes,1,opt,name=unique_name,json=uniqueName" json:"unique_name,omitempty"`
+	// (Required) The id of the Coder for this PCollection.
+	CoderId string `protobuf:"bytes,2,opt,name=coder_id,json=coderId" json:"coder_id,omitempty"`
+	// (Required) Whether this PCollection is bounded or unbounded
+	IsBounded IsBounded_Enum `protobuf:"varint,3,opt,name=is_bounded,json=isBounded,enum=org.apache.beam.model.pipeline.v1.IsBounded_Enum" json:"is_bounded,omitempty"`
+	// (Required) The id of the windowing strategy for this PCollection.
+	WindowingStrategyId string `protobuf:"bytes,4,opt,name=windowing_strategy_id,json=windowingStrategyId" json:"windowing_strategy_id,omitempty"`
+	// (Optional) Static display data for this PTransform application. If
+	// there is none, or it is not relevant (such as use by the Fn API)
+	// then it may be omitted.
+	DisplayData *DisplayData `protobuf:"bytes,5,opt,name=display_data,json=displayData" json:"display_data,omitempty"`
+}
+
+func (m *PCollection) Reset()                    { *m = PCollection{} }
+func (m *PCollection) String() string            { return proto.CompactTextString(m) }
+func (*PCollection) ProtoMessage()               {}
+func (*PCollection) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{4} }
+
+func (m *PCollection) GetUniqueName() string {
+	if m != nil {
+		return m.UniqueName
+	}
+	return ""
+}
+
+func (m *PCollection) GetCoderId() string {
+	if m != nil {
+		return m.CoderId
+	}
+	return ""
+}
+
+func (m *PCollection) GetIsBounded() IsBounded_Enum {
+	if m != nil {
+		return m.IsBounded
+	}
+	return IsBounded_UNSPECIFIED
+}
+
+func (m *PCollection) GetWindowingStrategyId() string {
+	if m != nil {
+		return m.WindowingStrategyId
+	}
+	return ""
+}
+
+func (m *PCollection) GetDisplayData() *DisplayData {
+	if m != nil {
+		return m.DisplayData
+	}
+	return nil
+}
+
+// The payload for the primitive ParDo transform.
+type ParDoPayload struct {
+	// (Required) The SdkFunctionSpec of the DoFn.
+	DoFn *SdkFunctionSpec `protobuf:"bytes,1,opt,name=do_fn,json=doFn" json:"do_fn,omitempty"`
+	// (Required) Additional pieces of context the DoFn may require that
+	// are not otherwise represented in the payload.
+	// (may force runners to execute the ParDo differently)
+	Parameters []*Parameter `protobuf:"bytes,2,rep,name=parameters" json:"parameters,omitempty"`
+	// (Optional) A mapping of local input names to side inputs, describing
+	// the expected access pattern.
+	SideInputs map[string]*SideInput `protobuf:"bytes,3,rep,name=side_inputs,json=sideInputs" json:"side_inputs,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"`
+	// (Optional) A mapping of local state names to state specifications.
+	StateSpecs map[string]*StateSpec `protobuf:"bytes,4,rep,name=state_specs,json=stateSpecs" json:"state_specs,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"`
+	// (Optional) A mapping of local timer names to timer specifications.
+	TimerSpecs map[string]*TimerSpec `protobuf:"bytes,5,rep,name=timer_specs,json=timerSpecs" json:"timer_specs,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"`
+	// Whether the DoFn is splittable
+	Splittable bool `protobuf:"varint,6,opt,name=splittable" json:"splittable,omitempty"`
+}
+
+func (m *ParDoPayload) Reset()                    { *m = ParDoPayload{} }
+func (m *ParDoPayload) String() string            { return proto.CompactTextString(m) }
+func (*ParDoPayload) ProtoMessage()               {}
+func (*ParDoPayload) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{5} }
+
+func (m *ParDoPayload) GetDoFn() *SdkFunctionSpec {
+	if m != nil {
+		return m.DoFn
+	}
+	return nil
+}
+
+func (m *ParDoPayload) GetParameters() []*Parameter {
+	if m != nil {
+		return m.Parameters
+	}
+	return nil
+}
+
+func (m *ParDoPayload) GetSideInputs() map[string]*SideInput {
+	if m != nil {
+		return m.SideInputs
+	}
+	return nil
+}
+
+func (m *ParDoPayload) GetStateSpecs() map[string]*StateSpec {
+	if m != nil {
+		return m.StateSpecs
+	}
+	return nil
+}
+
+func (m *ParDoPayload) GetTimerSpecs() map[string]*TimerSpec {
+	if m != nil {
+		return m.TimerSpecs
+	}
+	return nil
+}
+
+func (m *ParDoPayload) GetSplittable() bool {
+	if m != nil {
+		return m.Splittable
+	}
+	return false
+}
+
+// Parameters that a UDF might require.
+//
+// The details of how a runner sends these parameters to the SDK harness
+// are the subject of the Fn API.
+//
+// The details of how an SDK harness delivers them to the UDF is entirely
+// up to the SDK. (for some SDKs there may be parameters that are not
+// represented here if the runner doesn't need to do anything)
+//
+// Here, the parameters are simply indicators to the runner that they
+// need to run the function a particular way.
+//
+// TODO: the evolution of the Fn API will influence what needs explicit
+// representation here
+type Parameter struct {
+	Type Parameter_Type_Enum `protobuf:"varint,1,opt,name=type,enum=org.apache.beam.model.pipeline.v1.Parameter_Type_Enum" json:"type,omitempty"`
+}
+
+func (m *Parameter) Reset()                    { *m = Parameter{} }
+func (m *Parameter) String() string            { return proto.CompactTextString(m) }
+func (*Parameter) ProtoMessage()               {}
+func (*Parameter) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{6} }
+
+func (m *Parameter) GetType() Parameter_Type_Enum {
+	if m != nil {
+		return m.Type
+	}
+	return Parameter_Type_UNSPECIFIED
+}
+
+type Parameter_Type struct {
+}
+
+func (m *Parameter_Type) Reset()                    { *m = Parameter_Type{} }
+func (m *Parameter_Type) String() string            { return proto.CompactTextString(m) }
+func (*Parameter_Type) ProtoMessage()               {}
+func (*Parameter_Type) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{6, 0} }
+
+type StateSpec struct {
+	// Types that are valid to be assigned to Spec:
+	//	*StateSpec_ValueSpec
+	//	*StateSpec_BagSpec
+	//	*StateSpec_CombiningSpec
+	//	*StateSpec_MapSpec
+	//	*StateSpec_SetSpec
+	Spec isStateSpec_Spec `protobuf_oneof:"spec"`
+}
+
+func (m *StateSpec) Reset()                    { *m = StateSpec{} }
+func (m *StateSpec) String() string            { return proto.CompactTextString(m) }
+func (*StateSpec) ProtoMessage()               {}
+func (*StateSpec) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{7} }
+
+type isStateSpec_Spec interface {
+	isStateSpec_Spec()
+}
+
+type StateSpec_ValueSpec struct {
+	ValueSpec *ValueStateSpec `protobuf:"bytes,1,opt,name=value_spec,json=valueSpec,oneof"`
+}
+type StateSpec_BagSpec struct {
+	BagSpec *BagStateSpec `protobuf:"bytes,2,opt,name=bag_spec,json=bagSpec,oneof"`
+}
+type StateSpec_CombiningSpec struct {
+	CombiningSpec *CombiningStateSpec `protobuf:"bytes,3,opt,name=combining_spec,json=combiningSpec,oneof"`
+}
+type StateSpec_MapSpec struct {
+	MapSpec *MapStateSpec `protobuf:"bytes,4,opt,name=map_spec,json=mapSpec,oneof"`
+}
+type StateSpec_SetSpec struct {
+	SetSpec *SetStateSpec `protobuf:"bytes,5,opt,name=set_spec,json=setSpec,oneof"`
+}
+
+func (*StateSpec_ValueSpec) isStateSpec_Spec()     {}
+func (*StateSpec_BagSpec) isStateSpec_Spec()       {}
+func (*StateSpec_CombiningSpec) isStateSpec_Spec() {}
+func (*StateSpec_MapSpec) isStateSpec_Spec()       {}
+func (*StateSpec_SetSpec) isStateSpec_Spec()       {}
+
+func (m *StateSpec) GetSpec() isStateSpec_Spec {
+	if m != nil {
+		return m.Spec
+	}
+	return nil
+}
+
+func (m *StateSpec) GetValueSpec() *ValueStateSpec {
+	if x, ok := m.GetSpec().(*StateSpec_ValueSpec); ok {
+		return x.ValueSpec
+	}
+	return nil
+}
+
+func (m *StateSpec) GetBagSpec() *BagStateSpec {
+	if x, ok := m.GetSpec().(*StateSpec_BagSpec); ok {
+		return x.BagSpec
+	}
+	return nil
+}
+
+func (m *StateSpec) GetCombiningSpec() *CombiningStateSpec {
+	if x, ok := m.GetSpec().(*StateSpec_CombiningSpec); ok {
+		return x.CombiningSpec
+	}
+	return nil
+}
+
+func (m *StateSpec) GetMapSpec() *MapStateSpec {
+	if x, ok := m.GetSpec().(*StateSpec_MapSpec); ok {
+		return x.MapSpec
+	}
+	return nil
+}
+
+func (m *StateSpec) GetSetSpec() *SetStateSpec {
+	if x, ok := m.GetSpec().(*StateSpec_SetSpec); ok {
+		return x.SetSpec
+	}
+	return nil
+}
+
+// XXX_OneofFuncs is for the internal use of the proto package.
+func (*StateSpec) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) error, func(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error), func(msg proto.Message) (n int), []interface{}) {
+	return _StateSpec_OneofMarshaler, _StateSpec_OneofUnmarshaler, _StateSpec_OneofSizer, []interface{}{
+		(*StateSpec_ValueSpec)(nil),
+		(*StateSpec_BagSpec)(nil),
+		(*StateSpec_CombiningSpec)(nil),
+		(*StateSpec_MapSpec)(nil),
+		(*StateSpec_SetSpec)(nil),
+	}
+}
+
+func _StateSpec_OneofMarshaler(msg proto.Message, b *proto.Buffer) error {
+	m := msg.(*StateSpec)
+	// spec
+	switch x := m.Spec.(type) {
+	case *StateSpec_ValueSpec:
+		b.EncodeVarint(1<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.ValueSpec); err != nil {
+			return err
+		}
+	case *StateSpec_BagSpec:
+		b.EncodeVarint(2<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.BagSpec); err != nil {
+			return err
+		}
+	case *StateSpec_CombiningSpec:
+		b.EncodeVarint(3<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.CombiningSpec); err != nil {
+			return err
+		}
+	case *StateSpec_MapSpec:
+		b.EncodeVarint(4<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.MapSpec); err != nil {
+			return err
+		}
+	case *StateSpec_SetSpec:
+		b.EncodeVarint(5<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.SetSpec); err != nil {
+			return err
+		}
+	case nil:
+	default:
+		return fmt.Errorf("StateSpec.Spec has unexpected type %T", x)
+	}
+	return nil
+}
+
+func _StateSpec_OneofUnmarshaler(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error) {
+	m := msg.(*StateSpec)
+	switch tag {
+	case 1: // spec.value_spec
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(ValueStateSpec)
+		err := b.DecodeMessage(msg)
+		m.Spec = &StateSpec_ValueSpec{msg}
+		return true, err
+	case 2: // spec.bag_spec
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(BagStateSpec)
+		err := b.DecodeMessage(msg)
+		m.Spec = &StateSpec_BagSpec{msg}
+		return true, err
+	case 3: // spec.combining_spec
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(CombiningStateSpec)
+		err := b.DecodeMessage(msg)
+		m.Spec = &StateSpec_CombiningSpec{msg}
+		return true, err
+	case 4: // spec.map_spec
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(MapStateSpec)
+		err := b.DecodeMessage(msg)
+		m.Spec = &StateSpec_MapSpec{msg}
+		return true, err
+	case 5: // spec.set_spec
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(SetStateSpec)
+		err := b.DecodeMessage(msg)
+		m.Spec = &StateSpec_SetSpec{msg}
+		return true, err
+	default:
+		return false, nil
+	}
+}
+
+func _StateSpec_OneofSizer(msg proto.Message) (n int) {
+	m := msg.(*StateSpec)
+	// spec
+	switch x := m.Spec.(type) {
+	case *StateSpec_ValueSpec:
+		s := proto.Size(x.ValueSpec)
+		n += proto.SizeVarint(1<<3 | proto.WireBytes)
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case *StateSpec_BagSpec:
+		s := proto.Size(x.BagSpec)
+		n += proto.SizeVarint(2<<3 | proto.WireBytes)
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case *StateSpec_CombiningSpec:
+		s := proto.Size(x.CombiningSpec)
+		n += proto.SizeVarint(3<<3 | proto.WireBytes)
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case *StateSpec_MapSpec:
+		s := proto.Size(x.MapSpec)
+		n += proto.SizeVarint(4<<3 | proto.WireBytes)
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case *StateSpec_SetSpec:
+		s := proto.Size(x.SetSpec)
+		n += proto.SizeVarint(5<<3 | proto.WireBytes)
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case nil:
+	default:
+		panic(fmt.Sprintf("proto: unexpected type %T in oneof", x))
+	}
+	return n
+}
+
+type ValueStateSpec struct {
+	CoderId string `protobuf:"bytes,1,opt,name=coder_id,json=coderId" json:"coder_id,omitempty"`
+}
+
+func (m *ValueStateSpec) Reset()                    { *m = ValueStateSpec{} }
+func (m *ValueStateSpec) String() string            { return proto.CompactTextString(m) }
+func (*ValueStateSpec) ProtoMessage()               {}
+func (*ValueStateSpec) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{8} }
+
+func (m *ValueStateSpec) GetCoderId() string {
+	if m != nil {
+		return m.CoderId
+	}
+	return ""
+}
+
+type BagStateSpec struct {
+	ElementCoderId string `protobuf:"bytes,1,opt,name=element_coder_id,json=elementCoderId" json:"element_coder_id,omitempty"`
+}
+
+func (m *BagStateSpec) Reset()                    { *m = BagStateSpec{} }
+func (m *BagStateSpec) String() string            { return proto.CompactTextString(m) }
+func (*BagStateSpec) ProtoMessage()               {}
+func (*BagStateSpec) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{9} }
+
+func (m *BagStateSpec) GetElementCoderId() string {
+	if m != nil {
+		return m.ElementCoderId
+	}
+	return ""
+}
+
+type CombiningStateSpec struct {
+	AccumulatorCoderId string           `protobuf:"bytes,1,opt,name=accumulator_coder_id,json=accumulatorCoderId" json:"accumulator_coder_id,omitempty"`
+	CombineFn          *SdkFunctionSpec `protobuf:"bytes,2,opt,name=combine_fn,json=combineFn" json:"combine_fn,omitempty"`
+}
+
+func (m *CombiningStateSpec) Reset()                    { *m = CombiningStateSpec{} }
+func (m *CombiningStateSpec) String() string            { return proto.CompactTextString(m) }
+func (*CombiningStateSpec) ProtoMessage()               {}
+func (*CombiningStateSpec) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{10} }
+
+func (m *CombiningStateSpec) GetAccumulatorCoderId() string {
+	if m != nil {
+		return m.AccumulatorCoderId
+	}
+	return ""
+}
+
+func (m *CombiningStateSpec) GetCombineFn() *SdkFunctionSpec {
+	if m != nil {
+		return m.CombineFn
+	}
+	return nil
+}
+
+type MapStateSpec struct {
+	KeyCoderId   string `protobuf:"bytes,1,opt,name=key_coder_id,json=keyCoderId" json:"key_coder_id,omitempty"`
+	ValueCoderId string `protobuf:"bytes,2,opt,name=value_coder_id,json=valueCoderId" json:"value_coder_id,omitempty"`
+}
+
+func (m *MapStateSpec) Reset()                    { *m = MapStateSpec{} }
+func (m *MapStateSpec) String() string            { return proto.CompactTextString(m) }
+func (*MapStateSpec) ProtoMessage()               {}
+func (*MapStateSpec) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{11} }
+
+func (m *MapStateSpec) GetKeyCoderId() string {
+	if m != nil {
+		return m.KeyCoderId
+	}
+	return ""
+}
+
+func (m *MapStateSpec) GetValueCoderId() string {
+	if m != nil {
+		return m.ValueCoderId
+	}
+	return ""
+}
+
+type SetStateSpec struct {
+	ElementCoderId string `protobuf:"bytes,1,opt,name=element_coder_id,json=elementCoderId" json:"element_coder_id,omitempty"`
+}
+
+func (m *SetStateSpec) Reset()                    { *m = SetStateSpec{} }
+func (m *SetStateSpec) String() string            { return proto.CompactTextString(m) }
+func (*SetStateSpec) ProtoMessage()               {}
+func (*SetStateSpec) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{12} }
+
+func (m *SetStateSpec) GetElementCoderId() string {
+	if m != nil {
+		return m.ElementCoderId
+	}
+	return ""
+}
+
+type TimerSpec struct {
+	TimeDomain TimeDomain_Enum `protobuf:"varint,1,opt,name=time_domain,json=timeDomain,enum=org.apache.beam.model.pipeline.v1.TimeDomain_Enum" json:"time_domain,omitempty"`
+}
+
+func (m *TimerSpec) Reset()                    { *m = TimerSpec{} }
+func (m *TimerSpec) String() string            { return proto.CompactTextString(m) }
+func (*TimerSpec) ProtoMessage()               {}
+func (*TimerSpec) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{13} }
+
+func (m *TimerSpec) GetTimeDomain() TimeDomain_Enum {
+	if m != nil {
+		return m.TimeDomain
+	}
+	return TimeDomain_UNSPECIFIED
+}
+
+type IsBounded struct {
+}
+
+func (m *IsBounded) Reset()                    { *m = IsBounded{} }
+func (m *IsBounded) String() string            { return proto.CompactTextString(m) }
+func (*IsBounded) ProtoMessage()               {}
+func (*IsBounded) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{14} }
+
+// The payload for the primitive Read transform.
+type ReadPayload struct {
+	// (Required) The SdkFunctionSpec of the source for this Read.
+	Source *SdkFunctionSpec `protobuf:"bytes,1,opt,name=source" json:"source,omitempty"`
+	// (Required) Whether the source is bounded or unbounded
+	IsBounded IsBounded_Enum `protobuf:"varint,2,opt,name=is_bounded,json=isBounded,enum=org.apache.beam.model.pipeline.v1.IsBounded_Enum" json:"is_bounded,omitempty"`
+}
+
+func (m *ReadPayload) Reset()                    { *m = ReadPayload{} }
+func (m *ReadPayload) String() string            { return proto.CompactTextString(m) }
+func (*ReadPayload) ProtoMessage()               {}
+func (*ReadPayload) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{15} }
+
+func (m *ReadPayload) GetSource() *SdkFunctionSpec {
+	if m != nil {
+		return m.Source
+	}
+	return nil
+}
+
+func (m *ReadPayload) GetIsBounded() IsBounded_Enum {
+	if m != nil {
+		return m.IsBounded
+	}
+	return IsBounded_UNSPECIFIED
+}
+
+// The payload for the WindowInto transform.
+type WindowIntoPayload struct {
+	// (Required) The SdkFunctionSpec of the WindowFn.
+	WindowFn *SdkFunctionSpec `protobuf:"bytes,1,opt,name=window_fn,json=windowFn" json:"window_fn,omitempty"`
+}
+
+func (m *WindowIntoPayload) Reset()                    { *m = WindowIntoPayload{} }
+func (m *WindowIntoPayload) String() string            { return proto.CompactTextString(m) }
+func (*WindowIntoPayload) ProtoMessage()               {}
+func (*WindowIntoPayload) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{16} }
+
+func (m *WindowIntoPayload) GetWindowFn() *SdkFunctionSpec {
+	if m != nil {
+		return m.WindowFn
+	}
+	return nil
+}
+
+// The payload for the special-but-not-primitive Combine transform.
+type CombinePayload struct {
+	// (Required) The SdkFunctionSpec of the CombineFn.
+	CombineFn *SdkFunctionSpec `protobuf:"bytes,1,opt,name=combine_fn,json=combineFn" json:"combine_fn,omitempty"`
+	// (Required) A reference to the Coder to use for accumulators of the CombineFn
+	AccumulatorCoderId string `protobuf:"bytes,2,opt,name=accumulator_coder_id,json=accumulatorCoderId" json:"accumulator_coder_id,omitempty"`
+	// (Required) Additional pieces of context the DoFn may require that
+	// are not otherwise represented in the payload.
+	// (may force runners to execute the ParDo differently)
+	Parameters []*Parameter `protobuf:"bytes,3,rep,name=parameters" json:"parameters,omitempty"`
+	// (Optional) A mapping of local input names to side inputs, describing
+	// the expected access pattern.
+	SideInputs map[string]*SideInput `protobuf:"bytes,4,rep,name=side_inputs,json=sideInputs" json:"side_inputs,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"`
+}
+
+func (m *CombinePayload) Reset()                    { *m = CombinePayload{} }
+func (m *CombinePayload) String() string            { return proto.CompactTextString(m) }
+func (*CombinePayload) ProtoMessage()               {}
+func (*CombinePayload) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{17} }
+
+func (m *CombinePayload) GetCombineFn() *SdkFunctionSpec {
+	if m != nil {
+		return m.CombineFn
+	}
+	return nil
+}
+
+func (m *CombinePayload) GetAccumulatorCoderId() string {
+	if m != nil {
+		return m.AccumulatorCoderId
+	}
+	return ""
+}
+
+func (m *CombinePayload) GetParameters() []*Parameter {
+	if m != nil {
+		return m.Parameters
+	}
+	return nil
+}
+
+func (m *CombinePayload) GetSideInputs() map[string]*SideInput {
+	if m != nil {
+		return m.SideInputs
+	}
+	return nil
+}
+
+// The payload for the test-only primitive TestStream
+type TestStreamPayload struct {
+	// (Required) the coder for elements in the TestStream events
+	CoderId string                     `protobuf:"bytes,1,opt,name=coder_id,json=coderId" json:"coder_id,omitempty"`
+	Events  []*TestStreamPayload_Event `protobuf:"bytes,2,rep,name=events" json:"events,omitempty"`
+}
+
+func (m *TestStreamPayload) Reset()                    { *m = TestStreamPayload{} }
+func (m *TestStreamPayload) String() string            { return proto.CompactTextString(m) }
+func (*TestStreamPayload) ProtoMessage()               {}
+func (*TestStreamPayload) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{18} }
+
+func (m *TestStreamPayload) GetCoderId() string {
+	if m != nil {
+		return m.CoderId
+	}
+	return ""
+}
+
+func (m *TestStreamPayload) GetEvents() []*TestStreamPayload_Event {
+	if m != nil {
+		return m.Events
+	}
+	return nil
+}
+
+type TestStreamPayload_Event struct {
+	// Types that are valid to be assigned to Event:
+	//	*TestStreamPayload_Event_WatermarkEvent
+	//	*TestStreamPayload_Event_ProcessingTimeEvent
+	//	*TestStreamPayload_Event_ElementEvent
+	Event isTestStreamPayload_Event_Event `protobuf_oneof:"event"`
+}
+
+func (m *TestStreamPayload_Event) Reset()                    { *m = TestStreamPayload_Event{} }
+func (m *TestStreamPayload_Event) String() string            { return proto.CompactTextString(m) }
+func (*TestStreamPayload_Event) ProtoMessage()               {}
+func (*TestStreamPayload_Event) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{18, 0} }
+
+type isTestStreamPayload_Event_Event interface {
+	isTestStreamPayload_Event_Event()
+}
+
+type TestStreamPayload_Event_WatermarkEvent struct {
+	WatermarkEvent *TestStreamPayload_Event_AdvanceWatermark `protobuf:"bytes,1,opt,name=watermark_event,json=watermarkEvent,oneof"`
+}
+type TestStreamPayload_Event_ProcessingTimeEvent struct {
+	ProcessingTimeEvent *TestStreamPayload_Event_AdvanceProcessingTime `protobuf:"bytes,2,opt,name=processing_time_event,json=processingTimeEvent,oneof"`
+}
+type TestStreamPayload_Event_ElementEvent struct {
+	ElementEvent *TestStreamPayload_Event_AddElements `protobuf:"bytes,3,opt,name=element_event,json=elementEvent,oneof"`
+}
+
+func (*TestStreamPayload_Event_WatermarkEvent) isTestStreamPayload_Event_Event()      {}
+func (*TestStreamPayload_Event_ProcessingTimeEvent) isTestStreamPayload_Event_Event() {}
+func (*TestStreamPayload_Event_ElementEvent) isTestStreamPayload_Event_Event()        {}
+
+func (m *TestStreamPayload_Event) GetEvent() isTestStreamPayload_Event_Event {
+	if m != nil {
+		return m.Event
+	}
+	return nil
+}
+
+func (m *TestStreamPayload_Event) GetWatermarkEvent() *TestStreamPayload_Event_AdvanceWatermark {
+	if x, ok := m.GetEvent().(*TestStreamPayload_Event_WatermarkEvent); ok {
+		return x.WatermarkEvent
+	}
+	return nil
+}
+
+func (m *TestStreamPayload_Event) GetProcessingTimeEvent() *TestStreamPayload_Event_AdvanceProcessingTime {
+	if x, ok := m.GetEvent().(*TestStreamPayload_Event_ProcessingTimeEvent); ok {
+		return x.ProcessingTimeEvent
+	}
+	return nil
+}
+
+func (m *TestStreamPayload_Event) GetElementEvent() *TestStreamPayload_Event_AddElements {
+	if x, ok := m.GetEvent().(*TestStreamPayload_Event_ElementEvent); ok {
+		return x.ElementEvent
+	}
+	return nil
+}
+
+// XXX_OneofFuncs is for the internal use of the proto package.
+func (*TestStreamPayload_Event) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) error, func(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error), func(msg proto.Message) (n int), []interface{}) {
+	return _TestStreamPayload_Event_OneofMarshaler, _TestStreamPayload_Event_OneofUnmarshaler, _TestStreamPayload_Event_OneofSizer, []interface{}{
+		(*TestStreamPayload_Event_WatermarkEvent)(nil),
+		(*TestStreamPayload_Event_ProcessingTimeEvent)(nil),
+		(*TestStreamPayload_Event_ElementEvent)(nil),
+	}
+}
+
+func _TestStreamPayload_Event_OneofMarshaler(msg proto.Message, b *proto.Buffer) error {
+	m := msg.(*TestStreamPayload_Event)
+	// event
+	switch x := m.Event.(type) {
+	case *TestStreamPayload_Event_WatermarkEvent:
+		b.EncodeVarint(1<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.WatermarkEvent); err != nil {
+			return err
+		}
+	case *TestStreamPayload_Event_ProcessingTimeEvent:
+		b.EncodeVarint(2<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.ProcessingTimeEvent); err != nil {
+			return err
+		}
+	case *TestStreamPayload_Event_ElementEvent:
+		b.EncodeVarint(3<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.ElementEvent); err != nil {
+			return err
+		}
+	case nil:
+	default:
+		return fmt.Errorf("TestStreamPayload_Event.Event has unexpected type %T", x)
+	}
+	return nil
+}
+
+func _TestStreamPayload_Event_OneofUnmarshaler(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error) {
+	m := msg.(*TestStreamPayload_Event)
+	switch tag {
+	case 1: // event.watermark_event
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(TestStreamPayload_Event_AdvanceWatermark)
+		err := b.DecodeMessage(msg)
+		m.Event = &TestStreamPayload_Event_WatermarkEvent{msg}
+		return true, err
+	case 2: // event.processing_time_event
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(TestStreamPayload_Event_AdvanceProcessingTime)
+		err := b.DecodeMessage(msg)
+		m.Event = &TestStreamPayload_Event_ProcessingTimeEvent{msg}
+		return true, err
+	case 3: // event.element_event
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(TestStreamPayload_Event_AddElements)
+		err := b.DecodeMessage(msg)
+		m.Event = &TestStreamPayload_Event_ElementEvent{msg}
+		return true, err
+	default:
+		return false, nil
+	}
+}
+
+func _TestStreamPayload_Event_OneofSizer(msg proto.Message) (n int) {
+	m := msg.(*TestStreamPayload_Event)
+	// event
+	switch x := m.Event.(type) {
+	case *TestStreamPayload_Event_WatermarkEvent:
+		s := proto.Size(x.WatermarkEvent)
+		n += proto.SizeVarint(1<<3 | proto.WireBytes)
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case *TestStreamPayload_Event_ProcessingTimeEvent:
+		s := proto.Size(x.ProcessingTimeEvent)
+		n += proto.SizeVarint(2<<3 | proto.WireBytes)
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case *TestStreamPayload_Event_ElementEvent:
+		s := proto.Size(x.ElementEvent)
+		n += proto.SizeVarint(3<<3 | proto.WireBytes)
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case nil:
+	default:
+		panic(fmt.Sprintf("proto: unexpected type %T in oneof", x))
+	}
+	return n
+}
+
+type TestStreamPayload_Event_AdvanceWatermark struct {
+	NewWatermark int64 `protobuf:"varint,1,opt,name=new_watermark,json=newWatermark" json:"new_watermark,omitempty"`
+}
+
+func (m *TestStreamPayload_Event_AdvanceWatermark) Reset() {
+	*m = TestStreamPayload_Event_AdvanceWatermark{}
+}
+func (m *TestStreamPayload_Event_AdvanceWatermark) String() string { return proto.CompactTextString(m) }
+func (*TestStreamPayload_Event_AdvanceWatermark) ProtoMessage()    {}
+func (*TestStreamPayload_Event_AdvanceWatermark) Descriptor() ([]byte, []int) {
+	return fileDescriptor0, []int{18, 0, 0}
+}
+
+func (m *TestStreamPayload_Event_AdvanceWatermark) GetNewWatermark() int64 {
+	if m != nil {
+		return m.NewWatermark
+	}
+	return 0
+}
+
+type TestStreamPayload_Event_AdvanceProcessingTime struct {
+	AdvanceDuration int64 `protobuf:"varint,1,opt,name=advance_duration,json=advanceDuration" json:"advance_duration,omitempty"`
+}
+
+func (m *TestStreamPayload_Event_AdvanceProcessingTime) Reset() {
+	*m = TestStreamPayload_Event_AdvanceProcessingTime{}
+}
+func (m *TestStreamPayload_Event_AdvanceProcessingTime) String() string {
+	return proto.CompactTextString(m)
+}
+func (*TestStreamPayload_Event_AdvanceProcessingTime) ProtoMessage() {}
+func (*TestStreamPayload_Event_AdvanceProcessingTime) Descriptor() ([]byte, []int) {
+	return fileDescriptor0, []int{18, 0, 1}
+}
+
+func (m *TestStreamPayload_Event_AdvanceProcessingTime) GetAdvanceDuration() int64 {
+	if m != nil {
+		return m.AdvanceDuration
+	}
+	return 0
+}
+
+type TestStreamPayload_Event_AddElements struct {
+	Elements []*TestStreamPayload_TimestampedElement `protobuf:"bytes,1,rep,name=elements" json:"elements,omitempty"`
+}
+
+func (m *TestStreamPayload_Event_AddElements) Reset()         { *m = TestStreamPayload_Event_AddElements{} }
+func (m *TestStreamPayload_Event_AddElements) String() string { return proto.CompactTextString(m) }
+func (*TestStreamPayload_Event_AddElements) ProtoMessage()    {}
+func (*TestStreamPayload_Event_AddElements) Descriptor() ([]byte, []int) {
+	return fileDescriptor0, []int{18, 0, 2}
+}
+
+func (m *TestStreamPayload_Event_AddElements) GetElements() []*TestStreamPayload_TimestampedElement {
+	if m != nil {
+		return m.Elements
+	}
+	return nil
+}
+
+type TestStreamPayload_TimestampedElement struct {
+	EncodedElement []byte `protobuf:"bytes,1,opt,name=encoded_element,json=encodedElement,proto3" json:"encoded_element,omitempty"`
+	Timestamp      int64  `protobuf:"varint,2,opt,name=timestamp" json:"timestamp,omitempty"`
+}
+
+func (m *TestStreamPayload_TimestampedElement) Reset()         { *m = TestStreamPayload_TimestampedElement{} }
+func (m *TestStreamPayload_TimestampedElement) String() string { return proto.CompactTextString(m) }
+func (*TestStreamPayload_TimestampedElement) ProtoMessage()    {}
+func (*TestStreamPayload_TimestampedElement) Descriptor() ([]byte, []int) {
+	return fileDescriptor0, []int{18, 1}
+}
+
+func (m *TestStreamPayload_TimestampedElement) GetEncodedElement() []byte {
+	if m != nil {
+		return m.EncodedElement
+	}
+	return nil
+}
+
+func (m *TestStreamPayload_TimestampedElement) GetTimestamp() int64 {
+	if m != nil {
+		return m.Timestamp
+	}
+	return 0
+}
+
+// The payload for the special-but-not-primitive WriteFiles transform.
+type WriteFilesPayload struct {
+	// (Required) The SdkFunctionSpec of the FileBasedSink.
+	Sink *SdkFunctionSpec `protobuf:"bytes,1,opt,name=sink" json:"sink,omitempty"`
+	// (Required) The format function.
+	FormatFunction           *SdkFunctionSpec      `protobuf:"bytes,2,opt,name=format_function,json=formatFunction" json:"format_function,omitempty"`
+	WindowedWrites           bool                  `protobuf:"varint,3,opt,name=windowed_writes,json=windowedWrites" json:"windowed_writes,omitempty"`
+	RunnerDeterminedSharding bool                  `protobuf:"varint,4,opt,name=runner_determined_sharding,json=runnerDeterminedSharding" json:"runner_determined_sharding,omitempty"`
+	SideInputs               map[string]*SideInput `protobuf:"bytes,5,rep,name=side_inputs,json=sideInputs" json:"side_inputs,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"`
+}
+
+func (m *WriteFilesPayload) Reset()                    { *m = WriteFilesPayload{} }
+func (m *WriteFilesPayload) String() string            { return proto.CompactTextString(m) }
+func (*WriteFilesPayload) ProtoMessage()               {}
+func (*WriteFilesPayload) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{19} }
+
+func (m *WriteFilesPayload) GetSink() *SdkFunctionSpec {
+	if m != nil {
+		return m.Sink
+	}
+	return nil
+}
+
+func (m *WriteFilesPayload) GetFormatFunction() *SdkFunctionSpec {
+	if m != nil {
+		return m.FormatFunction
+	}
+	return nil
+}
+
+func (m *WriteFilesPayload) GetWindowedWrites() bool {
+	if m != nil {
+		return m.WindowedWrites
+	}
+	return false
+}
+
+func (m *WriteFilesPayload) GetRunnerDeterminedSharding() bool {
+	if m != nil {
+		return m.RunnerDeterminedSharding
+	}
+	return false
+}
+
+func (m *WriteFilesPayload) GetSideInputs() map[string]*SideInput {
+	if m != nil {
+		return m.SideInputs
+	}
+	return nil
+}
+
+// A coder, the binary format for serialization and deserialization of data in
+// a pipeline.
+type Coder struct {
+	// (Required) A specification for the coder, as a URN plus parameters. This
+	// may be a cross-language agreed-upon format, or it may be a "custom coder"
+	// that can only be used by a particular SDK. It does not include component
+	// coders, as it is beneficial for these to be comprehensible to a runner
+	// regardless of whether the binary format is agree-upon.
+	Spec *SdkFunctionSpec `protobuf:"bytes,1,opt,name=spec" json:"spec,omitempty"`
+	// (Optional) If this coder is parametric, such as ListCoder(VarIntCoder),
+	// this is a list of the components. In order for encodings to be identical,
+	// the SdkFunctionSpec and all components must be identical, recursively.
+	ComponentCoderIds []string `protobuf:"bytes,2,rep,name=component_coder_ids,json=componentCoderIds" json:"component_coder_ids,omitempty"`
+}
+
+func (m *Coder) Reset()                    { *m = Coder{} }
+func (m *Coder) String() string            { return proto.CompactTextString(m) }
+func (*Coder) ProtoMessage()               {}
+func (*Coder) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{20} }
+
+func (m *Coder) GetSpec() *SdkFunctionSpec {
+	if m != nil {
+		return m.Spec
+	}
+	return nil
+}
+
+func (m *Coder) GetComponentCoderIds() []string {
+	if m != nil {
+		return m.ComponentCoderIds
+	}
+	return nil
+}
+
+// A windowing strategy describes the window function, triggering, allowed
+// lateness, and accumulation mode for a PCollection.
+//
+// TODO: consider inlining field on PCollection
+type WindowingStrategy struct {
+	// (Required) The SdkFunctionSpec of the UDF that assigns windows,
+	// merges windows, and shifts timestamps before they are
+	// combined according to the OutputTime.
+	WindowFn *SdkFunctionSpec `protobuf:"bytes,1,opt,name=window_fn,json=windowFn" json:"window_fn,omitempty"`
+	// (Required) Whether or not the window fn is merging.
+	//
+	// This knowledge is required for many optimizations.
+	MergeStatus MergeStatus_Enum `protobuf:"varint,2,opt,name=merge_status,json=mergeStatus,enum=org.apache.beam.model.pipeline.v1.MergeStatus_Enum" json:"merge_status,omitempty"`
+	// (Required) The coder for the windows of this PCollection.
+	WindowCoderId string `protobuf:"bytes,3,opt,name=window_coder_id,json=windowCoderId" json:"window_coder_id,omitempty"`
+	// (Required) The trigger to use when grouping this PCollection.
+	Trigger *Trigger `protobuf:"bytes,4,opt,name=trigger" json:"trigger,omitempty"`
+	// (Required) The accumulation mode indicates whether new panes are a full
+	// replacement for prior panes or whether they are deltas to be combined
+	// with other panes (the combine should correspond to whatever the upstream
+	// grouping transform is).
+	AccumulationMode AccumulationMode_Enum `protobuf:"varint,5,opt,name=accumulation_mode,json=accumulationMode,enum=org.apache.beam.model.pipeline.v1.AccumulationMode_Enum" json:"accumulation_mode,omitempty"`
+	// (Required) The OutputTime specifies, for a grouping transform, how to
+	// compute the aggregate timestamp. The window_fn will first possibly shift
+	// it later, then the OutputTime takes the max, min, or ignores it and takes
+	// the end of window.
+	//
+	// This is actually only for input to grouping transforms, but since they
+	// may be introduced in runner-specific ways, it is carried along with the
+	// windowing strategy.
+	OutputTime OutputTime_Enum `protobuf:"varint,6,opt,name=output_time,json=outputTime,enum=org.apache.beam.model.pipeline.v1.OutputTime_Enum" json:"output_time,omitempty"`
+	// (Required) Indicate when output should be omitted upon window expiration.
+	ClosingBehavior ClosingBehavior_Enum `protobuf:"varint,7,opt,name=closing_behavior,json=closingBehavior,enum=org.apache.beam.model.pipeline.v1.ClosingBehavior_Enum" json:"closing_behavior,omitempty"`
+	// (Required) The duration, in milliseconds, beyond the end of a window at
+	// which the window becomes droppable.
+	AllowedLateness int64 `protobuf:"varint,8,opt,name=allowed_lateness,json=allowedLateness" json:"allowed_lateness,omitempty"`
+	// (Required) Indicate whether empty on-time panes should be omitted.
+	OnTimeBehavior OnTimeBehavior_Enum `protobuf:"varint,9,opt,name=OnTimeBehavior,enum=org.apache.beam.model.pipeline.v1.OnTimeBehavior_Enum" json:"OnTimeBehavior,omitempty"`
+	// (Required) Whether or not the window fn assigns inputs to exactly one window
+	//
+	// This knowledge is required for some optimizations
+	AssignsToOneWindow bool `protobuf:"varint,10,opt,name=assigns_to_one_window,json=assignsToOneWindow" json:"assigns_to_one_window,omitempty"`
+}
+
+func (m *WindowingStrategy) Reset()                    { *m = WindowingStrategy{} }
+func (m *WindowingStrategy) String() string            { return proto.CompactTextString(m) }
+func (*WindowingStrategy) ProtoMessage()               {}
+func (*WindowingStrategy) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{21} }
+
+func (m *WindowingStrategy) GetWindowFn() *SdkFunctionSpec {
+	if m != nil {
+		return m.WindowFn
+	}
+	return nil
+}
+
+func (m *WindowingStrategy) GetMergeStatus() MergeStatus_Enum {
+	if m != nil {
+		return m.MergeStatus
+	}
+	return MergeStatus_UNSPECIFIED
+}
+
+func (m *WindowingStrategy) GetWindowCoderId() string {
+	if m != nil {
+		return m.WindowCoderId
+	}
+	return ""
+}
+
+func (m *WindowingStrategy) GetTrigger() *Trigger {
+	if m != nil {
+		return m.Trigger
+	}
+	return nil
+}
+
+func (m *WindowingStrategy) GetAccumulationMode() AccumulationMode_Enum {
+	if m != nil {
+		return m.AccumulationMode
+	}
+	return AccumulationMode_UNSPECIFIED
+}
+
+func (m *WindowingStrategy) GetOutputTime() OutputTime_Enum {
+	if m != nil {
+		return m.OutputTime
+	}
+	return OutputTime_UNSPECIFIED
+}
+
+func (m *WindowingStrategy) GetClosingBehavior() ClosingBehavior_Enum {
+	if m != nil {
+		return m.ClosingBehavior
+	}
+	return ClosingBehavior_UNSPECIFIED
+}
+
+func (m *WindowingStrategy) GetAllowedLateness() int64 {
+	if m != nil {
+		return m.AllowedLateness
+	}
+	return 0
+}
+
+func (m *WindowingStrategy) GetOnTimeBehavior() OnTimeBehavior_Enum {
+	if m != nil {
+		return m.OnTimeBehavior
+	}
+	return OnTimeBehavior_UNSPECIFIED
+}
+
+func (m *WindowingStrategy) GetAssignsToOneWindow() bool {
+	if m != nil {
+		return m.AssignsToOneWindow
+	}
+	return false
+}
+
+// Whether or not a PCollection's WindowFn is non-merging, merging, or
+// merging-but-already-merged, in which case a subsequent GroupByKey is almost
+// always going to do something the user does not want
+type MergeStatus struct {
+}
+
+func (m *MergeStatus) Reset()                    { *m = MergeStatus{} }
+func (m *MergeStatus) String() string            { return proto.CompactTextString(m) }
+func (*MergeStatus) ProtoMessage()               {}
+func (*MergeStatus) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{22} }
+
+// Whether or not subsequent outputs of aggregations should be entire
+// replacement values or just the aggregation of inputs received since
+// the prior output.
+type AccumulationMode struct {
+}
+
+func (m *AccumulationMode) Reset()                    { *m = AccumulationMode{} }
+func (m *AccumulationMode) String() string            { return proto.CompactTextString(m) }
+func (*AccumulationMode) ProtoMessage()               {}
+func (*AccumulationMode) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{23} }
+
+// Controls whether or not an aggregating transform should output data
+// when a window expires.
+type ClosingBehavior struct {
+}
+
+func (m *ClosingBehavior) Reset()                    { *m = ClosingBehavior{} }
+func (m *ClosingBehavior) String() string            { return proto.CompactTextString(m) }
+func (*ClosingBehavior) ProtoMessage()               {}
+func (*ClosingBehavior) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{24} }
+
+// Controls whether or not an aggregating transform should output data
+// when an on-time pane is empty.
+type OnTimeBehavior struct {
+}
+
+func (m *OnTimeBehavior) Reset()                    { *m = OnTimeBehavior{} }
+func (m *OnTimeBehavior) String() string            { return proto.CompactTextString(m) }
+func (*OnTimeBehavior) ProtoMessage()               {}
+func (*OnTimeBehavior) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{25} }
+
+// When a number of windowed, timestamped inputs are aggregated, the timestamp
+// for the resulting output.
+type OutputTime struct {
+}
+
+func (m *OutputTime) Reset()                    { *m = OutputTime{} }
+func (m *OutputTime) String() string            { return proto.CompactTextString(m) }
+func (*OutputTime) ProtoMessage()               {}
+func (*OutputTime) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{26} }
+
+// The different time domains in the Beam model.
+type TimeDomain struct {
+}
+
+func (m *TimeDomain) Reset()                    { *m = TimeDomain{} }
+func (m *TimeDomain) String() string            { return proto.CompactTextString(m) }
+func (*TimeDomain) ProtoMessage()               {}
+func (*TimeDomain) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{27} }
+
+// A small DSL for expressing when to emit new aggregations
+// from a GroupByKey or CombinePerKey
+//
+// A trigger is described in terms of when it is _ready_ to permit output.
+type Trigger struct {
+	// The full disjoint union of possible triggers.
+	//
+	// Types that are valid to be assigned to Trigger:
+	//	*Trigger_AfterAll_
+	//	*Trigger_AfterAny_
+	//	*Trigger_AfterEach_
+	//	*Trigger_AfterEndOfWindow_
+	//	*Trigger_AfterProcessingTime_
+	//	*Trigger_AfterSynchronizedProcessingTime_
+	//	*Trigger_Always_
+	//	*Trigger_Default_
+	//	*Trigger_ElementCount_
+	//	*Trigger_Never_
+	//	*Trigger_OrFinally_
+	//	*Trigger_Repeat_
+	Trigger isTrigger_Trigger `protobuf_oneof:"trigger"`
+}
+
+func (m *Trigger) Reset()                    { *m = Trigger{} }
+func (m *Trigger) String() string            { return proto.CompactTextString(m) }
+func (*Trigger) ProtoMessage()               {}
+func (*Trigger) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{28} }
+
+type isTrigger_Trigger interface {
+	isTrigger_Trigger()
+}
+
+type Trigger_AfterAll_ struct {
+	AfterAll *Trigger_AfterAll `protobuf:"bytes,1,opt,name=after_all,json=afterAll,oneof"`
+}
+type Trigger_AfterAny_ struct {
+	AfterAny *Trigger_AfterAny `protobuf:"bytes,2,opt,name=after_any,json=afterAny,oneof"`
+}
+type Trigger_AfterEach_ struct {
+	AfterEach *Trigger_AfterEach `protobuf:"bytes,3,opt,name=after_each,json=afterEach,oneof"`
+}
+type Trigger_AfterEndOfWindow_ struct {
+	AfterEndOfWindow *Trigger_AfterEndOfWindow `protobuf:"bytes,4,opt,name=after_end_of_window,json=afterEndOfWindow,oneof"`
+}
+type Trigger_AfterProcessingTime_ struct {
+	AfterProcessingTime *Trigger_AfterProcessingTime `protobuf:"bytes,5,opt,name=after_processing_time,json=afterProcessingTime,oneof"`
+}
+type Trigger_AfterSynchronizedProcessingTime_ struct {
+	AfterSynchronizedProcessingTime *Trigger_AfterSynchronizedProcessingTime `protobuf:"bytes,6,opt,name=after_synchronized_processing_time,json=afterSynchronizedProcessingTime,oneof"`
+}
+type Trigger_Always_ struct {
+	Always *Trigger_Always `protobuf:"bytes,12,opt,name=always,oneof"`
+}
+type Trigger_Default_ struct {
+	Default *Trigger_Default `protobuf:"bytes,7,opt,name=default,oneof"`
+}
+type Trigger_ElementCount_ struct {
+	ElementCount *Trigger_ElementCount `protobuf:"bytes,8,opt,name=element_count,json=elementCount,oneof"`
+}
+type Trigger_Never_ struct {
+	Never *Trigger_Never `protobuf:"bytes,9,opt,name=never,oneof"`
+}
+type Trigger_OrFinally_ struct {
+	OrFinally *Trigger_OrFinally `protobuf:"bytes,10,opt,name=or_finally,json=orFinally,oneof"`
+}
+type Trigger_Repeat_ struct {
+	Repeat *Trigger_Repeat `protobuf:"bytes,11,opt,name=repeat,oneof"`
+}
+
+func (*Trigger_AfterAll_) isTrigger_Trigger()                        {}
+func (*Trigger_AfterAny_) isTrigger_Trigger()                        {}
+func (*Trigger_AfterEach_) isTrigger_Trigger()                       {}
+func (*Trigger_AfterEndOfWindow_) isTrigger_Trigger()                {}
+func (*Trigger_AfterProcessingTime_) isTrigger_Trigger()             {}
+func (*Trigger_AfterSynchronizedProcessingTime_) isTrigger_Trigger() {}
+func (*Trigger_Always_) isTrigger_Trigger()                          {}
+func (*Trigger_Default_) isTrigger_Trigger()                         {}
+func (*Trigger_ElementCount_) isTrigger_Trigger()                    {}
+func (*Trigger_Never_) isTrigger_Trigger()                           {}
+func (*Trigger_OrFinally_) isTrigger_Trigger()                       {}
+func (*Trigger_Repeat_) isTrigger_Trigger()                          {}
+
+func (m *Trigger) GetTrigger() isTrigger_Trigger {
+	if m != nil {
+		return m.Trigger
+	}
+	return nil
+}
+
+func (m *Trigger) GetAfterAll() *Trigger_AfterAll {
+	if x, ok := m.GetTrigger().(*Trigger_AfterAll_); ok {
+		return x.AfterAll
+	}
+	return nil
+}
+
+func (m *Trigger) GetAfterAny() *Trigger_AfterAny {
+	if x, ok := m.GetTrigger().(*Trigger_AfterAny_); ok {
+		return x.AfterAny
+	}
+	return nil
+}
+
+func (m *Trigger) GetAfterEach() *Trigger_AfterEach {
+	if x, ok := m.GetTrigger().(*Trigger_AfterEach_); ok {
+		return x.AfterEach
+	}
+	return nil
+}
+
+func (m *Trigger) GetAfterEndOfWindow() *Trigger_AfterEndOfWindow {
+	if x, ok := m.GetTrigger().(*Trigger_AfterEndOfWindow_); ok {
+		return x.AfterEndOfWindow
+	}
+	return nil
+}
+
+func (m *Trigger) GetAfterProcessingTime() *Trigger_AfterProcessingTime {
+	if x, ok := m.GetTrigger().(*Trigger_AfterProcessingTime_); ok {
+		return x.AfterProcessingTime
+	}
+	return nil
+}
+
+func (m *Trigger) GetAfterSynchronizedProcessingTime() *Trigger_AfterSynchronizedProcessingTime {
+	if x, ok := m.GetTrigger().(*Trigger_AfterSynchronizedProcessingTime_); ok {
+		return x.AfterSynchronizedProcessingTime
+	}
+	return nil
+}
+
+func (m *Trigger) GetAlways() *Trigger_Always {
+	if x, ok := m.GetTrigger().(*Trigger_Always_); ok {
+		return x.Always
+	}
+	return nil
+}
+
+func (m *Trigger) GetDefault() *Trigger_Default {
+	if x, ok := m.GetTrigger().(*Trigger_Default_); ok {
+		return x.Default
+	}
+	return nil
+}
+
+func (m *Trigger) GetElementCount() *Trigger_ElementCount {
+	if x, ok := m.GetTrigger().(*Trigger_ElementCount_); ok {
+		return x.ElementCount
+	}
+	return nil
+}
+
+func (m *Trigger) GetNever() *Trigger_Never {
+	if x, ok := m.GetTrigger().(*Trigger_Never_); ok {
+		return x.Never
+	}
+	return nil
+}
+
+func (m *Trigger) GetOrFinally() *Trigger_OrFinally {
+	if x, ok := m.GetTrigger().(*Trigger_OrFinally_); ok {
+		return x.OrFinally
+	}
+	return nil
+}
+
+func (m *Trigger) GetRepeat() *Trigger_Repeat {
+	if x, ok := m.GetTrigger().(*Trigger_Repeat_); ok {
+		return x.Repeat
+	}
+	return nil
+}
+
+// XXX_OneofFuncs is for the internal use of the proto package.
+func (*Trigger) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) error, func(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error), func(msg proto.Message) (n int), []interface{}) {
+	return _Trigger_OneofMarshaler, _Trigger_OneofUnmarshaler, _Trigger_OneofSizer, []interface{}{
+		(*Trigger_AfterAll_)(nil),
+		(*Trigger_AfterAny_)(nil),
+		(*Trigger_AfterEach_)(nil),
+		(*Trigger_AfterEndOfWindow_)(nil),
+		(*Trigger_AfterProcessingTime_)(nil),
+		(*Trigger_AfterSynchronizedProcessingTime_)(nil),
+		(*Trigger_Always_)(nil),
+		(*Trigger_Default_)(nil),
+		(*Trigger_ElementCount_)(nil),
+		(*Trigger_Never_)(nil),
+		(*Trigger_OrFinally_)(nil),
+		(*Trigger_Repeat_)(nil),
+	}
+}
+
+func _Trigger_OneofMarshaler(msg proto.Message, b *proto.Buffer) error {
+	m := msg.(*Trigger)
+	// trigger
+	switch x := m.Trigger.(type) {
+	case *Trigger_AfterAll_:
+		b.EncodeVarint(1<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.AfterAll); err != nil {
+			return err
+		}
+	case *Trigger_AfterAny_:
+		b.EncodeVarint(2<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.AfterAny); err != nil {
+			return err
+		}
+	case *Trigger_AfterEach_:
+		b.EncodeVarint(3<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.AfterEach); err != nil {
+			return err
+		}
+	case *Trigger_AfterEndOfWindow_:
+		b.EncodeVarint(4<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.AfterEndOfWindow); err != nil {
+			return err
+		}
+	case *Trigger_AfterProcessingTime_:
+		b.EncodeVarint(5<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.AfterProcessingTime); err != nil {
+			return err
+		}
+	case *Trigger_AfterSynchronizedProcessingTime_:
+		b.EncodeVarint(6<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.AfterSynchronizedProcessingTime); err != nil {
+			return err
+		}
+	case *Trigger_Always_:
+		b.EncodeVarint(12<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.Always); err != nil {
+			return err
+		}
+	case *Trigger_Default_:
+		b.EncodeVarint(7<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.Default); err != nil {
+			return err
+		}
+	case *Trigger_ElementCount_:
+		b.EncodeVarint(8<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.ElementCount); err != nil {
+			return err
+		}
+	case *Trigger_Never_:
+		b.EncodeVarint(9<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.Never); err != nil {
+			return err
+		}
+	case *Trigger_OrFinally_:
+		b.EncodeVarint(10<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.OrFinally); err != nil {
+			return err
+		}
+	case *Trigger_Repeat_:
+		b.EncodeVarint(11<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.Repeat); err != nil {
+			return err
+		}
+	case nil:
+	default:
+		return fmt.Errorf("Trigger.Trigger has unexpected type %T", x)
+	}
+	return nil
+}
+
+func _Trigger_OneofUnmarshaler(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error) {
+	m := msg.(*Trigger)
+	switch tag {
+	case 1: // trigger.after_all
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(Trigger_AfterAll)
+		err := b.DecodeMessage(msg)
+		m.Trigger = &Trigger_AfterAll_{msg}
+		return true, err
+	case 2: // trigger.after_any
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(Trigger_AfterAny)
+		err := b.DecodeMessage(msg)
+		m.Trigger = &Trigger_AfterAny_{msg}
+		return true, err
+	case 3: // trigger.after_each
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(Trigger_AfterEach)
+		err := b.DecodeMessage(msg)
+		m.Trigger = &Trigger_AfterEach_{msg}
+		return true, err
+	case 4: // trigger.after_end_of_window
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(Trigger_AfterEndOfWindow)
+		err := b.DecodeMessage(msg)
+		m.Trigger = &Trigger_AfterEndOfWindow_{msg}
+		return true, err
+	case 5: // trigger.after_processing_time
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(Trigger_AfterProcessingTime)
+		err := b.DecodeMessage(msg)
+		m.Trigger = &Trigger_AfterProcessingTime_{msg}
+		return true, err
+	case 6: // trigger.after_synchronized_processing_time
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(Trigger_AfterSynchronizedProcessingTime)
+		err := b.DecodeMessage(msg)
+		m.Trigger = &Trigger_AfterSynchronizedProcessingTime_{msg}
+		return true, err
+	case 12: // trigger.always
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(Trigger_Always)
+		err := b.DecodeMessage(msg)
+		m.Trigger = &Trigger_Always_{msg}
+		return true, err
+	case 7: // trigger.default
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(Trigger_Default)
+		err := b.DecodeMessage(msg)
+		m.Trigger = &Trigger_Default_{msg}
+		return true, err
+	case 8: // trigger.element_count
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(Trigger_ElementCount)
+		err := b.DecodeMessage(msg)
+		m.Trigger = &Trigger_ElementCount_{msg}
+		return true, err
+	case 9: // trigger.never
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(Trigger_Never)
+		err := b.DecodeMessage(msg)
+		m.Trigger = &Trigger_Never_{msg}
+		return true, err
+	case 10: // trigger.or_finally
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(Trigger_OrFinally)
+		err := b.DecodeMessage(msg)
+		m.Trigger = &Trigger_OrFinally_{msg}
+		return true, err
+	case 11: // trigger.repeat
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(Trigger_Repeat)
+		err := b.DecodeMessage(msg)
+		m.Trigger = &Trigger_Repeat_{msg}
+		return true, err
+	default:
+		return false, nil
+	}
+}
+
+func _Trigger_OneofSizer(msg proto.Message) (n int) {
+	m := msg.(*Trigger)
+	// trigger
+	switch x := m.Trigger.(type) {
+	case *Trigger_AfterAll_:
+		s := proto.Size(x.AfterAll)
+		n += proto.SizeVarint(1<<3 | proto.WireBytes)
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case *Trigger_AfterAny_:
+		s := proto.Size(x.AfterAny)
+		n += proto.SizeVarint(2<<3 | proto.WireBytes)
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case *Trigger_AfterEach_:
+		s := proto.Size(x.AfterEach)
+		n += proto.SizeVarint(3<<3 | proto.WireBytes)
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case *Trigger_AfterEndOfWindow_:
+		s := proto.Size(x.AfterEndOfWindow)
+		n += proto.SizeVarint(4<<3 | proto.WireBytes)
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case *Trigger_AfterProcessingTime_:
+		s := proto.Size(x.AfterProcessingTime)
+		n += proto.SizeVarint(5<<3 | proto.WireBytes)
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case *Trigger_AfterSynchronizedProcessingTime_:
+		s := proto.Size(x.AfterSynchronizedProcessingTime)
+		n += proto.SizeVarint(6<<3 | proto.WireBytes)
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case *Trigger_Always_:
+		s := proto.Size(x.Always)
+		n += proto.SizeVarint(12<<3 | proto.WireBytes)
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case *Trigger_Default_:
+		s := proto.Size(x.Default)
+		n += proto.SizeVarint(7<<3 | proto.WireBytes)
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case *Trigger_ElementCount_:
+		s := proto.Size(x.ElementCount)
+		n += proto.SizeVarint(8<<3 | proto.WireBytes)
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case *Trigger_Never_:
+		s := proto.Size(x.Never)
+		n += proto.SizeVarint(9<<3 | proto.WireBytes)
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case *Trigger_OrFinally_:
+		s := proto.Size(x.OrFinally)
+		n += proto.SizeVarint(10<<3 | proto.WireBytes)
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case *Trigger_Repeat_:
+		s := proto.Size(x.Repeat)
+		n += proto.SizeVarint(11<<3 | proto.WireBytes)
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case nil:
+	default:
+		panic(fmt.Sprintf("proto: unexpected type %T in oneof", x))
+	}
+	return n
+}
+
+// Ready when all subtriggers are ready.
+type Trigger_AfterAll struct {
+	Subtriggers []*Trigger `protobuf:"bytes,1,rep,name=subtriggers" json:"subtriggers,omitempty"`
+}
+
+func (m *Trigger_AfterAll) Reset()                    { *m = Trigger_AfterAll{} }
+func (m *Trigger_AfterAll) String() string            { return proto.CompactTextString(m) }
+func (*Trigger_AfterAll) ProtoMessage()               {}
+func (*Trigger_AfterAll) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{28, 0} }
+
+func (m *Trigger_AfterAll) GetSubtriggers() []*Trigger {
+	if m != nil {
+		return m.Subtriggers
+	}
+	return nil
+}
+
+// Ready when any subtrigger is ready.
+type Trigger_AfterAny struct {
+	Subtriggers []*Trigger `protobuf:"bytes,1,rep,name=subtriggers" json:"subtriggers,omitempty"`
+}
+
+func (m *Trigger_AfterAny) Reset()                    {

<TRUNCATED>

[14/50] [abbrv] beam git commit: Add License Header to SqlTypeUtils

Posted by ke...@apache.org.
Add License Header to SqlTypeUtils


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

Branch: refs/heads/mr-runner
Commit: d7329a036bb3e490b855b036c85664a92da58f7b
Parents: ee2292a
Author: Thomas Groh <tg...@google.com>
Authored: Thu Nov 2 09:51:47 2017 -0700
Committer: Thomas Groh <tg...@google.com>
Committed: Thu Nov 2 09:51:47 2017 -0700

----------------------------------------------------------------------
 .../extensions/sql/impl/utils/SqlTypeUtils.java    | 17 +++++++++++++++++
 1 file changed, 17 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/d7329a03/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/SqlTypeUtils.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/SqlTypeUtils.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/SqlTypeUtils.java
index 1ab703e..9658bab 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/SqlTypeUtils.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/SqlTypeUtils.java
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
 package org.apache.beam.sdk.extensions.sql.impl.utils;
 
 import com.google.common.base.Optional;


[18/50] [abbrv] beam git commit: Updates BigQueryTableSource to consider data in streaming buffer when determining estimated size.

Posted by ke...@apache.org.
Updates BigQueryTableSource to consider data in streaming buffer when determining estimated size.


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

Branch: refs/heads/mr-runner
Commit: b2c9fba4dd5f7f5a0ac0045f9ff8f30d55088a34
Parents: 482d178
Author: chamikara@google.com <ch...@google.com>
Authored: Sat Oct 21 19:20:07 2017 -0700
Committer: chamikara@google.com <ch...@google.com>
Committed: Thu Nov 2 15:20:27 2017 -0700

----------------------------------------------------------------------
 .../io/gcp/bigquery/BigQueryTableSource.java    | 10 ++-
 .../sdk/io/gcp/bigquery/BigQueryIOTest.java     | 81 +++++++++++++++++++-
 2 files changed, 88 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/b2c9fba4/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableSource.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableSource.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableSource.java
index f717cb7..dbac00f 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableSource.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableSource.java
@@ -21,6 +21,7 @@ package org.apache.beam.sdk.io.gcp.bigquery;
 import static com.google.common.base.Preconditions.checkNotNull;
 import static com.google.common.base.Preconditions.checkState;
 
+import com.google.api.services.bigquery.model.Table;
 import com.google.api.services.bigquery.model.TableReference;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Strings;
@@ -102,8 +103,13 @@ class BigQueryTableSource<T> extends BigQuerySourceBase<T> {
       TableReference table = setDefaultProjectIfAbsent(options.as(BigQueryOptions.class),
           BigQueryIO.JSON_FACTORY.fromString(jsonTable.get(), TableReference.class));
 
-      Long numBytes = bqServices.getDatasetService(options.as(BigQueryOptions.class))
-          .getTable(table).getNumBytes();
+      Table tableRef = bqServices.getDatasetService(options.as(BigQueryOptions.class))
+              .getTable(table);
+      Long numBytes = tableRef.getNumBytes();
+      if (tableRef.getStreamingBuffer() != null) {
+        numBytes += tableRef.getStreamingBuffer().getEstimatedBytes().longValue();
+      }
+
       tableSizeBytes.compareAndSet(null, numBytes);
     }
     return tableSizeBytes.get();

http://git-wip-us.apache.org/repos/asf/beam/blob/b2c9fba4/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java
index aa818c6..5b4b7e6 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java
@@ -41,6 +41,7 @@ import com.google.api.services.bigquery.model.JobStatistics;
 import com.google.api.services.bigquery.model.JobStatistics2;
 import com.google.api.services.bigquery.model.JobStatistics4;
 import com.google.api.services.bigquery.model.JobStatus;
+import com.google.api.services.bigquery.model.Streamingbuffer;
 import com.google.api.services.bigquery.model.Table;
 import com.google.api.services.bigquery.model.TableDataInsertAllResponse;
 import com.google.api.services.bigquery.model.TableFieldSchema;
@@ -64,6 +65,7 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.io.Serializable;
+import java.math.BigInteger;
 import java.nio.file.Files;
 import java.nio.file.Path;
 import java.nio.file.Paths;
@@ -343,7 +345,7 @@ public class BigQueryIOTest implements Serializable {
             }));
     PAssert.that(output).containsInAnyOrder(ImmutableList.of(KV.of("a", 1L), KV.of("b", 2L),
         KV.of("c", 3L), KV.of("d", 4L), KV.of("e", 5L), KV.of("f", 6L)));
-     p.run();
+    p.run();
   }
 
   @Test
@@ -1697,6 +1699,83 @@ public class BigQueryIOTest implements Serializable {
   }
 
   @Test
+  public void testEstimatedSizeWithoutStreamingBuffer() throws Exception {
+    FakeDatasetService fakeDatasetService = new FakeDatasetService();
+    FakeJobService fakeJobService = new FakeJobService();
+    FakeBigQueryServices fakeBqServices = new FakeBigQueryServices()
+            .withJobService(fakeJobService)
+            .withDatasetService(fakeDatasetService);
+
+    List<TableRow> data = ImmutableList.of(
+            new TableRow().set("name", "a").set("number", 1L),
+            new TableRow().set("name", "b").set("number", 2L),
+            new TableRow().set("name", "c").set("number", 3L),
+            new TableRow().set("name", "d").set("number", 4L),
+            new TableRow().set("name", "e").set("number", 5L),
+            new TableRow().set("name", "f").set("number", 6L));
+
+    TableReference table = BigQueryHelpers.parseTableSpec("project:data_set.table_name");
+    fakeDatasetService.createDataset("project", "data_set", "", "", null);
+    fakeDatasetService.createTable(new Table().setTableReference(table)
+            .setSchema(new TableSchema()
+                    .setFields(
+                            ImmutableList.of(
+                                    new TableFieldSchema().setName("name").setType("STRING"),
+                                    new TableFieldSchema().setName("number").setType("INTEGER")))));
+    fakeDatasetService.insertAll(table, data, null);
+
+    String stepUuid = "testStepUuid";
+    BoundedSource<TableRow> bqSource = BigQueryTableSource.create(
+            stepUuid,
+            StaticValueProvider.of(table),
+            fakeBqServices,
+            TableRowJsonCoder.of(),
+            BigQueryIO.TableRowParser.INSTANCE);
+
+    PipelineOptions options = PipelineOptionsFactory.create();
+    assertEquals(108, bqSource.getEstimatedSizeBytes(options));
+  }
+
+  @Test
+  public void testEstimatedSizeWithStreamingBuffer() throws Exception {
+    FakeDatasetService fakeDatasetService = new FakeDatasetService();
+    FakeJobService fakeJobService = new FakeJobService();
+    FakeBigQueryServices fakeBqServices = new FakeBigQueryServices()
+            .withJobService(fakeJobService)
+            .withDatasetService(fakeDatasetService);
+
+    List<TableRow> data = ImmutableList.of(
+            new TableRow().set("name", "a").set("number", 1L),
+            new TableRow().set("name", "b").set("number", 2L),
+            new TableRow().set("name", "c").set("number", 3L),
+            new TableRow().set("name", "d").set("number", 4L),
+            new TableRow().set("name", "e").set("number", 5L),
+            new TableRow().set("name", "f").set("number", 6L));
+
+    TableReference table = BigQueryHelpers.parseTableSpec("project:data_set.table_name");
+    fakeDatasetService.createDataset("project", "data_set", "", "", null);
+    fakeDatasetService.createTable(new Table().setTableReference(table)
+            .setSchema(new TableSchema()
+                    .setFields(
+                            ImmutableList.of(
+                                    new TableFieldSchema().setName("name").setType("STRING"),
+                                    new TableFieldSchema().setName("number").setType("INTEGER"))))
+            .setStreamingBuffer(new Streamingbuffer().setEstimatedBytes(BigInteger.valueOf(10))));
+    fakeDatasetService.insertAll(table, data, null);
+
+    String stepUuid = "testStepUuid";
+    BoundedSource<TableRow> bqSource = BigQueryTableSource.create(
+            stepUuid,
+            StaticValueProvider.of(table),
+            fakeBqServices,
+            TableRowJsonCoder.of(),
+            BigQueryIO.TableRowParser.INSTANCE);
+
+    PipelineOptions options = PipelineOptionsFactory.create();
+    assertEquals(118, bqSource.getEstimatedSizeBytes(options));
+  }
+
+  @Test
   public void testBigQueryQuerySourceInitSplit() throws Exception {
     TableReference dryRunTable = new TableReference();
 


[10/50] [abbrv] beam git commit: Fix working dir in website precommits

Posted by ke...@apache.org.
Fix working dir in website precommits


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

Branch: refs/heads/mr-runner
Commit: 7f9cfdc762a2d71eb9f0ad80f05c2c54744d5c99
Parents: 239319b
Author: Kenneth Knowles <ke...@apache.org>
Authored: Wed Nov 1 14:05:10 2017 -0700
Committer: Kenneth Knowles <ke...@apache.org>
Committed: Wed Nov 1 14:05:10 2017 -0700

----------------------------------------------------------------------
 .test-infra/jenkins/job_beam_PreCommit_Website_Merge.groovy | 3 +++
 .test-infra/jenkins/job_beam_PreCommit_Website_Stage.groovy | 3 +++
 .test-infra/jenkins/job_beam_PreCommit_Website_Test.groovy  | 3 +++
 3 files changed, 9 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/7f9cfdc7/.test-infra/jenkins/job_beam_PreCommit_Website_Merge.groovy
----------------------------------------------------------------------
diff --git a/.test-infra/jenkins/job_beam_PreCommit_Website_Merge.groovy b/.test-infra/jenkins/job_beam_PreCommit_Website_Merge.groovy
index 0e2ae3f..f386d85 100644
--- a/.test-infra/jenkins/job_beam_PreCommit_Website_Merge.groovy
+++ b/.test-infra/jenkins/job_beam_PreCommit_Website_Merge.groovy
@@ -48,6 +48,9 @@ job('beam_PreCommit_Website_Merge') {
         GEM_PATH=~/.gem/ruby/$RUBY_VERSION_NUM/:$GEM_PATH
         gem install bundler --user-install
 
+        # Enter the git clone for remaining commands
+        cd src
+
         # Install all needed gems.
         bundle install --path ~/.gem/
 

http://git-wip-us.apache.org/repos/asf/beam/blob/7f9cfdc7/.test-infra/jenkins/job_beam_PreCommit_Website_Stage.groovy
----------------------------------------------------------------------
diff --git a/.test-infra/jenkins/job_beam_PreCommit_Website_Stage.groovy b/.test-infra/jenkins/job_beam_PreCommit_Website_Stage.groovy
index 7c64f11..0b4d738 100644
--- a/.test-infra/jenkins/job_beam_PreCommit_Website_Stage.groovy
+++ b/.test-infra/jenkins/job_beam_PreCommit_Website_Stage.groovy
@@ -56,6 +56,9 @@ job('beam_PreCommit_Website_Stage') {
         GEM_PATH=~/.gem/ruby/$RUBY_VERSION_NUM/:$GEM_PATH
         gem install bundler --user-install
 
+        # Enter the git clone for remaining commands
+        cd src
+
         # Install all needed gems.
         bundle install --path ~/.gem/
 

http://git-wip-us.apache.org/repos/asf/beam/blob/7f9cfdc7/.test-infra/jenkins/job_beam_PreCommit_Website_Test.groovy
----------------------------------------------------------------------
diff --git a/.test-infra/jenkins/job_beam_PreCommit_Website_Test.groovy b/.test-infra/jenkins/job_beam_PreCommit_Website_Test.groovy
index 421b58a..9b0aa74 100644
--- a/.test-infra/jenkins/job_beam_PreCommit_Website_Test.groovy
+++ b/.test-infra/jenkins/job_beam_PreCommit_Website_Test.groovy
@@ -54,6 +54,9 @@ job('beam_PreCommit_Website_Test') {
         GEM_PATH=~/.gem/ruby/$RUBY_VERSION_NUM/:$GEM_PATH
         gem install bundler --user-install
 
+        # Enter the git clone for remaining commands
+        cd src
+
         # Install all needed gems.
         bundle install --path ~/.gem/
 


[27/50] [abbrv] beam git commit: This closes #3991: [BEAM-2728] Extension for sketch-based statistics : HyperLogLog

Posted by ke...@apache.org.
This closes #3991: [BEAM-2728] Extension for sketch-based statistics : HyperLogLog


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

Branch: refs/heads/mr-runner
Commit: e25d217f7f5272f7ec54a85dbe67f536a09eb460
Parents: 4f4632c fd58a42
Author: Eugene Kirpichov <ek...@gmail.com>
Authored: Fri Nov 3 15:24:54 2017 -0700
Committer: Eugene Kirpichov <ek...@gmail.com>
Committed: Fri Nov 3 15:24:54 2017 -0700

----------------------------------------------------------------------
 pom.xml                                         |   6 +
 sdks/java/extensions/pom.xml                    |   1 +
 sdks/java/extensions/sketching/pom.xml          | 104 ++++
 .../sketching/ApproximateDistinct.java          | 573 +++++++++++++++++++
 .../sdk/extensions/sketching/package-info.java  |  22 +
 .../sketching/ApproximateDistinctTest.java      | 209 +++++++
 sdks/java/javadoc/pom.xml                       |   5 +
 7 files changed, 920 insertions(+)
----------------------------------------------------------------------



[16/50] [abbrv] beam git commit: Getting AutoValue 1.5.1 working in Beam.

Posted by ke...@apache.org.
Getting AutoValue 1.5.1 working in Beam.


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

Branch: refs/heads/mr-runner
Commit: a63345bb674e61c40a681c31c3aace6d6448b4f3
Parents: 5aaff7b
Author: Daniel Oliveira <da...@gmail.com>
Authored: Tue Oct 24 14:34:38 2017 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Thu Nov 2 14:05:37 2017 -0700

----------------------------------------------------------------------
 examples/java/pom.xml     | 6 ++++++
 pom.xml                   | 8 +++++++-
 sdks/java/nexmark/pom.xml | 6 ++++++
 3 files changed, 19 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/a63345bb/examples/java/pom.xml
----------------------------------------------------------------------
diff --git a/examples/java/pom.xml b/examples/java/pom.xml
index dd95d02..e47e9a1 100644
--- a/examples/java/pom.xml
+++ b/examples/java/pom.xml
@@ -491,6 +491,12 @@
       <optional>true</optional>
     </dependency>
 
+    <dependency>
+      <groupId>com.google.auto.value</groupId>
+      <artifactId>auto-value</artifactId>
+      <scope>provided</scope>
+    </dependency>
+
     <!-- Hamcrest and JUnit are required dependencies of PAssert,
          which is used in the main code of DebuggingWordCount example. -->
 

http://git-wip-us.apache.org/repos/asf/beam/blob/a63345bb/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 3cb3b11..fcd0339 100644
--- a/pom.xml
+++ b/pom.xml
@@ -119,7 +119,7 @@
     <datastore.proto.version>1.3.0</datastore.proto.version>
     <google-api-common.version>1.0.0-rc2</google-api-common.version>
     <google-auto-service.version>1.0-rc2</google-auto-service.version>
-    <google-auto-value.version>1.4.1</google-auto-value.version>
+    <google-auto-value.version>1.5.1</google-auto-value.version>
     <google-auth.version>0.7.1</google-auth.version>
     <google-clients.version>1.22.0</google-clients.version>
     <google-cloud-bigdataoss.version>1.4.5</google-cloud-bigdataoss.version>
@@ -1992,6 +1992,11 @@
               <!-- Ignore runtime-only dependencies in analysis -->
               <ignoreNonCompile>true</ignoreNonCompile>
               <failOnWarning>true</failOnWarning>
+
+              <!-- ignore jsr305 for both "used but undeclared" and "declared but unused" -->
+              <ignoredDependencies>
+                <ignoredDependency>com.google.code.findbugs:jsr305</ignoredDependency>
+              </ignoredDependencies>
             </configuration>
           </execution>
         </executions>
@@ -2016,6 +2021,7 @@
                       version. Is not shaded, so safe to ignore.
                     -->
                     <exclude>jdk.tools:jdk.tools</exclude>
+                    <exclude>com.google.auto.value:auto-value</exclude>
                   </excludes>
                 </enforceBytecodeVersion>
                 <requireJavaVersion>

http://git-wip-us.apache.org/repos/asf/beam/blob/a63345bb/sdks/java/nexmark/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/nexmark/pom.xml b/sdks/java/nexmark/pom.xml
index 05163dc..8210ddc 100644
--- a/sdks/java/nexmark/pom.xml
+++ b/sdks/java/nexmark/pom.xml
@@ -255,5 +255,11 @@
       <artifactId>beam-runners-direct-java</artifactId>
       <scope>test</scope>
     </dependency>
+
+    <dependency>
+      <groupId>com.google.auto.value</groupId>
+      <artifactId>auto-value</artifactId>
+      <scope>provided</scope>
+    </dependency>
   </dependencies>
 </project>


[48/50] [abbrv] beam git commit: mr-runner: update to 2.3.0-SNAPSHOT.

Posted by ke...@apache.org.
mr-runner: update to 2.3.0-SNAPSHOT.


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

Branch: refs/heads/mr-runner
Commit: f63cecd40357b19fe7baa48d7bad9158de1faeba
Parents: 7181d0c
Author: Pei He <pe...@apache.org>
Authored: Tue Nov 7 16:50:52 2017 +0800
Committer: Pei He <pe...@apache.org>
Committed: Tue Nov 7 16:54:21 2017 +0800

----------------------------------------------------------------------
 runners/map-reduce/pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/f63cecd4/runners/map-reduce/pom.xml
----------------------------------------------------------------------
diff --git a/runners/map-reduce/pom.xml b/runners/map-reduce/pom.xml
index 90d876b..000f20c 100644
--- a/runners/map-reduce/pom.xml
+++ b/runners/map-reduce/pom.xml
@@ -20,7 +20,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-runners-parent</artifactId>
-    <version>2.2.0-SNAPSHOT</version>
+    <version>2.3.0-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
   


[49/50] [abbrv] beam git commit: mr-runner: re-enable sdks/python module.

Posted by ke...@apache.org.
mr-runner: re-enable sdks/python module.


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

Branch: refs/heads/mr-runner
Commit: 202a2cb242df8ce8b11ff19c7d8307991d61af3a
Parents: f63cecd
Author: Pei He <pe...@apache.org>
Authored: Tue Nov 7 17:15:12 2017 +0800
Committer: Pei He <pe...@apache.org>
Committed: Tue Nov 7 17:15:12 2017 +0800

----------------------------------------------------------------------
 sdks/pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/202a2cb2/sdks/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/pom.xml b/sdks/pom.xml
index f0afdad..7c85489 100644
--- a/sdks/pom.xml
+++ b/sdks/pom.xml
@@ -35,7 +35,7 @@
   <modules>
     <module>go</module>
     <module>java</module>
-    <!--<module>python</module>-->
+    <module>python</module>
   </modules>
 
   <profiles>


[43/50] [abbrv] beam git commit: Added a preprocessing step to the Cloud Spanner sink.

Posted by ke...@apache.org.
Added a preprocessing step to the Cloud Spanner sink.

The general intuition we follow here: if mutations are presorted by the primary key before batching, it is more likely that mutations in the batch will end up in the same partition. It minimizes the number of participants in the distributed transaction on the Cloud Spanner side and leads to a better throughput.

Mutations are encoded before running other steps to avoid paying the serialization price. Primary keys are encoded using OrderedCode library, and ApproximateQuantiles transform is used to sample keys.

Once primary keys are sampled, for each mutation we assign the index of the closest primary key as a key and group by that key. Range deletes are submitted separately.


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

Branch: refs/heads/mr-runner
Commit: 227801b31e31294c18dcc28bf98078f0868c41b0
Parents: 35952f6
Author: Mairbek Khadikov <ma...@google.com>
Authored: Fri Sep 29 16:43:05 2017 -0700
Committer: chamikara@google.com <ch...@google.com>
Committed: Mon Nov 6 17:22:54 2017 -0800

----------------------------------------------------------------------
 .../sdk/io/gcp/spanner/NaiveSpannerReadFn.java  |   1 +
 .../sdk/io/gcp/spanner/SerializedMutation.java  |  35 ++
 .../io/gcp/spanner/SerializedMutationCoder.java |  60 +++
 .../beam/sdk/io/gcp/spanner/SpannerIO.java      | 309 ++++++++++++-
 .../sdk/io/gcp/spanner/SpannerWriteGroupFn.java | 133 ------
 .../sdk/io/gcp/spanner/SpannerIOWriteTest.java  | 447 ++++++++++++++-----
 .../beam/sdk/io/gcp/spanner/SpannerWriteIT.java |   5 +-
 7 files changed, 746 insertions(+), 244 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/227801b3/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/NaiveSpannerReadFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/NaiveSpannerReadFn.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/NaiveSpannerReadFn.java
index 5dc6ead..34996f1 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/NaiveSpannerReadFn.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/NaiveSpannerReadFn.java
@@ -48,6 +48,7 @@ class NaiveSpannerReadFn extends DoFn<ReadOperation, Struct> {
   public void setup() throws Exception {
     spannerAccessor = config.connectToSpanner();
   }
+
   @Teardown
   public void teardown() throws Exception {
     spannerAccessor.close();

http://git-wip-us.apache.org/repos/asf/beam/blob/227801b3/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SerializedMutation.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SerializedMutation.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SerializedMutation.java
new file mode 100644
index 0000000..a5bebce
--- /dev/null
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SerializedMutation.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.io.gcp.spanner;
+
+import com.google.auto.value.AutoValue;
+
+@AutoValue
+abstract class SerializedMutation {
+  static SerializedMutation create(String tableName, byte[] key,
+      byte[] bytes) {
+    return new AutoValue_SerializedMutation(tableName, key, bytes);
+  }
+
+  abstract String getTableName();
+
+  abstract byte[] getEncodedKey();
+
+  abstract byte[] getMutationGroupBytes();
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/227801b3/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SerializedMutationCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SerializedMutationCoder.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SerializedMutationCoder.java
new file mode 100644
index 0000000..33ec1ed
--- /dev/null
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SerializedMutationCoder.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.io.gcp.spanner;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.ByteArrayCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+
+class SerializedMutationCoder extends AtomicCoder<SerializedMutation> {
+
+  private static final SerializedMutationCoder INSTANCE = new SerializedMutationCoder();
+
+  public static SerializedMutationCoder of() {
+    return INSTANCE;
+  }
+
+  private final ByteArrayCoder byteArrayCoder;
+  private final StringUtf8Coder stringCoder;
+
+  private SerializedMutationCoder() {
+    byteArrayCoder = ByteArrayCoder.of();
+    stringCoder = StringUtf8Coder.of();
+  }
+
+  @Override
+  public void encode(SerializedMutation value, OutputStream out)
+      throws IOException {
+    stringCoder.encode(value.getTableName(), out);
+    byteArrayCoder.encode(value.getEncodedKey(), out);
+    byteArrayCoder.encode(value.getMutationGroupBytes(), out);
+  }
+
+  @Override
+  public SerializedMutation decode(InputStream in)
+      throws IOException {
+    String tableName =  stringCoder.decode(in);
+    byte[] encodedKey = byteArrayCoder.decode(in);
+    byte[] mutationBytes = byteArrayCoder.decode(in);
+    return SerializedMutation.create(tableName, encodedKey, mutationBytes);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/227801b3/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java
index be4417b..530c466 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java
@@ -23,6 +23,8 @@ import static com.google.common.base.Preconditions.checkNotNull;
 import com.google.auto.value.AutoValue;
 import com.google.cloud.ServiceFactory;
 import com.google.cloud.Timestamp;
+import com.google.cloud.spanner.AbortedException;
+import com.google.cloud.spanner.Key;
 import com.google.cloud.spanner.KeySet;
 import com.google.cloud.spanner.Mutation;
 import com.google.cloud.spanner.Spanner;
@@ -31,23 +33,41 @@ import com.google.cloud.spanner.Statement;
 import com.google.cloud.spanner.Struct;
 import com.google.cloud.spanner.TimestampBound;
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Iterables;
+import com.google.common.primitives.UnsignedBytes;
+import java.io.Serializable;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
+import java.util.Comparator;
 import java.util.List;
+import java.util.Map;
+import java.util.UUID;
 import javax.annotation.Nullable;
 import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
 import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.transforms.ApproximateQuantiles;
+import org.apache.beam.sdk.transforms.Combine;
 import org.apache.beam.sdk.transforms.Create;
 import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.transforms.Reshuffle;
 import org.apache.beam.sdk.transforms.View;
 import org.apache.beam.sdk.transforms.display.DisplayData;
+import org.apache.beam.sdk.util.BackOff;
+import org.apache.beam.sdk.util.BackOffUtils;
+import org.apache.beam.sdk.util.FluentBackoff;
+import org.apache.beam.sdk.util.Sleeper;
+import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PBegin;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.PDone;
+import org.joda.time.Duration;
 
 /**
  * Experimental {@link PTransform Transforms} for reading from and writing to <a
@@ -159,6 +179,12 @@ import org.apache.beam.sdk.values.PDone;
 public class SpannerIO {
 
   private static final long DEFAULT_BATCH_SIZE_BYTES = 1024 * 1024; // 1 MB
+  // Max number of mutations to batch together.
+  private static final int MAX_NUM_MUTATIONS = 10000;
+  // The maximum number of keys to fit in memory when computing approximate quantiles.
+  private static final long MAX_NUM_KEYS = (long) 1e6;
+  // TODO calculate number of samples based on the size of the input.
+  private static final int DEFAULT_NUM_SAMPLES = 1000;
 
   /**
    * Creates an uninitialized instance of {@link Read}. Before use, the {@link Read} must be
@@ -208,6 +234,7 @@ public class SpannerIO {
     return new AutoValue_SpannerIO_Write.Builder()
         .setSpannerConfig(SpannerConfig.create())
         .setBatchSizeBytes(DEFAULT_BATCH_SIZE_BYTES)
+        .setNumSamples(DEFAULT_NUM_SAMPLES)
         .build();
   }
 
@@ -572,6 +599,12 @@ public class SpannerIO {
 
     abstract long getBatchSizeBytes();
 
+    abstract int getNumSamples();
+
+    @Nullable
+     abstract PTransform<PCollection<KV<String, byte[]>>, PCollection<KV<String, List<byte[]>>>>
+         getSampler();
+
     abstract Builder toBuilder();
 
     @AutoValue.Builder
@@ -581,6 +614,12 @@ public class SpannerIO {
 
       abstract Builder setBatchSizeBytes(long batchSizeBytes);
 
+      abstract Builder setNumSamples(int numSamples);
+
+      abstract Builder setSampler(
+          PTransform<PCollection<KV<String, byte[]>>, PCollection<KV<String, List<byte[]>>>>
+              sampler);
+
       abstract Write build();
     }
 
@@ -634,6 +673,13 @@ public class SpannerIO {
       return withSpannerConfig(config.withServiceFactory(serviceFactory));
     }
 
+    @VisibleForTesting
+    Write withSampler(
+        PTransform<PCollection<KV<String, byte[]>>, PCollection<KV<String, List<byte[]>>>>
+            sampler) {
+      return toBuilder().setSampler(sampler).build();
+    }
+
     /**
      * Same transform but can be applied to {@link PCollection} of {@link MutationGroup}.
      */
@@ -652,11 +698,10 @@ public class SpannerIO {
 
       input
           .apply("To mutation group", ParDo.of(new ToMutationGroupFn()))
-          .apply("Write mutations to Cloud Spanner", ParDo.of(new SpannerWriteGroupFn(this)));
+          .apply("Write mutations to Cloud Spanner", new WriteGrouped(this));
       return PDone.in(input.getPipeline());
     }
 
-
     @Override
     public void populateDisplayData(DisplayData.Builder builder) {
       super.populateDisplayData(builder);
@@ -666,6 +711,19 @@ public class SpannerIO {
     }
   }
 
+  /**
+   * A singleton that wraps {@code UnsignedBytes#lexicographicalComparator} which unfortunately
+   * is not serializable.
+   */
+  @VisibleForTesting
+  enum SerializableBytesComparator implements Comparator<byte[]>, Serializable {
+    INSTANCE {
+      @Override public int compare(byte[] a, byte[] b) {
+        return UnsignedBytes.lexicographicalComparator().compare(a, b);
+      }
+    }
+  }
+
   /** Same as {@link Write} but supports grouped mutations. */
   public static class WriteGrouped extends PTransform<PCollection<MutationGroup>, PDone> {
     private final Write spec;
@@ -674,9 +732,53 @@ public class SpannerIO {
       this.spec = spec;
     }
 
-    @Override public PDone expand(PCollection<MutationGroup> input) {
-      input.apply("Write mutations to Cloud Spanner", ParDo.of(new SpannerWriteGroupFn(spec)));
+    @Override
+    public PDone expand(PCollection<MutationGroup> input) {
+      PTransform<PCollection<KV<String, byte[]>>, PCollection<KV<String, List<byte[]>>>>
+          sampler = spec.getSampler();
+      if (sampler == null) {
+        sampler = createDefaultSampler();
+      }
+      // First, read the Cloud Spanner schema.
+      final PCollectionView<SpannerSchema> schemaView = input.getPipeline()
+          .apply(Create.of((Void) null))
+          .apply("Read information schema",
+              ParDo.of(new ReadSpannerSchema(spec.getSpannerConfig())))
+          .apply("Schema View", View.<SpannerSchema>asSingleton());
+
+      // Serialize mutations, we don't need to encode/decode them while reshuffling.
+      // The primary key is encoded via OrderedCode so we can calculate quantiles.
+      PCollection<SerializedMutation> serialized = input
+          .apply("Serialize mutations",
+              ParDo.of(new SerializeMutationsFn(schemaView)).withSideInputs(schemaView))
+          .setCoder(SerializedMutationCoder.of());
+
+      // Sample primary keys using ApproximateQuantiles.
+      PCollectionView<Map<String, List<byte[]>>> keySample = serialized
+          .apply("Extract keys", ParDo.of(new ExtractKeys()))
+          .apply("Sample keys", sampler)
+          .apply("Keys sample as view", View.<String, List<byte[]>>asMap());
+
+      // Assign partition based on the closest element in the sample and group mutations.
+      AssignPartitionFn assignPartitionFn = new AssignPartitionFn(keySample);
+      serialized
+          .apply("Partition input", ParDo.of(assignPartitionFn).withSideInputs(keySample))
+          .setCoder(KvCoder.of(StringUtf8Coder.of(), SerializedMutationCoder.of()))
+          .apply("Group by partition", GroupByKey.<String, SerializedMutation>create())
+          .apply("Batch mutations together",
+              ParDo.of(new BatchFn(spec.getBatchSizeBytes(), spec.getSpannerConfig(), schemaView))
+                  .withSideInputs(schemaView))
+          .apply("Write mutations to Spanner",
+          ParDo.of(new WriteToSpannerFn(spec.getSpannerConfig())));
       return PDone.in(input.getPipeline());
+
+    }
+
+    private PTransform<PCollection<KV<String, byte[]>>, PCollection<KV<String, List<byte[]>>>>
+        createDefaultSampler() {
+      return Combine.perKey(ApproximateQuantiles.ApproximateQuantilesCombineFn
+          .create(spec.getNumSamples(), SerializableBytesComparator.INSTANCE, MAX_NUM_KEYS,
+              1. / spec.getNumSamples()));
     }
   }
 
@@ -688,5 +790,202 @@ public class SpannerIO {
     }
   }
 
-  private SpannerIO() {} // Prevent construction.
+  /**
+   * Serializes mutations to ((table name, serialized key), serialized value) tuple.
+   */
+  private static class SerializeMutationsFn
+      extends DoFn<MutationGroup, SerializedMutation> {
+
+    final PCollectionView<SpannerSchema> schemaView;
+
+    private SerializeMutationsFn(PCollectionView<SpannerSchema> schemaView) {
+      this.schemaView = schemaView;
+    }
+
+    @ProcessElement
+    public void processElement(ProcessContext c) {
+      MutationGroup g = c.element();
+      Mutation m = g.primary();
+      SpannerSchema schema = c.sideInput(schemaView);
+      String table = m.getTable();
+      MutationGroupEncoder mutationGroupEncoder = new MutationGroupEncoder(schema);
+
+      byte[] key;
+      if (m.getOperation() != Mutation.Op.DELETE) {
+        key = mutationGroupEncoder.encodeKey(m);
+      } else if (isPointDelete(m)) {
+        Key next = m.getKeySet().getKeys().iterator().next();
+        key = mutationGroupEncoder.encodeKey(m.getTable(), next);
+      } else {
+        // The key is left empty for non-point deletes, since there is no general way to batch them.
+        key = new byte[] {};
+      }
+      byte[] value = mutationGroupEncoder.encode(g);
+      c.output(SerializedMutation.create(table, key, value));
+    }
+  }
+
+  private static class ExtractKeys
+      extends DoFn<SerializedMutation, KV<String, byte[]>> {
+
+    @ProcessElement
+    public void processElement(ProcessContext c) {
+      SerializedMutation m = c.element();
+      c.output(KV.of(m.getTableName(), m.getEncodedKey()));
+    }
+  }
+
+
+
+  private static boolean isPointDelete(Mutation m) {
+    return m.getOperation() == Mutation.Op.DELETE && Iterables.isEmpty(m.getKeySet().getRanges())
+        && Iterables.size(m.getKeySet().getKeys()) == 1;
+  }
+
+  /**
+   * Assigns a partition to the mutation group token based on the sampled data.
+   */
+  private static class AssignPartitionFn
+      extends DoFn<SerializedMutation, KV<String, SerializedMutation>> {
+
+    final PCollectionView<Map<String, List<byte[]>>> sampleView;
+
+    public AssignPartitionFn(PCollectionView<Map<String, List<byte[]>>> sampleView) {
+      this.sampleView = sampleView;
+    }
+
+    @ProcessElement public void processElement(ProcessContext c) {
+      Map<String, List<byte[]>> sample = c.sideInput(sampleView);
+      SerializedMutation g = c.element();
+      String table = g.getTableName();
+      byte[] key = g.getEncodedKey();
+      String groupKey;
+      if (key.length == 0) {
+        // This is a range or multi-key delete mutation. We cannot group it with other mutations
+        // so we assign a random group key to it so it is applied independently.
+        groupKey = UUID.randomUUID().toString();
+      } else {
+        int partition = Collections
+            .binarySearch(sample.get(table), key, SerializableBytesComparator.INSTANCE);
+        if (partition < 0) {
+          partition = -partition - 1;
+        }
+        groupKey = table + "%" + partition;
+      }
+      c.output(KV.of(groupKey, g));
+    }
+  }
+
+  /**
+   * Batches mutations together.
+   */
+  private static class BatchFn
+      extends DoFn<KV<String, Iterable<SerializedMutation>>, Iterable<Mutation>> {
+
+    private static final int MAX_RETRIES = 5;
+    private static final FluentBackoff BUNDLE_WRITE_BACKOFF = FluentBackoff.DEFAULT
+        .withMaxRetries(MAX_RETRIES).withInitialBackoff(Duration.standardSeconds(5));
+
+    private final long maxBatchSizeBytes;
+    private final SpannerConfig spannerConfig;
+    private final PCollectionView<SpannerSchema> schemaView;
+
+    private transient SpannerAccessor spannerAccessor;
+    // Current batch of mutations to be written.
+    private List<Mutation> mutations;
+    // total size of the current batch.
+    private long batchSizeBytes;
+
+    private BatchFn(long maxBatchSizeBytes, SpannerConfig spannerConfig,
+        PCollectionView<SpannerSchema> schemaView) {
+      this.maxBatchSizeBytes = maxBatchSizeBytes;
+      this.spannerConfig = spannerConfig;
+      this.schemaView = schemaView;
+    }
+
+    @Setup
+    public void setup() throws Exception {
+      mutations = new ArrayList<>();
+      batchSizeBytes = 0;
+      spannerAccessor = spannerConfig.connectToSpanner();
+    }
+
+    @Teardown
+    public void teardown() throws Exception {
+      spannerAccessor.close();
+    }
+
+    @ProcessElement
+    public void processElement(ProcessContext c) throws Exception {
+      MutationGroupEncoder mutationGroupEncoder = new MutationGroupEncoder(c.sideInput(schemaView));
+      KV<String, Iterable<SerializedMutation>> element = c.element();
+      for (SerializedMutation kv : element.getValue()) {
+        byte[] value = kv.getMutationGroupBytes();
+        MutationGroup mg = mutationGroupEncoder.decode(value);
+        Iterables.addAll(mutations, mg);
+        batchSizeBytes += MutationSizeEstimator.sizeOf(mg);
+        if (batchSizeBytes >= maxBatchSizeBytes || mutations.size() > MAX_NUM_MUTATIONS) {
+          c.output(mutations);
+          mutations = new ArrayList<>();
+          batchSizeBytes = 0;
+        }
+      }
+      if (!mutations.isEmpty()) {
+        c.output(mutations);
+        mutations = new ArrayList<>();
+        batchSizeBytes = 0;
+      }
+    }
+  }
+
+  private static class WriteToSpannerFn
+      extends DoFn<Iterable<Mutation>, Void> {
+    private static final int MAX_RETRIES = 5;
+    private static final FluentBackoff BUNDLE_WRITE_BACKOFF = FluentBackoff.DEFAULT
+        .withMaxRetries(MAX_RETRIES).withInitialBackoff(Duration.standardSeconds(5));
+
+    private transient SpannerAccessor spannerAccessor;
+    private final SpannerConfig spannerConfig;
+
+    public WriteToSpannerFn(SpannerConfig spannerConfig) {
+      this.spannerConfig = spannerConfig;
+    }
+
+    @Setup
+    public void setup() throws Exception {
+      spannerAccessor = spannerConfig.connectToSpanner();
+    }
+
+    @Teardown
+    public void teardown() throws Exception {
+      spannerAccessor.close();
+    }
+
+
+    @ProcessElement
+    public void processElement(ProcessContext c) throws Exception {
+      Sleeper sleeper = Sleeper.DEFAULT;
+      BackOff backoff = BUNDLE_WRITE_BACKOFF.backoff();
+
+      Iterable<Mutation> mutations = c.element();
+
+      while (true) {
+        // Batch upsert rows.
+        try {
+          spannerAccessor.getDatabaseClient().writeAtLeastOnce(mutations);
+          // Break if the commit threw no exception.
+          break;
+        } catch (AbortedException exception) {
+          // Only log the code and message for potentially-transient errors. The entire exception
+          // will be propagated upon the last retry.
+          if (!BackOffUtils.next(sleeper, backoff)) {
+            throw exception;
+          }
+        }
+      }
+    }
+
+  }
+
+    private SpannerIO() {} // Prevent construction.
 }

http://git-wip-us.apache.org/repos/asf/beam/blob/227801b3/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerWriteGroupFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerWriteGroupFn.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerWriteGroupFn.java
deleted file mode 100644
index 9343c0c..0000000
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerWriteGroupFn.java
+++ /dev/null
@@ -1,133 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.sdk.io.gcp.spanner;
-
-import com.google.cloud.spanner.AbortedException;
-import com.google.cloud.spanner.DatabaseClient;
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.Iterables;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.display.DisplayData;
-import org.apache.beam.sdk.util.BackOff;
-import org.apache.beam.sdk.util.BackOffUtils;
-import org.apache.beam.sdk.util.FluentBackoff;
-import org.apache.beam.sdk.util.Sleeper;
-import org.joda.time.Duration;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/** Batches together and writes mutations to Google Cloud Spanner. */
-@VisibleForTesting
-class SpannerWriteGroupFn extends DoFn<MutationGroup, Void> {
-  private static final Logger LOG = LoggerFactory.getLogger(SpannerWriteGroupFn.class);
-  private final SpannerIO.Write spec;
-  // Current batch of mutations to be written.
-  private List<MutationGroup> mutations;
-  private long batchSizeBytes = 0;
-
-  private static final int MAX_RETRIES = 5;
-  private static final FluentBackoff BUNDLE_WRITE_BACKOFF =
-      FluentBackoff.DEFAULT
-          .withMaxRetries(MAX_RETRIES)
-          .withInitialBackoff(Duration.standardSeconds(5));
-
-  private transient SpannerAccessor spannerAccessor;
-
-  @VisibleForTesting
-  SpannerWriteGroupFn(SpannerIO.Write spec) {
-    this.spec = spec;
-  }
-
-  @Setup
-  public void setup() throws Exception {
-    spannerAccessor = spec.getSpannerConfig().connectToSpanner();
-    mutations = new ArrayList<>();
-    batchSizeBytes = 0;
-  }
-
-  @Teardown
-  public void teardown() throws Exception {
-    spannerAccessor.close();
-  }
-
-  @ProcessElement
-  public void processElement(ProcessContext c) throws Exception {
-    MutationGroup m = c.element();
-    mutations.add(m);
-    batchSizeBytes += MutationSizeEstimator.sizeOf(m);
-    if (batchSizeBytes >= spec.getBatchSizeBytes()) {
-      flushBatch();
-    }
-  }
-
-  @FinishBundle
-  public void finishBundle() throws Exception {
-    if (!mutations.isEmpty()) {
-      flushBatch();
-    }
-  }
-
-  /**
-   * Writes a batch of mutations to Cloud Spanner.
-   *
-   * <p>If a commit fails, it will be retried up to {@link #MAX_RETRIES} times. If the retry limit
-   * is exceeded, the last exception from Cloud Spanner will be thrown.
-   *
-   * @throws AbortedException if the commit fails or IOException or InterruptedException if
-   *     backing off between retries fails.
-   */
-  private void flushBatch() throws AbortedException, IOException, InterruptedException {
-    LOG.debug("Writing batch of {} mutations", mutations.size());
-    Sleeper sleeper = Sleeper.DEFAULT;
-    BackOff backoff = BUNDLE_WRITE_BACKOFF.backoff();
-
-    DatabaseClient databaseClient = spannerAccessor.getDatabaseClient();
-    while (true) {
-      // Batch upsert rows.
-      try {
-        databaseClient.writeAtLeastOnce(Iterables.concat(mutations));
-
-        // Break if the commit threw no exception.
-        break;
-      } catch (AbortedException exception) {
-        // Only log the code and message for potentially-transient errors. The entire exception
-        // will be propagated upon the last retry.
-        LOG.error(
-            "Error writing to Spanner ({}): {}", exception.getCode(), exception.getMessage());
-        if (!BackOffUtils.next(sleeper, backoff)) {
-          LOG.error("Aborting after {} retries.", MAX_RETRIES);
-          throw exception;
-        }
-      }
-    }
-    LOG.debug("Successfully wrote {} mutations", mutations.size());
-    mutations = new ArrayList<>();
-    batchSizeBytes = 0;
-  }
-
-  @Override
-  public void populateDisplayData(DisplayData.Builder builder) {
-    super.populateDisplayData(builder);
-    spec.populateDisplayData(builder);
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/227801b3/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIOWriteTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIOWriteTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIOWriteTest.java
index 53783d1..de1d403 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIOWriteTest.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIOWriteTest.java
@@ -21,21 +21,38 @@ import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisp
 import static org.hamcrest.Matchers.hasSize;
 import static org.junit.Assert.assertThat;
 import static org.mockito.Mockito.argThat;
+import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
 
-import com.google.cloud.spanner.DatabaseId;
+import com.google.cloud.spanner.Key;
+import com.google.cloud.spanner.KeyRange;
+import com.google.cloud.spanner.KeySet;
 import com.google.cloud.spanner.Mutation;
+import com.google.cloud.spanner.ReadOnlyTransaction;
+import com.google.cloud.spanner.ResultSets;
+import com.google.cloud.spanner.Statement;
+import com.google.cloud.spanner.Struct;
+import com.google.cloud.spanner.Type;
+import com.google.cloud.spanner.Value;
+import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Iterables;
 import java.io.Serializable;
+import java.util.ArrayList;
 import java.util.Arrays;
-
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 import org.apache.beam.sdk.testing.NeedsRunner;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.DoFnTester;
+import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.display.DisplayData;
+import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
+import org.hamcrest.Description;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
@@ -50,17 +67,68 @@ import org.mockito.ArgumentMatcher;
  */
 @RunWith(JUnit4.class)
 public class SpannerIOWriteTest implements Serializable {
-  @Rule public final transient TestPipeline pipeline = TestPipeline.create();
+
+  @Rule public transient TestPipeline pipeline = TestPipeline.create();
   @Rule public transient ExpectedException thrown = ExpectedException.none();
 
   private FakeServiceFactory serviceFactory;
 
-  @Before
-  @SuppressWarnings("unchecked")
-  public void setUp() throws Exception {
+  @Before @SuppressWarnings("unchecked") public void setUp() throws Exception {
     serviceFactory = new FakeServiceFactory();
+
+    ReadOnlyTransaction tx = mock(ReadOnlyTransaction.class);
+    when(serviceFactory.mockDatabaseClient().readOnlyTransaction()).thenReturn(tx);
+
+    // Simplest schema: a table with int64 key
+    preparePkMetadata(tx, Arrays.asList(pkMetadata("test", "key", "ASC")));
+    prepareColumnMetadata(tx, Arrays.asList(columnMetadata("test", "key", "INT64")));
+  }
+
+  private static Struct columnMetadata(String tableName, String columnName, String type) {
+    return Struct.newBuilder().add("table_name", Value.string(tableName))
+        .add("column_name", Value.string(columnName)).add("spanner_type", Value.string(type))
+        .build();
+  }
+
+  private static Struct pkMetadata(String tableName, String columnName, String ordering) {
+    return Struct.newBuilder().add("table_name", Value.string(tableName))
+        .add("column_name", Value.string(columnName)).add("column_ordering", Value.string(ordering))
+        .build();
+  }
+
+  private void prepareColumnMetadata(ReadOnlyTransaction tx, List<Struct> rows) {
+    Type type = Type.struct(Type.StructField.of("table_name", Type.string()),
+        Type.StructField.of("column_name", Type.string()),
+        Type.StructField.of("spanner_type", Type.string()));
+    when(tx.executeQuery(argThat(new ArgumentMatcher<Statement>() {
+
+      @Override public boolean matches(Object argument) {
+        if (!(argument instanceof Statement)) {
+          return false;
+        }
+        Statement st = (Statement) argument;
+        return st.getSql().contains("information_schema.columns");
+      }
+    }))).thenReturn(ResultSets.forRows(type, rows));
+  }
+
+  private void preparePkMetadata(ReadOnlyTransaction tx, List<Struct> rows) {
+    Type type = Type.struct(Type.StructField.of("table_name", Type.string()),
+        Type.StructField.of("column_name", Type.string()),
+        Type.StructField.of("column_ordering", Type.string()));
+    when(tx.executeQuery(argThat(new ArgumentMatcher<Statement>() {
+
+      @Override public boolean matches(Object argument) {
+        if (!(argument instanceof Statement)) {
+          return false;
+        }
+        Statement st = (Statement) argument;
+        return st.getSql().contains("information_schema.index_columns");
+      }
+    }))).thenReturn(ResultSets.forRows(type, rows));
   }
 
+
   @Test
   public void emptyTransform() throws Exception {
     SpannerIO.Write write = SpannerIO.write();
@@ -88,7 +156,7 @@ public class SpannerIOWriteTest implements Serializable {
   @Test
   @Category(NeedsRunner.class)
   public void singleMutationPipeline() throws Exception {
-    Mutation mutation = Mutation.newInsertOrUpdateBuilder("test").set("one").to(2).build();
+    Mutation mutation = m(2L);
     PCollection<Mutation> mutations = pipeline.apply(Create.of(mutation));
 
     mutations.apply(
@@ -98,20 +166,17 @@ public class SpannerIOWriteTest implements Serializable {
             .withDatabaseId("test-database")
             .withServiceFactory(serviceFactory));
     pipeline.run();
-    verify(serviceFactory.mockSpanner())
-        .getDatabaseClient(DatabaseId.of("test-project", "test-instance", "test-database"));
-    verify(serviceFactory.mockDatabaseClient(), times(1))
-        .writeAtLeastOnce(argThat(new IterableOfSize(1)));
+
+    verifyBatches(
+        batch(m(2L))
+    );
   }
 
   @Test
   @Category(NeedsRunner.class)
   public void singleMutationGroupPipeline() throws Exception {
-    Mutation one = Mutation.newInsertOrUpdateBuilder("test").set("one").to(1).build();
-    Mutation two = Mutation.newInsertOrUpdateBuilder("test").set("two").to(2).build();
-    Mutation three = Mutation.newInsertOrUpdateBuilder("test").set("three").to(3).build();
     PCollection<MutationGroup> mutations = pipeline
-        .apply(Create.<MutationGroup>of(g(one, two, three)));
+        .apply(Create.<MutationGroup>of(g(m(1L), m(2L), m(3L))));
     mutations.apply(
         SpannerIO.write()
             .withProjectId("test-project")
@@ -120,106 +185,195 @@ public class SpannerIOWriteTest implements Serializable {
             .withServiceFactory(serviceFactory)
             .grouped());
     pipeline.run();
-    verify(serviceFactory.mockSpanner())
-        .getDatabaseClient(DatabaseId.of("test-project", "test-instance", "test-database"));
-    verify(serviceFactory.mockDatabaseClient(), times(1))
-        .writeAtLeastOnce(argThat(new IterableOfSize(3)));
+
+    verifyBatches(
+        batch(m(1L), m(2L), m(3L))
+    );
   }
 
   @Test
+  @Category(NeedsRunner.class)
   public void batching() throws Exception {
-    MutationGroup one = g(Mutation.newInsertOrUpdateBuilder("test").set("one").to(1).build());
-    MutationGroup two = g(Mutation.newInsertOrUpdateBuilder("test").set("two").to(2).build());
-    SpannerIO.Write write =
-        SpannerIO.write()
-            .withProjectId("test-project")
-            .withInstanceId("test-instance")
-            .withDatabaseId("test-database")
-            .withBatchSizeBytes(1000000000)
-            .withServiceFactory(serviceFactory);
-    SpannerWriteGroupFn writerFn = new SpannerWriteGroupFn(write);
-    DoFnTester<MutationGroup, Void> fnTester = DoFnTester.of(writerFn);
-    fnTester.processBundle(Arrays.asList(one, two));
-
-    verify(serviceFactory.mockSpanner())
-        .getDatabaseClient(DatabaseId.of("test-project", "test-instance", "test-database"));
-    verify(serviceFactory.mockDatabaseClient(), times(1))
-        .writeAtLeastOnce(argThat(new IterableOfSize(2)));
+    MutationGroup one = g(m(1L));
+    MutationGroup two = g(m(2L));
+    PCollection<MutationGroup> mutations = pipeline.apply(Create.of(one, two));
+    mutations.apply(SpannerIO.write()
+        .withProjectId("test-project")
+        .withInstanceId("test-instance")
+        .withDatabaseId("test-database")
+        .withServiceFactory(serviceFactory)
+        .withBatchSizeBytes(1000000000)
+        .withSampler(fakeSampler(m(1000L)))
+        .grouped());
+    pipeline.run();
+
+    verifyBatches(
+        batch(m(1L), m(2L))
+    );
   }
 
   @Test
+  @Category(NeedsRunner.class)
+  public void batchingWithDeletes() throws Exception {
+    PCollection<MutationGroup> mutations = pipeline
+        .apply(Create.of(g(m(1L)), g(m(2L)), g(del(3L)), g(del(4L))));
+    mutations.apply(SpannerIO.write()
+        .withProjectId("test-project")
+        .withInstanceId("test-instance")
+        .withDatabaseId("test-database")
+        .withServiceFactory(serviceFactory)
+        .withBatchSizeBytes(1000000000)
+        .withSampler(fakeSampler(m(1000L)))
+        .grouped());
+    pipeline.run();
+
+    verifyBatches(
+        batch(m(1L), m(2L), del(3L), del(4L))
+    );
+  }
+
+  @Test
+  @Category(NeedsRunner.class)
+  public void noBatchingRangeDelete() throws Exception {
+    Mutation all = Mutation.delete("test", KeySet.all());
+    Mutation prefix = Mutation.delete("test", KeySet.prefixRange(Key.of(1L)));
+    Mutation range = Mutation.delete("test", KeySet.range(KeyRange.openOpen(Key.of(1L), Key
+        .newBuilder().build())));
+
+    PCollection<MutationGroup> mutations = pipeline.apply(Create
+        .of(
+            g(m(1L)),
+            g(m(2L)),
+            g(del(5L, 6L)),
+            g(delRange(50L, 55L)),
+            g(delRange(11L, 20L)),
+            g(all),
+            g(prefix), g(range)
+        )
+    );
+    mutations.apply(SpannerIO.write()
+        .withProjectId("test-project")
+        .withInstanceId("test-instance")
+        .withDatabaseId("test-database")
+        .withServiceFactory(serviceFactory)
+        .withBatchSizeBytes(1000000000)
+        .withSampler(fakeSampler(m(1000L)))
+        .grouped());
+    pipeline.run();
+
+    verifyBatches(
+        batch(m(1L), m(2L)),
+        batch(del(5L, 6L)),
+        batch(delRange(11L, 20L)),
+        batch(delRange(50L, 55L)),
+        batch(all),
+        batch(prefix),
+        batch(range)
+    );
+  }
+
+  private void verifyBatches(Iterable<Mutation>... batches) {
+    for (Iterable<Mutation> b : batches) {
+      verify(serviceFactory.mockDatabaseClient(), times(1)).writeAtLeastOnce(mutationsInNoOrder(b));
+    }
+
+  }
+
+  @Test
+  @Category(NeedsRunner.class)
   public void batchingGroups() throws Exception {
-    MutationGroup one = g(Mutation.newInsertOrUpdateBuilder("test").set("one").to(1).build());
-    MutationGroup two = g(Mutation.newInsertOrUpdateBuilder("test").set("two").to(2).build());
-    MutationGroup three = g(Mutation.newInsertOrUpdateBuilder("test").set("three").to(3).build());
 
     // Have a room to accumulate one more item.
-    long batchSize = MutationSizeEstimator.sizeOf(one) + 1;
+    long batchSize = MutationSizeEstimator.sizeOf(g(m(1L))) + 1;
 
-    SpannerIO.Write write =
-        SpannerIO.write()
-            .withProjectId("test-project")
-            .withInstanceId("test-instance")
-            .withDatabaseId("test-database")
-            .withBatchSizeBytes(batchSize)
-            .withServiceFactory(serviceFactory);
-    SpannerWriteGroupFn writerFn = new SpannerWriteGroupFn(write);
-    DoFnTester<MutationGroup, Void> fnTester = DoFnTester.of(writerFn);
-    fnTester.processBundle(Arrays.asList(one, two, three));
-
-    verify(serviceFactory.mockSpanner())
-        .getDatabaseClient(DatabaseId.of("test-project", "test-instance", "test-database"));
+    PCollection<MutationGroup> mutations = pipeline.apply(Create.of(g(m(1L)), g(m(2L)), g(m(3L))));
+    mutations.apply(SpannerIO.write()
+        .withProjectId("test-project")
+        .withInstanceId("test-instance")
+        .withDatabaseId("test-database")
+        .withServiceFactory(serviceFactory)
+        .withBatchSizeBytes(batchSize)
+        .withSampler(fakeSampler(m(1000L)))
+        .grouped());
+
+    pipeline.run();
+
+    // The content of batches is not deterministic. Just verify that the size is correct.
     verify(serviceFactory.mockDatabaseClient(), times(1))
-        .writeAtLeastOnce(argThat(new IterableOfSize(2)));
+        .writeAtLeastOnce(iterableOfSize(2));
     verify(serviceFactory.mockDatabaseClient(), times(1))
-        .writeAtLeastOnce(argThat(new IterableOfSize(1)));
+        .writeAtLeastOnce(iterableOfSize(1));
   }
 
   @Test
+  @Category(NeedsRunner.class)
   public void noBatching() throws Exception {
-    MutationGroup one = g(Mutation.newInsertOrUpdateBuilder("test").set("one").to(1).build());
-    MutationGroup two = g(Mutation.newInsertOrUpdateBuilder("test").set("two").to(2).build());
-    SpannerIO.Write write =
-        SpannerIO.write()
-            .withProjectId("test-project")
-            .withInstanceId("test-instance")
-            .withDatabaseId("test-database")
-            .withBatchSizeBytes(0) // turn off batching.
-            .withServiceFactory(serviceFactory);
-    SpannerWriteGroupFn writerFn = new SpannerWriteGroupFn(write);
-    DoFnTester<MutationGroup, Void> fnTester = DoFnTester.of(writerFn);
-    fnTester.processBundle(Arrays.asList(one, two));
-
-    verify(serviceFactory.mockSpanner())
-        .getDatabaseClient(DatabaseId.of("test-project", "test-instance", "test-database"));
-    verify(serviceFactory.mockDatabaseClient(), times(2))
-        .writeAtLeastOnce(argThat(new IterableOfSize(1)));
+    PCollection<MutationGroup> mutations = pipeline.apply(Create.of(g(m(1L)), g(m(2L))));
+    mutations.apply(SpannerIO.write()
+        .withProjectId("test-project")
+        .withInstanceId("test-instance")
+        .withDatabaseId("test-database")
+        .withServiceFactory(serviceFactory)
+        .withBatchSizeBytes(1)
+        .withSampler(fakeSampler(m(1000L)))
+        .grouped());
+    pipeline.run();
+
+    verifyBatches(
+        batch(m(1L)),
+        batch(m(2L))
+    );
   }
 
   @Test
-  public void groups() throws Exception {
-    Mutation one = Mutation.newInsertOrUpdateBuilder("test").set("one").to(1).build();
-    Mutation two = Mutation.newInsertOrUpdateBuilder("test").set("two").to(2).build();
-    Mutation three = Mutation.newInsertOrUpdateBuilder("test").set("three").to(3).build();
+  @Category(NeedsRunner.class)
+  public void batchingPlusSampling() throws Exception {
+    PCollection<MutationGroup> mutations = pipeline
+        .apply(Create.of(
+            g(m(1L)), g(m(2L)), g(m(3L)), g(m(4L)),  g(m(5L)),
+            g(m(6L)), g(m(7L)), g(m(8L)), g(m(9L)),  g(m(10L)))
+        );
 
-    // Smallest batch size
-    long batchSize = 1;
+    mutations.apply(SpannerIO.write()
+        .withProjectId("test-project")
+        .withInstanceId("test-instance")
+        .withDatabaseId("test-database")
+        .withServiceFactory(serviceFactory)
+        .withBatchSizeBytes(1000000000)
+        .withSampler(fakeSampler(m(2L), m(5L), m(10L)))
+        .grouped());
+    pipeline.run();
 
-    SpannerIO.Write write =
-        SpannerIO.write()
-            .withProjectId("test-project")
-            .withInstanceId("test-instance")
-            .withDatabaseId("test-database")
-            .withBatchSizeBytes(batchSize)
-            .withServiceFactory(serviceFactory);
-    SpannerWriteGroupFn writerFn = new SpannerWriteGroupFn(write);
-    DoFnTester<MutationGroup, Void> fnTester = DoFnTester.of(writerFn);
-    fnTester.processBundle(Arrays.asList(g(one, two, three)));
-
-    verify(serviceFactory.mockSpanner())
-        .getDatabaseClient(DatabaseId.of("test-project", "test-instance", "test-database"));
-    verify(serviceFactory.mockDatabaseClient(), times(1))
-        .writeAtLeastOnce(argThat(new IterableOfSize(3)));
+    verifyBatches(
+        batch(m(1L), m(2L)),
+        batch(m(3L), m(4L), m(5L)),
+        batch(m(6L), m(7L), m(8L), m(9L), m(10L))
+    );
+  }
+
+  @Test
+  @Category(NeedsRunner.class)
+  public void noBatchingPlusSampling() throws Exception {
+    PCollection<MutationGroup> mutations = pipeline
+        .apply(Create.of(g(m(1L)), g(m(2L)), g(m(3L)), g(m(4L)), g(m(5L))));
+    mutations.apply(SpannerIO.write()
+        .withProjectId("test-project")
+        .withInstanceId("test-instance")
+        .withDatabaseId("test-database")
+        .withServiceFactory(serviceFactory)
+        .withBatchSizeBytes(1)
+        .withSampler(fakeSampler(m(2L)))
+        .grouped());
+
+    pipeline.run();
+
+    verifyBatches(
+        batch(m(1L)),
+        batch(m(2L)),
+        batch(m(3L)),
+        batch(m(4L)),
+        batch(m(5L))
+    );
   }
 
   @Test
@@ -239,20 +393,105 @@ public class SpannerIOWriteTest implements Serializable {
     assertThat(data, hasDisplayItem("batchSizeBytes", 123));
   }
 
-  private static class IterableOfSize extends ArgumentMatcher<Iterable<Mutation>> {
-    private final int size;
+  private static MutationGroup g(Mutation m, Mutation... other) {
+    return MutationGroup.create(m, other);
+  }
 
-    private IterableOfSize(int size) {
-      this.size = size;
-    }
+  private static Mutation m(Long key) {
+    return Mutation.newInsertOrUpdateBuilder("test").set("key").to(key).build();
+  }
 
-    @Override
-    public boolean matches(Object argument) {
-      return argument instanceof Iterable && Iterables.size((Iterable<?>) argument) == size;
+  private static Iterable<Mutation> batch(Mutation... m) {
+    return Arrays.asList(m);
+  }
+
+  private static Mutation del(Long... keys) {
+
+    KeySet.Builder builder = KeySet.newBuilder();
+    for (Long key : keys) {
+      builder.addKey(Key.of(key));
     }
+    return Mutation.delete("test", builder.build());
   }
 
-  private static MutationGroup g(Mutation m, Mutation... other) {
-    return MutationGroup.create(m, other);
+  private static Mutation delRange(Long start, Long end) {
+    return Mutation.delete("test", KeySet.range(KeyRange.closedClosed(Key.of(start), Key.of(end))));
+  }
+
+  private static Iterable<Mutation> mutationsInNoOrder(Iterable<Mutation> expected) {
+    final ImmutableSet<Mutation> mutations = ImmutableSet.copyOf(expected);
+    return argThat(new ArgumentMatcher<Iterable<Mutation>>() {
+
+      @Override
+      public boolean matches(Object argument) {
+        if (!(argument instanceof Iterable)) {
+          return false;
+        }
+        ImmutableSet<Mutation> actual = ImmutableSet.copyOf((Iterable) argument);
+        return actual.equals(mutations);
+      }
+
+      @Override
+      public void describeTo(Description description) {
+        description.appendText("Iterable must match ").appendValue(mutations);
+      }
+
+    });
+  }
+
+  private Iterable<Mutation> iterableOfSize(final int size) {
+    return argThat(new ArgumentMatcher<Iterable<Mutation>>() {
+
+      @Override
+      public boolean matches(Object argument) {
+        return argument instanceof Iterable && Iterables.size((Iterable<?>) argument) == size;
+      }
+
+      @Override
+      public void describeTo(Description description) {
+        description.appendText("The size of the iterable must equal ").appendValue(size);
+      }
+    });
+  }
+
+  private static FakeSampler fakeSampler(Mutation... mutations) {
+    SpannerSchema.Builder schema = SpannerSchema.builder();
+    schema.addColumn("test", "key", "INT64");
+    schema.addKeyPart("test", "key", false);
+    return new FakeSampler(schema.build(), Arrays.asList(mutations));
+  }
+
+  private static class FakeSampler
+      extends PTransform<PCollection<KV<String, byte[]>>, PCollection<KV<String, List<byte[]>>>> {
+
+    private final SpannerSchema schema;
+    private final List<Mutation> mutations;
+
+    private FakeSampler(SpannerSchema schema, List<Mutation> mutations) {
+      this.schema = schema;
+      this.mutations = mutations;
+    }
+
+    @Override
+    public PCollection<KV<String, List<byte[]>>> expand(
+        PCollection<KV<String, byte[]>> input) {
+      MutationGroupEncoder coder = new MutationGroupEncoder(schema);
+      Map<String, List<byte[]>> map = new HashMap<>();
+      for (Mutation m : mutations) {
+        String table = m.getTable();
+        List<byte[]> list = map.get(table);
+        if (list == null) {
+          list = new ArrayList<>();
+          map.put(table, list);
+        }
+        list.add(coder.encodeKey(m));
+      }
+      List<KV<String, List<byte[]>>> result = new ArrayList<>();
+      for (Map.Entry<String, List<byte[]>> entry : map.entrySet()) {
+        Collections.sort(entry.getValue(), SpannerIO.SerializableBytesComparator.INSTANCE);
+        result.add(KV.of(entry.getKey(), entry.getValue()));
+      }
+      return input.getPipeline().apply(Create.of(result));
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/beam/blob/227801b3/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerWriteIT.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerWriteIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerWriteIT.java
index d208f5c..89be159 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerWriteIT.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerWriteIT.java
@@ -119,7 +119,8 @@ public class SpannerWriteIT {
 
   @Test
   public void testWrite() throws Exception {
-    p.apply(GenerateSequence.from(0).to(100))
+    int numRecords = 100;
+    p.apply(GenerateSequence.from(0).to(numRecords))
         .apply(ParDo.of(new GenerateMutations(options.getTable())))
         .apply(
             SpannerIO.write()
@@ -138,7 +139,7 @@ public class SpannerWriteIT {
             .singleUse()
             .executeQuery(Statement.of("SELECT COUNT(*) FROM " + options.getTable()));
     assertThat(resultSet.next(), is(true));
-    assertThat(resultSet.getLong(0), equalTo(100L));
+    assertThat(resultSet.getLong(0), equalTo((long) numRecords));
     assertThat(resultSet.next(), is(false));
   }
 


[42/50] [abbrv] beam git commit: This closes #4076: [BEAM-3139] Update dataflow.version on the beam root pom.xml

Posted by ke...@apache.org.
This closes #4076: [BEAM-3139] Update dataflow.version on the beam root pom.xml


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

Branch: refs/heads/mr-runner
Commit: 35952f6553366207b837f2e1c19187c009d235ed
Parents: 6e28892 9e5f641
Author: Kenneth Knowles <ke...@apache.org>
Authored: Mon Nov 6 16:48:39 2017 -0800
Committer: Kenneth Knowles <ke...@apache.org>
Committed: Mon Nov 6 16:48:39 2017 -0800

----------------------------------------------------------------------
 pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------



[33/50] [abbrv] beam git commit: Add all portability protos to Go

Posted by ke...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/42100456/sdks/go/pkg/beam/model/fnexecution_v1/beam_fn_api.pb.go
----------------------------------------------------------------------
diff --git a/sdks/go/pkg/beam/model/fnexecution_v1/beam_fn_api.pb.go b/sdks/go/pkg/beam/model/fnexecution_v1/beam_fn_api.pb.go
new file mode 100644
index 0000000..9a31a57
--- /dev/null
+++ b/sdks/go/pkg/beam/model/fnexecution_v1/beam_fn_api.pb.go
@@ -0,0 +1,2729 @@
+// Code generated by protoc-gen-go. DO NOT EDIT.
+// source: beam_fn_api.proto
+
+/*
+Package fnexecution_v1 is a generated protocol buffer package.
+
+It is generated from these files:
+	beam_fn_api.proto
+	beam_provision_api.proto
+
+It has these top-level messages:
+	Target
+	RemoteGrpcPort
+	InstructionRequest
+	InstructionResponse
+	RegisterRequest
+	RegisterResponse
+	ProcessBundleDescriptor
+	ProcessBundleRequest
+	ProcessBundleResponse
+	ProcessBundleProgressRequest
+	Metrics
+	ProcessBundleProgressResponse
+	ProcessBundleSplitRequest
+	ElementCountRestriction
+	ElementCountSkipRestriction
+	PrimitiveTransformSplit
+	ProcessBundleSplitResponse
+	Elements
+	StateRequest
+	StateResponse
+	StateKey
+	StateGetRequest
+	StateGetResponse
+	StateAppendRequest
+	StateAppendResponse
+	StateClearRequest
+	StateClearResponse
+	LogEntry
+	LogControl
+	DockerContainer
+	GetProvisionInfoRequest
+	GetProvisionInfoResponse
+	ProvisionInfo
+	Resources
+*/
+package fnexecution_v1
+
+import proto "github.com/golang/protobuf/proto"
+import fmt "fmt"
+import math "math"
+import org_apache_beam_model_pipeline_v1 "github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1"
+import org_apache_beam_model_pipeline_v11 "github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1"
+import google_protobuf1 "github.com/golang/protobuf/ptypes/timestamp"
+
+import (
+	context "golang.org/x/net/context"
+	grpc "google.golang.org/grpc"
+)
+
+// Reference imports to suppress errors if they are not otherwise used.
+var _ = proto.Marshal
+var _ = fmt.Errorf
+var _ = math.Inf
+
+// This is a compile-time assertion to ensure that this generated file
+// is compatible with the proto package it is being compiled against.
+// A compilation error at this line likely means your copy of the
+// proto package needs to be updated.
+const _ = proto.ProtoPackageIsVersion2 // please upgrade the proto package
+
+type LogEntry_Severity_Enum int32
+
+const (
+	LogEntry_Severity_UNSPECIFIED LogEntry_Severity_Enum = 0
+	// Trace level information, also the default log level unless
+	// another severity is specified.
+	LogEntry_Severity_TRACE LogEntry_Severity_Enum = 1
+	// Debugging information.
+	LogEntry_Severity_DEBUG LogEntry_Severity_Enum = 2
+	// Normal events.
+	LogEntry_Severity_INFO LogEntry_Severity_Enum = 3
+	// Normal but significant events, such as start up, shut down, or
+	// configuration.
+	LogEntry_Severity_NOTICE LogEntry_Severity_Enum = 4
+	// Warning events might cause problems.
+	LogEntry_Severity_WARN LogEntry_Severity_Enum = 5
+	// Error events are likely to cause problems.
+	LogEntry_Severity_ERROR LogEntry_Severity_Enum = 6
+	// Critical events cause severe problems or brief outages and may
+	// indicate that a person must take action.
+	LogEntry_Severity_CRITICAL LogEntry_Severity_Enum = 7
+)
+
+var LogEntry_Severity_Enum_name = map[int32]string{
+	0: "UNSPECIFIED",
+	1: "TRACE",
+	2: "DEBUG",
+	3: "INFO",
+	4: "NOTICE",
+	5: "WARN",
+	6: "ERROR",
+	7: "CRITICAL",
+}
+var LogEntry_Severity_Enum_value = map[string]int32{
+	"UNSPECIFIED": 0,
+	"TRACE":       1,
+	"DEBUG":       2,
+	"INFO":        3,
+	"NOTICE":      4,
+	"WARN":        5,
+	"ERROR":       6,
+	"CRITICAL":    7,
+}
+
+func (x LogEntry_Severity_Enum) String() string {
+	return proto.EnumName(LogEntry_Severity_Enum_name, int32(x))
+}
+func (LogEntry_Severity_Enum) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor0, []int{27, 1, 0}
+}
+
+// A representation of an input or output definition on a primitive transform.
+// Stable
+type Target struct {
+	// (Required) The id of the PrimitiveTransform which is the target.
+	PrimitiveTransformReference string `protobuf:"bytes,1,opt,name=primitive_transform_reference,json=primitiveTransformReference" json:"primitive_transform_reference,omitempty"`
+	// (Required) The local name of an input or output defined on the primitive
+	// transform.
+	Name string `protobuf:"bytes,2,opt,name=name" json:"name,omitempty"`
+}
+
+func (m *Target) Reset()                    { *m = Target{} }
+func (m *Target) String() string            { return proto.CompactTextString(m) }
+func (*Target) ProtoMessage()               {}
+func (*Target) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{0} }
+
+func (m *Target) GetPrimitiveTransformReference() string {
+	if m != nil {
+		return m.PrimitiveTransformReference
+	}
+	return ""
+}
+
+func (m *Target) GetName() string {
+	if m != nil {
+		return m.Name
+	}
+	return ""
+}
+
+// A repeated list of target definitions.
+type Target_List struct {
+	Target []*Target `protobuf:"bytes,1,rep,name=target" json:"target,omitempty"`
+}
+
+func (m *Target_List) Reset()                    { *m = Target_List{} }
+func (m *Target_List) String() string            { return proto.CompactTextString(m) }
+func (*Target_List) ProtoMessage()               {}
+func (*Target_List) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{0, 0} }
+
+func (m *Target_List) GetTarget() []*Target {
+	if m != nil {
+		return m.Target
+	}
+	return nil
+}
+
+// A descriptor for connecting to a remote port using the Beam Fn Data API.
+// Allows for communication between two environments (for example between the
+// runner and the SDK).
+// Stable
+type RemoteGrpcPort struct {
+	// (Required) An API descriptor which describes where to
+	// connect to including any authentication that is required.
+	ApiServiceDescriptor *org_apache_beam_model_pipeline_v11.ApiServiceDescriptor `protobuf:"bytes,1,opt,name=api_service_descriptor,json=apiServiceDescriptor" json:"api_service_descriptor,omitempty"`
+}
+
+func (m *RemoteGrpcPort) Reset()                    { *m = RemoteGrpcPort{} }
+func (m *RemoteGrpcPort) String() string            { return proto.CompactTextString(m) }
+func (*RemoteGrpcPort) ProtoMessage()               {}
+func (*RemoteGrpcPort) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{1} }
+
+func (m *RemoteGrpcPort) GetApiServiceDescriptor() *org_apache_beam_model_pipeline_v11.ApiServiceDescriptor {
+	if m != nil {
+		return m.ApiServiceDescriptor
+	}
+	return nil
+}
+
+// A request sent by a runner which the SDK is asked to fulfill.
+// For any unsupported request type, an error should be returned with a
+// matching instruction id.
+// Stable
+type InstructionRequest struct {
+	// (Required) An unique identifier provided by the runner which represents
+	// this requests execution. The InstructionResponse MUST have the matching id.
+	InstructionId string `protobuf:"bytes,1,opt,name=instruction_id,json=instructionId" json:"instruction_id,omitempty"`
+	// (Required) A request that the SDK Harness needs to interpret.
+	//
+	// Types that are valid to be assigned to Request:
+	//	*InstructionRequest_Register
+	//	*InstructionRequest_ProcessBundle
+	//	*InstructionRequest_ProcessBundleProgress
+	//	*InstructionRequest_ProcessBundleSplit
+	Request isInstructionRequest_Request `protobuf_oneof:"request"`
+}
+
+func (m *InstructionRequest) Reset()                    { *m = InstructionRequest{} }
+func (m *InstructionRequest) String() string            { return proto.CompactTextString(m) }
+func (*InstructionRequest) ProtoMessage()               {}
+func (*InstructionRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{2} }
+
+type isInstructionRequest_Request interface {
+	isInstructionRequest_Request()
+}
+
+type InstructionRequest_Register struct {
+	Register *RegisterRequest `protobuf:"bytes,1000,opt,name=register,oneof"`
+}
+type InstructionRequest_ProcessBundle struct {
+	ProcessBundle *ProcessBundleRequest `protobuf:"bytes,1001,opt,name=process_bundle,json=processBundle,oneof"`
+}
+type InstructionRequest_ProcessBundleProgress struct {
+	ProcessBundleProgress *ProcessBundleProgressRequest `protobuf:"bytes,1002,opt,name=process_bundle_progress,json=processBundleProgress,oneof"`
+}
+type InstructionRequest_ProcessBundleSplit struct {
+	ProcessBundleSplit *ProcessBundleSplitRequest `protobuf:"bytes,1003,opt,name=process_bundle_split,json=processBundleSplit,oneof"`
+}
+
+func (*InstructionRequest_Register) isInstructionRequest_Request()              {}
+func (*InstructionRequest_ProcessBundle) isInstructionRequest_Request()         {}
+func (*InstructionRequest_ProcessBundleProgress) isInstructionRequest_Request() {}
+func (*InstructionRequest_ProcessBundleSplit) isInstructionRequest_Request()    {}
+
+func (m *InstructionRequest) GetRequest() isInstructionRequest_Request {
+	if m != nil {
+		return m.Request
+	}
+	return nil
+}
+
+func (m *InstructionRequest) GetInstructionId() string {
+	if m != nil {
+		return m.InstructionId
+	}
+	return ""
+}
+
+func (m *InstructionRequest) GetRegister() *RegisterRequest {
+	if x, ok := m.GetRequest().(*InstructionRequest_Register); ok {
+		return x.Register
+	}
+	return nil
+}
+
+func (m *InstructionRequest) GetProcessBundle() *ProcessBundleRequest {
+	if x, ok := m.GetRequest().(*InstructionRequest_ProcessBundle); ok {
+		return x.ProcessBundle
+	}
+	return nil
+}
+
+func (m *InstructionRequest) GetProcessBundleProgress() *ProcessBundleProgressRequest {
+	if x, ok := m.GetRequest().(*InstructionRequest_ProcessBundleProgress); ok {
+		return x.ProcessBundleProgress
+	}
+	return nil
+}
+
+func (m *InstructionRequest) GetProcessBundleSplit() *ProcessBundleSplitRequest {
+	if x, ok := m.GetRequest().(*InstructionRequest_ProcessBundleSplit); ok {
+		return x.ProcessBundleSplit
+	}
+	return nil
+}
+
+// XXX_OneofFuncs is for the internal use of the proto package.
+func (*InstructionRequest) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) error, func(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error), func(msg proto.Message) (n int), []interface{}) {
+	return _InstructionRequest_OneofMarshaler, _InstructionRequest_OneofUnmarshaler, _InstructionRequest_OneofSizer, []interface{}{
+		(*InstructionRequest_Register)(nil),
+		(*InstructionRequest_ProcessBundle)(nil),
+		(*InstructionRequest_ProcessBundleProgress)(nil),
+		(*InstructionRequest_ProcessBundleSplit)(nil),
+	}
+}
+
+func _InstructionRequest_OneofMarshaler(msg proto.Message, b *proto.Buffer) error {
+	m := msg.(*InstructionRequest)
+	// request
+	switch x := m.Request.(type) {
+	case *InstructionRequest_Register:
+		b.EncodeVarint(1000<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.Register); err != nil {
+			return err
+		}
+	case *InstructionRequest_ProcessBundle:
+		b.EncodeVarint(1001<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.ProcessBundle); err != nil {
+			return err
+		}
+	case *InstructionRequest_ProcessBundleProgress:
+		b.EncodeVarint(1002<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.ProcessBundleProgress); err != nil {
+			return err
+		}
+	case *InstructionRequest_ProcessBundleSplit:
+		b.EncodeVarint(1003<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.ProcessBundleSplit); err != nil {
+			return err
+		}
+	case nil:
+	default:
+		return fmt.Errorf("InstructionRequest.Request has unexpected type %T", x)
+	}
+	return nil
+}
+
+func _InstructionRequest_OneofUnmarshaler(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error) {
+	m := msg.(*InstructionRequest)
+	switch tag {
+	case 1000: // request.register
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(RegisterRequest)
+		err := b.DecodeMessage(msg)
+		m.Request = &InstructionRequest_Register{msg}
+		return true, err
+	case 1001: // request.process_bundle
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(ProcessBundleRequest)
+		err := b.DecodeMessage(msg)
+		m.Request = &InstructionRequest_ProcessBundle{msg}
+		return true, err
+	case 1002: // request.process_bundle_progress
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(ProcessBundleProgressRequest)
+		err := b.DecodeMessage(msg)
+		m.Request = &InstructionRequest_ProcessBundleProgress{msg}
+		return true, err
+	case 1003: // request.process_bundle_split
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(ProcessBundleSplitRequest)
+		err := b.DecodeMessage(msg)
+		m.Request = &InstructionRequest_ProcessBundleSplit{msg}
+		return true, err
+	default:
+		return false, nil
+	}
+}
+
+func _InstructionRequest_OneofSizer(msg proto.Message) (n int) {
+	m := msg.(*InstructionRequest)
+	// request
+	switch x := m.Request.(type) {
+	case *InstructionRequest_Register:
+		s := proto.Size(x.Register)
+		n += proto.SizeVarint(1000<<3 | proto.WireBytes)
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case *InstructionRequest_ProcessBundle:
+		s := proto.Size(x.ProcessBundle)
+		n += proto.SizeVarint(1001<<3 | proto.WireBytes)
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case *InstructionRequest_ProcessBundleProgress:
+		s := proto.Size(x.ProcessBundleProgress)
+		n += proto.SizeVarint(1002<<3 | proto.WireBytes)
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case *InstructionRequest_ProcessBundleSplit:
+		s := proto.Size(x.ProcessBundleSplit)
+		n += proto.SizeVarint(1003<<3 | proto.WireBytes)
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case nil:
+	default:
+		panic(fmt.Sprintf("proto: unexpected type %T in oneof", x))
+	}
+	return n
+}
+
+// The response for an associated request the SDK had been asked to fulfill.
+// Stable
+type InstructionResponse struct {
+	// (Required) A reference provided by the runner which represents a requests
+	// execution. The InstructionResponse MUST have the matching id when
+	// responding to the runner.
+	InstructionId string `protobuf:"bytes,1,opt,name=instruction_id,json=instructionId" json:"instruction_id,omitempty"`
+	// If this is specified, then this instruction has failed.
+	// A human readable string representing the reason as to why processing has
+	// failed.
+	Error string `protobuf:"bytes,2,opt,name=error" json:"error,omitempty"`
+	// If the instruction did not fail, it is required to return an equivalent
+	// response type depending on the request this matches.
+	//
+	// Types that are valid to be assigned to Response:
+	//	*InstructionResponse_Register
+	//	*InstructionResponse_ProcessBundle
+	//	*InstructionResponse_ProcessBundleProgress
+	//	*InstructionResponse_ProcessBundleSplit
+	Response isInstructionResponse_Response `protobuf_oneof:"response"`
+}
+
+func (m *InstructionResponse) Reset()                    { *m = InstructionResponse{} }
+func (m *InstructionResponse) String() string            { return proto.CompactTextString(m) }
+func (*InstructionResponse) ProtoMessage()               {}
+func (*InstructionResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{3} }
+
+type isInstructionResponse_Response interface {
+	isInstructionResponse_Response()
+}
+
+type InstructionResponse_Register struct {
+	Register *RegisterResponse `protobuf:"bytes,1000,opt,name=register,oneof"`
+}
+type InstructionResponse_ProcessBundle struct {
+	ProcessBundle *ProcessBundleResponse `protobuf:"bytes,1001,opt,name=process_bundle,json=processBundle,oneof"`
+}
+type InstructionResponse_ProcessBundleProgress struct {
+	ProcessBundleProgress *ProcessBundleProgressResponse `protobuf:"bytes,1002,opt,name=process_bundle_progress,json=processBundleProgress,oneof"`
+}
+type InstructionResponse_ProcessBundleSplit struct {
+	ProcessBundleSplit *ProcessBundleSplitResponse `protobuf:"bytes,1003,opt,name=process_bundle_split,json=processBundleSplit,oneof"`
+}
+
+func (*InstructionResponse_Register) isInstructionResponse_Response()              {}
+func (*InstructionResponse_ProcessBundle) isInstructionResponse_Response()         {}
+func (*InstructionResponse_ProcessBundleProgress) isInstructionResponse_Response() {}
+func (*InstructionResponse_ProcessBundleSplit) isInstructionResponse_Response()    {}
+
+func (m *InstructionResponse) GetResponse() isInstructionResponse_Response {
+	if m != nil {
+		return m.Response
+	}
+	return nil
+}
+
+func (m *InstructionResponse) GetInstructionId() string {
+	if m != nil {
+		return m.InstructionId
+	}
+	return ""
+}
+
+func (m *InstructionResponse) GetError() string {
+	if m != nil {
+		return m.Error
+	}
+	return ""
+}
+
+func (m *InstructionResponse) GetRegister() *RegisterResponse {
+	if x, ok := m.GetResponse().(*InstructionResponse_Register); ok {
+		return x.Register
+	}
+	return nil
+}
+
+func (m *InstructionResponse) GetProcessBundle() *ProcessBundleResponse {
+	if x, ok := m.GetResponse().(*InstructionResponse_ProcessBundle); ok {
+		return x.ProcessBundle
+	}
+	return nil
+}
+
+func (m *InstructionResponse) GetProcessBundleProgress() *ProcessBundleProgressResponse {
+	if x, ok := m.GetResponse().(*InstructionResponse_ProcessBundleProgress); ok {
+		return x.ProcessBundleProgress
+	}
+	return nil
+}
+
+func (m *InstructionResponse) GetProcessBundleSplit() *ProcessBundleSplitResponse {
+	if x, ok := m.GetResponse().(*InstructionResponse_ProcessBundleSplit); ok {
+		return x.ProcessBundleSplit
+	}
+	return nil
+}
+
+// XXX_OneofFuncs is for the internal use of the proto package.
+func (*InstructionResponse) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) error, func(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error), func(msg proto.Message) (n int), []interface{}) {
+	return _InstructionResponse_OneofMarshaler, _InstructionResponse_OneofUnmarshaler, _InstructionResponse_OneofSizer, []interface{}{
+		(*InstructionResponse_Register)(nil),
+		(*InstructionResponse_ProcessBundle)(nil),
+		(*InstructionResponse_ProcessBundleProgress)(nil),
+		(*InstructionResponse_ProcessBundleSplit)(nil),
+	}
+}
+
+func _InstructionResponse_OneofMarshaler(msg proto.Message, b *proto.Buffer) error {
+	m := msg.(*InstructionResponse)
+	// response
+	switch x := m.Response.(type) {
+	case *InstructionResponse_Register:
+		b.EncodeVarint(1000<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.Register); err != nil {
+			return err
+		}
+	case *InstructionResponse_ProcessBundle:
+		b.EncodeVarint(1001<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.ProcessBundle); err != nil {
+			return err
+		}
+	case *InstructionResponse_ProcessBundleProgress:
+		b.EncodeVarint(1002<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.ProcessBundleProgress); err != nil {
+			return err
+		}
+	case *InstructionResponse_ProcessBundleSplit:
+		b.EncodeVarint(1003<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.ProcessBundleSplit); err != nil {
+			return err
+		}
+	case nil:
+	default:
+		return fmt.Errorf("InstructionResponse.Response has unexpected type %T", x)
+	}
+	return nil
+}
+
+func _InstructionResponse_OneofUnmarshaler(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error) {
+	m := msg.(*InstructionResponse)
+	switch tag {
+	case 1000: // response.register
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(RegisterResponse)
+		err := b.DecodeMessage(msg)
+		m.Response = &InstructionResponse_Register{msg}
+		return true, err
+	case 1001: // response.process_bundle
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(ProcessBundleResponse)
+		err := b.DecodeMessage(msg)
+		m.Response = &InstructionResponse_ProcessBundle{msg}
+		return true, err
+	case 1002: // response.process_bundle_progress
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(ProcessBundleProgressResponse)
+		err := b.DecodeMessage(msg)
+		m.Response = &InstructionResponse_ProcessBundleProgress{msg}
+		return true, err
+	case 1003: // response.process_bundle_split
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(ProcessBundleSplitResponse)
+		err := b.DecodeMessage(msg)
+		m.Response = &InstructionResponse_ProcessBundleSplit{msg}
+		return true, err
+	default:
+		return false, nil
+	}
+}
+
+func _InstructionResponse_OneofSizer(msg proto.Message) (n int) {
+	m := msg.(*InstructionResponse)
+	// response
+	switch x := m.Response.(type) {
+	case *InstructionResponse_Register:
+		s := proto.Size(x.Register)
+		n += proto.SizeVarint(1000<<3 | proto.WireBytes)
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case *InstructionResponse_ProcessBundle:
+		s := proto.Size(x.ProcessBundle)
+		n += proto.SizeVarint(1001<<3 | proto.WireBytes)
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case *InstructionResponse_ProcessBundleProgress:
+		s := proto.Size(x.ProcessBundleProgress)
+		n += proto.SizeVarint(1002<<3 | proto.WireBytes)
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case *InstructionResponse_ProcessBundleSplit:
+		s := proto.Size(x.ProcessBundleSplit)
+		n += proto.SizeVarint(1003<<3 | proto.WireBytes)
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case nil:
+	default:
+		panic(fmt.Sprintf("proto: unexpected type %T in oneof", x))
+	}
+	return n
+}
+
+// A list of objects which can be referred to by the runner in
+// future requests.
+// Stable
+type RegisterRequest struct {
+	// (Optional) The set of descriptors used to process bundles.
+	ProcessBundleDescriptor []*ProcessBundleDescriptor `protobuf:"bytes,1,rep,name=process_bundle_descriptor,json=processBundleDescriptor" json:"process_bundle_descriptor,omitempty"`
+}
+
+func (m *RegisterRequest) Reset()                    { *m = RegisterRequest{} }
+func (m *RegisterRequest) String() string            { return proto.CompactTextString(m) }
+func (*RegisterRequest) ProtoMessage()               {}
+func (*RegisterRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{4} }
+
+func (m *RegisterRequest) GetProcessBundleDescriptor() []*ProcessBundleDescriptor {
+	if m != nil {
+		return m.ProcessBundleDescriptor
+	}
+	return nil
+}
+
+// Stable
+type RegisterResponse struct {
+}
+
+func (m *RegisterResponse) Reset()                    { *m = RegisterResponse{} }
+func (m *RegisterResponse) String() string            { return proto.CompactTextString(m) }
+func (*RegisterResponse) ProtoMessage()               {}
+func (*RegisterResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{5} }
+
+// Definitions that should be used to construct the bundle processing graph.
+type ProcessBundleDescriptor struct {
+	// (Required) A pipeline level unique id which can be used as a reference to
+	// refer to this.
+	Id string `protobuf:"bytes,1,opt,name=id" json:"id,omitempty"`
+	// (Required) A map from pipeline-scoped id to PTransform.
+	Transforms map[string]*org_apache_beam_model_pipeline_v1.PTransform `protobuf:"bytes,2,rep,name=transforms" json:"transforms,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"`
+	// (Required) A map from pipeline-scoped id to PCollection.
+	Pcollections map[string]*org_apache_beam_model_pipeline_v1.PCollection `protobuf:"bytes,3,rep,name=pcollections" json:"pcollections,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"`
+	// (Required) A map from pipeline-scoped id to WindowingStrategy.
+	WindowingStrategies map[string]*org_apache_beam_model_pipeline_v1.WindowingStrategy `protobuf:"bytes,4,rep,name=windowing_strategies,json=windowingStrategies" json:"windowing_strategies,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"`
+	// (Required) A map from pipeline-scoped id to Coder.
+	Coders map[string]*org_apache_beam_model_pipeline_v1.Coder `protobuf:"bytes,5,rep,name=coders" json:"coders,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"`
+	// (Required) A map from pipeline-scoped id to Environment.
+	Environments map[string]*org_apache_beam_model_pipeline_v1.Environment `protobuf:"bytes,6,rep,name=environments" json:"environments,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"`
+	// A descriptor describing the end point to use for State API
+	// calls. Required if the Runner intends to send remote references over the
+	// data plane or if any of the transforms rely on user state or side inputs.
+	StateApiServiceDescriptor *org_apache_beam_model_pipeline_v11.ApiServiceDescriptor `protobuf:"bytes,7,opt,name=state_api_service_descriptor,json=stateApiServiceDescriptor" json:"state_api_service_descriptor,omitempty"`
+}
+
+func (m *ProcessBundleDescriptor) Reset()                    { *m = ProcessBundleDescriptor{} }
+func (m *ProcessBundleDescriptor) String() string            { return proto.CompactTextString(m) }
+func (*ProcessBundleDescriptor) ProtoMessage()               {}
+func (*ProcessBundleDescriptor) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{6} }
+
+func (m *ProcessBundleDescriptor) GetId() string {
+	if m != nil {
+		return m.Id
+	}
+	return ""
+}
+
+func (m *ProcessBundleDescriptor) GetTransforms() map[string]*org_apache_beam_model_pipeline_v1.PTransform {
+	if m != nil {
+		return m.Transforms
+	}
+	return nil
+}
+
+func (m *ProcessBundleDescriptor) GetPcollections() map[string]*org_apache_beam_model_pipeline_v1.PCollection {
+	if m != nil {
+		return m.Pcollections
+	}
+	return nil
+}
+
+func (m *ProcessBundleDescriptor) GetWindowingStrategies() map[string]*org_apache_beam_model_pipeline_v1.WindowingStrategy {
+	if m != nil {
+		return m.WindowingStrategies
+	}
+	return nil
+}
+
+func (m *ProcessBundleDescriptor) GetCoders() map[string]*org_apache_beam_model_pipeline_v1.Coder {
+	if m != nil {
+		return m.Coders
+	}
+	return nil
+}
+
+func (m *ProcessBundleDescriptor) GetEnvironments() map[string]*org_apache_beam_model_pipeline_v1.Environment {
+	if m != nil {
+		return m.Environments
+	}
+	return nil
+}
+
+func (m *ProcessBundleDescriptor) GetStateApiServiceDescriptor() *org_apache_beam_model_pipeline_v11.ApiServiceDescriptor {
+	if m != nil {
+		return m.StateApiServiceDescriptor
+	}
+	return nil
+}
+
+// A request to process a given bundle.
+// Stable
+type ProcessBundleRequest struct {
+	// (Required) A reference to the process bundle descriptor that must be
+	// instantiated and executed by the SDK harness.
+	ProcessBundleDescriptorReference string `protobuf:"bytes,1,opt,name=process_bundle_descriptor_reference,json=processBundleDescriptorReference" json:"process_bundle_descriptor_reference,omitempty"`
+	// (Optional) A list of cache tokens that can be used by an SDK to reuse
+	// cached data returned by the State API across multiple bundles.
+	CacheTokens [][]byte `protobuf:"bytes,2,rep,name=cache_tokens,json=cacheTokens,proto3" json:"cache_tokens,omitempty"`
+}
+
+func (m *ProcessBundleRequest) Reset()                    { *m = ProcessBundleRequest{} }
+func (m *ProcessBundleRequest) String() string            { return proto.CompactTextString(m) }
+func (*ProcessBundleRequest) ProtoMessage()               {}
+func (*ProcessBundleRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{7} }
+
+func (m *ProcessBundleRequest) GetProcessBundleDescriptorReference() string {
+	if m != nil {
+		return m.ProcessBundleDescriptorReference
+	}
+	return ""
+}
+
+func (m *ProcessBundleRequest) GetCacheTokens() [][]byte {
+	if m != nil {
+		return m.CacheTokens
+	}
+	return nil
+}
+
+// Stable
+type ProcessBundleResponse struct {
+	// (Optional) If metrics reporting is supported by the SDK, this represents
+	// the final metrics to record for this bundle.
+	Metrics *Metrics `protobuf:"bytes,1,opt,name=metrics" json:"metrics,omitempty"`
+}
+
+func (m *ProcessBundleResponse) Reset()                    { *m = ProcessBundleResponse{} }
+func (m *ProcessBundleResponse) String() string            { return proto.CompactTextString(m) }
+func (*ProcessBundleResponse) ProtoMessage()               {}
+func (*ProcessBundleResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{8} }
+
+func (m *ProcessBundleResponse) GetMetrics() *Metrics {
+	if m != nil {
+		return m.Metrics
+	}
+	return nil
+}
+
+// A request to report progress information for a given bundle.
+// This is an optional request to be handled and is used to support advanced
+// SDK features such as SplittableDoFn, user level metrics etc.
+type ProcessBundleProgressRequest struct {
+	// (Required) A reference to an active process bundle request with the given
+	// instruction id.
+	InstructionReference string `protobuf:"bytes,1,opt,name=instruction_reference,json=instructionReference" json:"instruction_reference,omitempty"`
+}
+
+func (m *ProcessBundleProgressRequest) Reset()                    { *m = ProcessBundleProgressRequest{} }
+func (m *ProcessBundleProgressRequest) String() string            { return proto.CompactTextString(m) }
+func (*ProcessBundleProgressRequest) ProtoMessage()               {}
+func (*ProcessBundleProgressRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{9} }
+
+func (m *ProcessBundleProgressRequest) GetInstructionReference() string {
+	if m != nil {
+		return m.InstructionReference
+	}
+	return ""
+}
+
+type Metrics struct {
+	Ptransforms map[string]*Metrics_PTransform `protobuf:"bytes,1,rep,name=ptransforms" json:"ptransforms,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"`
+	User        map[string]*Metrics_User       `protobuf:"bytes,2,rep,name=user" json:"user,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"`
+}
+
+func (m *Metrics) Reset()                    { *m = Metrics{} }
+func (m *Metrics) String() string            { return proto.CompactTextString(m) }
+func (*Metrics) ProtoMessage()               {}
+func (*Metrics) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{10} }
+
+func (m *Metrics) GetPtransforms() map[string]*Metrics_PTransform {
+	if m != nil {
+		return m.Ptransforms
+	}
+	return nil
+}
+
+func (m *Metrics) GetUser() map[string]*Metrics_User {
+	if m != nil {
+		return m.User
+	}
+	return nil
+}
+
+// PTransform level metrics.
+// These metrics are split into processed and active element groups for
+// progress reporting purposes. This allows a Runner to see what is measured,
+// what is estimated and what can be extrapolated to be able to accurately
+// estimate the backlog of remaining work.
+type Metrics_PTransform struct {
+	// (Required): Metrics for processed elements.
+	ProcessedElements *Metrics_PTransform_ProcessedElements `protobuf:"bytes,1,opt,name=processed_elements,json=processedElements" json:"processed_elements,omitempty"`
+	// (Required): Metrics for active elements.
+	ActiveElements *Metrics_PTransform_ActiveElements `protobuf:"bytes,2,opt,name=active_elements,json=activeElements" json:"active_elements,omitempty"`
+	// (Optional): Map from local output name to its watermark.
+	// The watermarks reported are tentative, to get a better sense of progress
+	// while processing a bundle but before it is committed. At bundle commit
+	// time, a Runner needs to also take into account the timers set to compute
+	// the actual watermarks.
+	Watermarks map[string]int64 `protobuf:"bytes,3,rep,name=watermarks" json:"watermarks,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"varint,2,opt,name=value"`
+}
+
+func (m *Metrics_PTransform) Reset()                    { *m = Metrics_PTransform{} }
+func (m *Metrics_PTransform) String() string            { return proto.CompactTextString(m) }
+func (*Metrics_PTransform) ProtoMessage()               {}
+func (*Metrics_PTransform) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{10, 0} }
+
+func (m *Metrics_PTransform) GetProcessedElements() *Metrics_PTransform_ProcessedElements {
+	if m != nil {
+		return m.ProcessedElements
+	}
+	return nil
+}
+
+func (m *Metrics_PTransform) GetActiveElements() *Metrics_PTransform_ActiveElements {
+	if m != nil {
+		return m.ActiveElements
+	}
+	return nil
+}
+
+func (m *Metrics_PTransform) GetWatermarks() map[string]int64 {
+	if m != nil {
+		return m.Watermarks
+	}
+	return nil
+}
+
+// Metrics that are measured for processed and active element groups.
+type Metrics_PTransform_Measured struct {
+	// (Optional) Map from local input name to number of elements processed
+	// from this input.
+	// If unset, assumed to be the sum of the outputs of all producers to
+	// this transform (for ProcessedElements) and 0 (for ActiveElements).
+	InputElementCounts map[string]int64 `protobuf:"bytes,1,rep,name=input_element_counts,json=inputElementCounts" json:"input_element_counts,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"varint,2,opt,name=value"`
+	// (Required) Map from local output name to number of elements produced
+	// for this output.
+	OutputElementCounts map[string]int64 `protobuf:"bytes,2,rep,name=output_element_counts,json=outputElementCounts" json:"output_element_counts,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"varint,2,opt,name=value"`
+	// (Optional) The total time spent so far in processing the elements in
+	// this group, in seconds.
+	TotalTimeSpent float64 `protobuf:"fixed64,3,opt,name=total_time_spent,json=totalTimeSpent" json:"total_time_spent,omitempty"`
+}
+
+func (m *Metrics_PTransform_Measured) Reset()         { *m = Metrics_PTransform_Measured{} }
+func (m *Metrics_PTransform_Measured) String() string { return proto.CompactTextString(m) }
+func (*Metrics_PTransform_Measured) ProtoMessage()    {}
+func (*Metrics_PTransform_Measured) Descriptor() ([]byte, []int) {
+	return fileDescriptor0, []int{10, 0, 0}
+}
+
+func (m *Metrics_PTransform_Measured) GetInputElementCounts() map[string]int64 {
+	if m != nil {
+		return m.InputElementCounts
+	}
+	return nil
+}
+
+func (m *Metrics_PTransform_Measured) GetOutputElementCounts() map[string]int64 {
+	if m != nil {
+		return m.OutputElementCounts
+	}
+	return nil
+}
+
+func (m *Metrics_PTransform_Measured) GetTotalTimeSpent() float64 {
+	if m != nil {
+		return m.TotalTimeSpent
+	}
+	return 0
+}
+
+// Metrics for fully processed elements.
+type Metrics_PTransform_ProcessedElements struct {
+	// (Required)
+	Measured *Metrics_PTransform_Measured `protobuf:"bytes,1,opt,name=measured" json:"measured,omitempty"`
+}
+
+func (m *Metrics_PTransform_ProcessedElements) Reset()         { *m = Metrics_PTransform_ProcessedElements{} }
+func (m *Metrics_PTransform_ProcessedElements) String() string { return proto.CompactTextString(m) }
+func (*Metrics_PTransform_ProcessedElements) ProtoMessage()    {}
+func (*Metrics_PTransform_ProcessedElements) Descriptor() ([]byte, []int) {
+	return fileDescriptor0, []int{10, 0, 1}
+}
+
+func (m *Metrics_PTransform_ProcessedElements) GetMeasured() *Metrics_PTransform_Measured {
+	if m != nil {
+		return m.Measured
+	}
+	return nil
+}
+
+// Metrics for active elements.
+// An element is considered active if the SDK has started but not finished
+// processing it yet.
+type Metrics_PTransform_ActiveElements struct {
+	// (Required)
+	Measured *Metrics_PTransform_Measured `protobuf:"bytes,1,opt,name=measured" json:"measured,omitempty"`
+	// (Optional) Sum of estimated fraction of known work remaining for all
+	// active elements, as reported by this transform.
+	// If not reported, a Runner could extrapolate this from the processed
+	// elements.
+	// TODO: Handle the case when known work is infinite.
+	FractionRemaining float64 `protobuf:"fixed64,2,opt,name=fraction_remaining,json=fractionRemaining" json:"fraction_remaining,omitempty"`
+	// (Optional) Map from local output name to sum of estimated number
+	// of elements remaining for this output from all active elements,
+	// as reported by this transform.
+	// If not reported, a Runner could extrapolate this from the processed
+	// elements.
+	OutputElementsRemaining map[string]int64 `protobuf:"bytes,3,rep,name=output_elements_remaining,json=outputElementsRemaining" json:"output_elements_remaining,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"varint,2,opt,name=value"`
+}
+
+func (m *Metrics_PTransform_ActiveElements) Reset()         { *m = Metrics_PTransform_ActiveElements{} }
+func (m *Metrics_PTransform_ActiveElements) String() string { return proto.CompactTextString(m) }
+func (*Metrics_PTransform_ActiveElements) ProtoMessage()    {}
+func (*Metrics_PTransform_ActiveElements) Descriptor() ([]byte, []int) {
+	return fileDescriptor0, []int{10, 0, 2}
+}
+
+func (m *Metrics_PTransform_ActiveElements) GetMeasured() *Metrics_PTransform_Measured {
+	if m != nil {
+		return m.Measured
+	}
+	return nil
+}
+
+func (m *Metrics_PTransform_ActiveElements) GetFractionRemaining() float64 {
+	if m != nil {
+		return m.FractionRemaining
+	}
+	return 0
+}
+
+func (m *Metrics_PTransform_ActiveElements) GetOutputElementsRemaining() map[string]int64 {
+	if m != nil {
+		return m.OutputElementsRemaining
+	}
+	return nil
+}
+
+// User defined metrics
+type Metrics_User struct {
+}
+
+func (m *Metrics_User) Reset()                    { *m = Metrics_User{} }
+func (m *Metrics_User) String() string            { return proto.CompactTextString(m) }
+func (*Metrics_User) ProtoMessage()               {}
+func (*Metrics_User) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{10, 1} }
+
+type ProcessBundleProgressResponse struct {
+	// (Required)
+	Metrics *Metrics `protobuf:"bytes,1,opt,name=metrics" json:"metrics,omitempty"`
+}
+
+func (m *ProcessBundleProgressResponse) Reset()                    { *m = ProcessBundleProgressResponse{} }
+func (m *ProcessBundleProgressResponse) String() string            { return proto.CompactTextString(m) }
+func (*ProcessBundleProgressResponse) ProtoMessage()               {}
+func (*ProcessBundleProgressResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{11} }
+
+func (m *ProcessBundleProgressResponse) GetMetrics() *Metrics {
+	if m != nil {
+		return m.Metrics
+	}
+	return nil
+}
+
+type ProcessBundleSplitRequest struct {
+	// (Required) A reference to an active process bundle request with the given
+	// instruction id.
+	InstructionReference string `protobuf:"bytes,1,opt,name=instruction_reference,json=instructionReference" json:"instruction_reference,omitempty"`
+	// (Required) The fraction of work (when compared to the known amount of work)
+	// the process bundle request should try to split at.
+	Fraction float64 `protobuf:"fixed64,2,opt,name=fraction" json:"fraction,omitempty"`
+}
+
+func (m *ProcessBundleSplitRequest) Reset()                    { *m = ProcessBundleSplitRequest{} }
+func (m *ProcessBundleSplitRequest) String() string            { return proto.CompactTextString(m) }
+func (*ProcessBundleSplitRequest) ProtoMessage()               {}
+func (*ProcessBundleSplitRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{12} }
+
+func (m *ProcessBundleSplitRequest) GetInstructionReference() string {
+	if m != nil {
+		return m.InstructionReference
+	}
+	return ""
+}
+
+func (m *ProcessBundleSplitRequest) GetFraction() float64 {
+	if m != nil {
+		return m.Fraction
+	}
+	return 0
+}
+
+// urn:org.apache.beam:restriction:element-count:1.0
+type ElementCountRestriction struct {
+	// A restriction representing the number of elements that should be processed.
+	// Effectively the range [0, count]
+	Count int64 `protobuf:"varint,1,opt,name=count" json:"count,omitempty"`
+}
+
+func (m *ElementCountRestriction) Reset()                    { *m = ElementCountRestriction{} }
+func (m *ElementCountRestriction) String() string            { return proto.CompactTextString(m) }
+func (*ElementCountRestriction) ProtoMessage()               {}
+func (*ElementCountRestriction) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{13} }
+
+func (m *ElementCountRestriction) GetCount() int64 {
+	if m != nil {
+		return m.Count
+	}
+	return 0
+}
+
+// urn:org.apache.beam:restriction:element-count-skip:1.0
+type ElementCountSkipRestriction struct {
+	// A restriction representing the number of elements that should be skipped.
+	// Effectively the range (count, infinity]
+	Count int64 `protobuf:"varint,1,opt,name=count" json:"count,omitempty"`
+}
+
+func (m *ElementCountSkipRestriction) Reset()                    { *m = ElementCountSkipRestriction{} }
+func (m *ElementCountSkipRestriction) String() string            { return proto.CompactTextString(m) }
+func (*ElementCountSkipRestriction) ProtoMessage()               {}
+func (*ElementCountSkipRestriction) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{14} }
+
+func (m *ElementCountSkipRestriction) GetCount() int64 {
+	if m != nil {
+		return m.Count
+	}
+	return 0
+}
+
+// Each primitive transform that is splittable is defined by a restriction
+// it is currently processing. During splitting, that currently active
+// restriction (R_initial) is split into 2 components:
+//   * a restriction (R_done) representing all elements that will be fully
+//     processed
+//   * a restriction (R_todo) representing all elements that will not be fully
+//     processed
+//
+// where:
+//   R_initial = R_done ⋃ R_todo
+type PrimitiveTransformSplit struct {
+	// (Required) A reference to a primitive transform with the given id that
+	// is part of the active process bundle request with the given instruction
+	// id.
+	PrimitiveTransformReference string `protobuf:"bytes,1,opt,name=primitive_transform_reference,json=primitiveTransformReference" json:"primitive_transform_reference,omitempty"`
+	// (Required) A function specification describing the restriction
+	// that has been completed by the primitive transform.
+	//
+	// For example, a remote GRPC source will have a specific urn and data
+	// block containing an ElementCountRestriction.
+	CompletedRestriction *org_apache_beam_model_pipeline_v1.FunctionSpec `protobuf:"bytes,2,opt,name=completed_restriction,json=completedRestriction" json:"completed_restriction,omitempty"`
+	// (Required) A function specification describing the restriction
+	// representing the remainder of work for the primitive transform.
+	//
+	// FOr example, a remote GRPC source will have a specific urn and data
+	// block contain an ElemntCountSkipRestriction.
+	RemainingRestriction *org_apache_beam_model_pipeline_v1.FunctionSpec `protobuf:"bytes,3,opt,name=remaining_restriction,json=remainingRestriction" json:"remaining_restriction,omitempty"`
+}
+
+func (m *PrimitiveTransformSplit) Reset()                    { *m = PrimitiveTransformSplit{} }
+func (m *PrimitiveTransformSplit) String() string            { return proto.CompactTextString(m) }
+func (*PrimitiveTransformSplit) ProtoMessage()               {}
+func (*PrimitiveTransformSplit) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{15} }
+
+func (m *PrimitiveTransformSplit) GetPrimitiveTransformReference() string {
+	if m != nil {
+		return m.PrimitiveTransformReference
+	}
+	return ""
+}
+
+func (m *PrimitiveTransformSplit) GetCompletedRestriction() *org_apache_beam_model_pipeline_v1.FunctionSpec {
+	if m != nil {
+		return m.CompletedRestriction
+	}
+	return nil
+}
+
+func (m *PrimitiveTransformSplit) GetRemainingRestriction() *org_apache_beam_model_pipeline_v1.FunctionSpec {
+	if m != nil {
+		return m.RemainingRestriction
+	}
+	return nil
+}
+
+type ProcessBundleSplitResponse struct {
+	// If primitive transform B and C are siblings and descendants of A and A, B,
+	// and C report a split. Then B and C's restrictions are relative to A's.
+	//   R = A_done
+	//     ⋃ (A_boundary ⋂ B_done)
+	//     ⋃ (A_boundary ⋂ B_todo)
+	//     ⋃ (A_boundary ⋂ B_todo)
+	//     ⋃ (A_boundary ⋂ C_todo)
+	//     ⋃ A_todo
+	// If there is no descendant of B or C also reporting a split, than
+	//   B_boundary = ∅ and C_boundary = ∅
+	//
+	// This restriction is processed and completed by the currently active process
+	// bundle request:
+	//   A_done ⋃ (A_boundary ⋂ B_done)
+	//          ⋃ (A_boundary ⋂ C_done)
+	// and these restrictions will be processed by future process bundle requests:
+	//   A_boundary â‹‚ B_todo (passed to SDF B directly)
+	//   A_boundary â‹‚ C_todo (passed to SDF C directly)
+	//   A_todo (passed to SDF A directly)
+	//
+	// Note that descendants splits should only be reported if it is inexpensive
+	// to compute the boundary restriction intersected with descendants splits.
+	// Also note, that the boundary restriction may represent a set of elements
+	// produced by a parent primitive transform which can not be split at each
+	// element or that there are intermediate unsplittable primitive transforms
+	// between an ancestor splittable function and a descendant splittable
+	// function which may have more than one output per element. Finally note
+	// that the descendant splits should only be reported if the split
+	// information is relatively compact.
+	Splits []*PrimitiveTransformSplit `protobuf:"bytes,1,rep,name=splits" json:"splits,omitempty"`
+}
+
+func (m *ProcessBundleSplitResponse) Reset()                    { *m = ProcessBundleSplitResponse{} }
+func (m *ProcessBundleSplitResponse) String() string            { return proto.CompactTextString(m) }
+func (*ProcessBundleSplitResponse) ProtoMessage()               {}
+func (*ProcessBundleSplitResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{16} }
+
+func (m *ProcessBundleSplitResponse) GetSplits() []*PrimitiveTransformSplit {
+	if m != nil {
+		return m.Splits
+	}
+	return nil
+}
+
+// Messages used to represent logical byte streams.
+// Stable
+type Elements struct {
+	// (Required) A list containing parts of logical byte streams.
+	Data []*Elements_Data `protobuf:"bytes,1,rep,name=data" json:"data,omitempty"`
+}
+
+func (m *Elements) Reset()                    { *m = Elements{} }
+func (m *Elements) String() string            { return proto.CompactTextString(m) }
+func (*Elements) ProtoMessage()               {}
+func (*Elements) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{17} }
+
+func (m *Elements) GetData() []*Elements_Data {
+	if m != nil {
+		return m.Data
+	}
+	return nil
+}
+
+// Represents multiple encoded elements in nested context for a given named
+// instruction and target.
+type Elements_Data struct {
+	// (Required) A reference to an active instruction request with the given
+	// instruction id.
+	InstructionReference string `protobuf:"bytes,1,opt,name=instruction_reference,json=instructionReference" json:"instruction_reference,omitempty"`
+	// (Required) A definition representing a consumer or producer of this data.
+	// If received by a harness, this represents the consumer within that
+	// harness that should consume these bytes. If sent by a harness, this
+	// represents the producer of these bytes.
+	//
+	// Note that a single element may span multiple Data messages.
+	//
+	// Note that a sending/receiving pair should share the same target
+	// identifier.
+	Target *Target `protobuf:"bytes,2,opt,name=target" json:"target,omitempty"`
+	// (Optional) Represents a part of a logical byte stream. Elements within
+	// the logical byte stream are encoded in the nested context and
+	// concatenated together.
+	//
+	// An empty data block represents the end of stream for the given
+	// instruction and target.
+	Data []byte `protobuf:"bytes,3,opt,name=data,proto3" json:"data,omitempty"`
+}
+
+func (m *Elements_Data) Reset()                    { *m = Elements_Data{} }
+func (m *Elements_Data) String() string            { return proto.CompactTextString(m) }
+func (*Elements_Data) ProtoMessage()               {}
+func (*Elements_Data) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{17, 0} }
+
+func (m *Elements_Data) GetInstructionReference() string {
+	if m != nil {
+		return m.InstructionReference
+	}
+	return ""
+}
+
+func (m *Elements_Data) GetTarget() *Target {
+	if m != nil {
+		return m.Target
+	}
+	return nil
+}
+
+func (m *Elements_Data) GetData() []byte {
+	if m != nil {
+		return m.Data
+	}
+	return nil
+}
+
+type StateRequest struct {
+	// (Required) An unique identifier provided by the SDK which represents this
+	// requests execution. The StateResponse corresponding with this request
+	// will have the matching id.
+	Id string `protobuf:"bytes,1,opt,name=id" json:"id,omitempty"`
+	// (Required) The associated instruction id of the work that is currently
+	// being processed. This allows for the runner to associate any modifications
+	// to state to be committed with the appropriate work execution.
+	InstructionReference string `protobuf:"bytes,2,opt,name=instruction_reference,json=instructionReference" json:"instruction_reference,omitempty"`
+	// (Required) The state key this request is for.
+	StateKey *StateKey `protobuf:"bytes,3,opt,name=state_key,json=stateKey" json:"state_key,omitempty"`
+	// (Required) The action to take on this request.
+	//
+	// Types that are valid to be assigned to Request:
+	//	*StateRequest_Get
+	//	*StateRequest_Append
+	//	*StateRequest_Clear
+	Request isStateRequest_Request `protobuf_oneof:"request"`
+}
+
+func (m *StateRequest) Reset()                    { *m = StateRequest{} }
+func (m *StateRequest) String() string            { return proto.CompactTextString(m) }
+func (*StateRequest) ProtoMessage()               {}
+func (*StateRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{18} }
+
+type isStateRequest_Request interface {
+	isStateRequest_Request()
+}
+
+type StateRequest_Get struct {
+	Get *StateGetRequest `protobuf:"bytes,1000,opt,name=get,oneof"`
+}
+type StateRequest_Append struct {
+	Append *StateAppendRequest `protobuf:"bytes,1001,opt,name=append,oneof"`
+}
+type StateRequest_Clear struct {
+	Clear *StateClearRequest `protobuf:"bytes,1002,opt,name=clear,oneof"`
+}
+
+func (*StateRequest_Get) isStateRequest_Request()    {}
+func (*StateRequest_Append) isStateRequest_Request() {}
+func (*StateRequest_Clear) isStateRequest_Request()  {}
+
+func (m *StateRequest) GetRequest() isStateRequest_Request {
+	if m != nil {
+		return m.Request
+	}
+	return nil
+}
+
+func (m *StateRequest) GetId() string {
+	if m != nil {
+		return m.Id
+	}
+	return ""
+}
+
+func (m *StateRequest) GetInstructionReference() string {
+	if m != nil {
+		return m.InstructionReference
+	}
+	return ""
+}
+
+func (m *StateRequest) GetStateKey() *StateKey {
+	if m != nil {
+		return m.StateKey
+	}
+	return nil
+}
+
+func (m *StateRequest) GetGet() *StateGetRequest {
+	if x, ok := m.GetRequest().(*StateRequest_Get); ok {
+		return x.Get
+	}
+	return nil
+}
+
+func (m *StateRequest) GetAppend() *StateAppendRequest {
+	if x, ok := m.GetRequest().(*StateRequest_Append); ok {
+		return x.Append
+	}
+	return nil
+}
+
+func (m *StateRequest) GetClear() *StateClearRequest {
+	if x, ok := m.GetRequest().(*StateRequest_Clear); ok {
+		return x.Clear
+	}
+	return nil
+}
+
+// XXX_OneofFuncs is for the internal use of the proto package.
+func (*StateRequest) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) error, func(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error), func(msg proto.Message) (n int), []interface{}) {
+	return _StateRequest_OneofMarshaler, _StateRequest_OneofUnmarshaler, _StateRequest_OneofSizer, []interface{}{
+		(*StateRequest_Get)(nil),
+		(*StateRequest_Append)(nil),
+		(*StateRequest_Clear)(nil),
+	}
+}
+
+func _StateRequest_OneofMarshaler(msg proto.Message, b *proto.Buffer) error {
+	m := msg.(*StateRequest)
+	// request
+	switch x := m.Request.(type) {
+	case *StateRequest_Get:
+		b.EncodeVarint(1000<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.Get); err != nil {
+			return err
+		}
+	case *StateRequest_Append:
+		b.EncodeVarint(1001<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.Append); err != nil {
+			return err
+		}
+	case *StateRequest_Clear:
+		b.EncodeVarint(1002<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.Clear); err != nil {
+			return err
+		}
+	case nil:
+	default:
+		return fmt.Errorf("StateRequest.Request has unexpected type %T", x)
+	}
+	return nil
+}
+
+func _StateRequest_OneofUnmarshaler(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error) {
+	m := msg.(*StateRequest)
+	switch tag {
+	case 1000: // request.get
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(StateGetRequest)
+		err := b.DecodeMessage(msg)
+		m.Request = &StateRequest_Get{msg}
+		return true, err
+	case 1001: // request.append
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(StateAppendRequest)
+		err := b.DecodeMessage(msg)
+		m.Request = &StateRequest_Append{msg}
+		return true, err
+	case 1002: // request.clear
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(StateClearRequest)
+		err := b.DecodeMessage(msg)
+		m.Request = &StateRequest_Clear{msg}
+		return true, err
+	default:
+		return false, nil
+	}
+}
+
+func _StateRequest_OneofSizer(msg proto.Message) (n int) {
+	m := msg.(*StateRequest)
+	// request
+	switch x := m.Request.(type) {
+	case *StateRequest_Get:
+		s := proto.Size(x.Get)
+		n += proto.SizeVarint(1000<<3 | proto.WireBytes)
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case *StateRequest_Append:
+		s := proto.Size(x.Append)
+		n += proto.SizeVarint(1001<<3 | proto.WireBytes)
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case *StateRequest_Clear:
+		s := proto.Size(x.Clear)
+		n += proto.SizeVarint(1002<<3 | proto.WireBytes)
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case nil:
+	default:
+		panic(fmt.Sprintf("proto: unexpected type %T in oneof", x))
+	}
+	return n
+}
+
+type StateResponse struct {
+	// (Required) A reference provided by the SDK which represents a requests
+	// execution. The StateResponse must have the matching id when responding
+	// to the SDK.
+	Id string `protobuf:"bytes,1,opt,name=id" json:"id,omitempty"`
+	// (Optional) If this is specified, then the state request has failed.
+	// A human readable string representing the reason as to why the request
+	// failed.
+	Error string `protobuf:"bytes,2,opt,name=error" json:"error,omitempty"`
+	// (Optional) If this is specified, then the result of this state request
+	// can be cached using the supplied token.
+	CacheToken []byte `protobuf:"bytes,3,opt,name=cache_token,json=cacheToken,proto3" json:"cache_token,omitempty"`
+	// A corresponding response matching the request will be populated.
+	//
+	// Types that are valid to be assigned to Response:
+	//	*StateResponse_Get
+	//	*StateResponse_Append
+	//	*StateResponse_Clear
+	Response isStateResponse_Response `protobuf_oneof:"response"`
+}
+
+func (m *StateResponse) Reset()                    { *m = StateResponse{} }
+func (m *StateResponse) String() string            { return proto.CompactTextString(m) }
+func (*StateResponse) ProtoMessage()               {}
+func (*StateResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{19} }
+
+type isStateResponse_Response interface {
+	isStateResponse_Response()
+}
+
+type StateResponse_Get struct {
+	Get *StateGetResponse `protobuf:"bytes,1000,opt,name=get,oneof"`
+}
+type StateResponse_Append struct {
+	Append *StateAppendResponse `protobuf:"bytes,1001,opt,name=append,oneof"`
+}
+type StateResponse_Clear struct {
+	Clear *StateClearResponse `protobuf:"bytes,1002,opt,name=clear,oneof"`
+}
+
+func (*StateResponse_Get) isStateResponse_Response()    {}
+func (*StateResponse_Append) isStateResponse_Response() {}
+func (*StateResponse_Clear) isStateResponse_Response()  {}
+
+func (m *StateResponse) GetResponse() isStateResponse_Response {
+	if m != nil {
+		return m.Response
+	}
+	return nil
+}
+
+func (m *StateResponse) GetId() string {
+	if m != nil {
+		return m.Id
+	}
+	return ""
+}
+
+func (m *StateResponse) GetError() string {
+	if m != nil {
+		return m.Error
+	}
+	return ""
+}
+
+func (m *StateResponse) GetCacheToken() []byte {
+	if m != nil {
+		return m.CacheToken
+	}
+	return nil
+}
+
+func (m *StateResponse) GetGet() *StateGetResponse {
+	if x, ok := m.GetResponse().(*StateResponse_Get); ok {
+		return x.Get
+	}
+	return nil
+}
+
+func (m *StateResponse) GetAppend() *StateAppendResponse {
+	if x, ok := m.GetResponse().(*StateResponse_Append); ok {
+		return x.Append
+	}
+	return nil
+}
+
+func (m *StateResponse) GetClear() *StateClearResponse {
+	if x, ok := m.GetResponse().(*StateResponse_Clear); ok {
+		return x.Clear
+	}
+	return nil
+}
+
+// XXX_OneofFuncs is for the internal use of the proto package.
+func (*StateResponse) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) error, func(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error), func(msg proto.Message) (n int), []interface{}) {
+	return _StateResponse_OneofMarshaler, _StateResponse_OneofUnmarshaler, _StateResponse_OneofSizer, []interface{}{
+		(*StateResponse_Get)(nil),
+		(*StateResponse_Append)(nil),
+		(*StateResponse_Clear)(nil),
+	}
+}
+
+func _StateResponse_OneofMarshaler(msg proto.Message, b *proto.Buffer) error {
+	m := msg.(*StateResponse)
+	// response
+	switch x := m.Response.(type) {
+	case *StateResponse_Get:
+		b.EncodeVarint(1000<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.Get); err != nil {
+			return err
+		}
+	case *StateResponse_Append:
+		b.EncodeVarint(1001<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.Append); err != nil {
+			return err
+		}
+	case *StateResponse_Clear:
+		b.EncodeVarint(1002<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.Clear); err != nil {
+			return err
+		}
+	case nil:
+	default:
+		return fmt.Errorf("StateResponse.Response has unexpected type %T", x)
+	}
+	return nil
+}
+
+func _StateResponse_OneofUnmarshaler(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error) {
+	m := msg.(*StateResponse)
+	switch tag {
+	case 1000: // response.get
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(StateGetResponse)
+		err := b.DecodeMessage(msg)
+		m.Response = &StateResponse_Get{msg}
+		return true, err
+	case 1001: // response.append
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(StateAppendResponse)
+		err := b.DecodeMessage(msg)
+		m.Response = &StateResponse_Append{msg}
+		return true, err
+	case 1002: // response.clear
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(StateClearResponse)
+		err := b.DecodeMessage(msg)
+		m.Response = &StateResponse_Clear{msg}
+		return true, err
+	default:
+		return false, nil
+	}
+}
+
+func _StateResponse_OneofSizer(msg proto.Message) (n int) {
+	m := msg.(*StateResponse)
+	// response
+	switch x := m.Response.(type) {
+	case *StateResponse_Get:
+		s := proto.Size(x.Get)
+		n += proto.SizeVarint(1000<<3 | proto.WireBytes)
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case *StateResponse_Append:
+		s := proto.Size(x.Append)
+		n += proto.SizeVarint(1001<<3 | proto.WireBytes)
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case *StateResponse_Clear:
+		s := proto.Size(x.Clear)
+		n += proto.SizeVarint(1002<<3 | proto.WireBytes)
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case nil:
+	default:
+		panic(fmt.Sprintf("proto: unexpected type %T in oneof", x))
+	}
+	return n
+}
+
+type StateKey struct {
+	// (Required) One of the following state keys must be set.
+	//
+	// Types that are valid to be assigned to Type:
+	//	*StateKey_Runner_
+	//	*StateKey_MultimapSideInput_
+	//	*StateKey_BagUserState_
+	Type isStateKey_Type `protobuf_oneof:"type"`
+}
+
+func (m *StateKey) Reset()                    { *m = StateKey{} }
+func (m *StateKey) String() string            { return proto.CompactTextString(m) }
+func (*StateKey) ProtoMessage()               {}
+func (*StateKey) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{20} }
+
+type isStateKey_Type interface {
+	isStateKey_Type()
+}
+
+type StateKey_Runner_ struct {
+	Runner *StateKey_Runner `protobuf:"bytes,1,opt,name=runner,oneof"`
+}
+type StateKey_MultimapSideInput_ struct {
+	MultimapSideInput *StateKey_MultimapSideInput `protobuf:"bytes,2,opt,name=multimap_side_input,json=multimapSideInput,oneof"`
+}
+type StateKey_BagUserState_ struct {
+	BagUserState *StateKey_BagUserState `protobuf:"bytes,3,opt,name=bag_user_state,json=bagUserState,oneof"`
+}
+
+func (*StateKey_Runner_) isStateKey_Type()            {}
+func (*StateKey_MultimapSideInput_) isStateKey_Type() {}
+func (*StateKey_BagUserState_) isStateKey_Type()      {}
+
+func (m *StateKey) GetType() isStateKey_Type {
+	if m != nil {
+		return m.Type
+	}
+	return nil
+}
+
+func (m *StateKey) GetRunner() *StateKey_Runner {
+	if x, ok := m.GetType().(*StateKey_Runner_); ok {
+		return x.Runner
+	}
+	return nil
+}
+
+func (m *StateKey) GetMultimapSideInput() *StateKey_MultimapSideInput {
+	if x, ok := m.GetType().(*StateKey_MultimapSideInput_); ok {
+		return x.MultimapSideInput
+	}
+	return nil
+}
+
+func (m *StateKey) GetBagUserState() *StateKey_BagUserState {
+	if x, ok := m.GetType().(*StateKey_BagUserState_); ok {
+		return x.BagUserState
+	}
+	return nil
+}
+
+// XXX_OneofFuncs is for the internal use of the proto package.
+func (*StateKey) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) error, func(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error), func(msg proto.Message) (n int), []interface{}) {
+	return _StateKey_OneofMarshaler, _StateKey_OneofUnmarshaler, _StateKey_OneofSizer, []interface{}{
+		(*StateKey_Runner_)(nil),
+		(*StateKey_MultimapSideInput_)(nil),
+		(*StateKey_BagUserState_)(nil),
+	}
+}
+
+func _StateKey_OneofMarshaler(msg proto.Message, b *proto.Buffer) error {
+	m := msg.(*StateKey)
+	// type
+	switch x := m.Type.(type) {
+	case *StateKey_Runner_:
+		b.EncodeVarint(1<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.Runner); err != nil {
+			return err
+		}
+	case *StateKey_MultimapSideInput_:
+		b.EncodeVarint(2<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.MultimapSideInput); err != nil {
+			return err
+		}
+	case *StateKey_BagUserState_:
+		b.EncodeVarint(3<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.BagUserState); err != nil {
+			return err
+		}
+	case nil:
+	default:
+		return fmt.Errorf("StateKey.Type has unexpected type %T", x)
+	}
+	return nil
+}
+
+func _StateKey_OneofUnmarshaler(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error) {
+	m := msg.(*StateKey)
+	switch tag {
+	case 1: // type.runner
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(StateKey_Runner)
+		err := b.DecodeMessage(msg)
+		m.Type = &StateKey_Runner_{msg}
+		return true, err
+	case 2: // type.multimap_side_input
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(StateKey_MultimapSideInput)
+		err := b.DecodeMessage(msg)
+		m.Type = &StateKey_MultimapSideInput_{msg}
+		return true, err
+	case 3: // type.bag_user_state
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(StateKey_BagUserState)
+		err := b.DecodeMessage(msg)
+		m.Type = &StateKey_BagUserState_{msg}
+		return true, err
+	default:
+		return false, nil
+	}
+}
+
+func _StateKey_OneofSizer(msg proto.Message) (n int) {
+	m := msg.(*StateKey)
+	// type
+	switch x := m.Type.(type) {
+	case *StateKey_Runner_:
+		s := proto.Size(x.Runner)
+		n += proto.SizeVarint(1<<3 | proto.WireBytes)
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case *StateKey_MultimapSideInput_:
+		s := proto.Size(x.MultimapSideInput)
+		n += proto.SizeVarint(2<<3 | proto.WireBytes)
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case *StateKey_BagUserState_:
+		s := proto.Size(x.BagUserState)
+		n += proto.SizeVarint(3<<3 | proto.WireBytes)
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case nil:
+	default:
+		panic(fmt.Sprintf("proto: unexpected type %T in oneof", x))
+	}
+	return n
+}
+
+type StateKey_Runner struct {
+	// (Required) Opaque information supplied by the runner. Used to support
+	// remote references.
+	Key []byte `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"`
+}
+
+func (m *StateKey_Runner) Reset()                    { *m = StateKey_Runner{} }
+func (m *StateKey_Runner) String() string            { return proto.CompactTextString(m) }
+func (*StateKey_Runner) ProtoMessage()               {}
+func (*StateKey_Runner) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{20, 0} }
+
+func (m *StateKey_Runner) GetKey() []byte {
+	if m != nil {
+		return m.Key
+	}
+	return nil
+}
+
+type StateKey_MultimapSideInput struct {
+	// (Required) The id of the PTransform containing a side input.
+	PtransformId string `protobuf:"bytes,1,opt,name=ptransform_id,json=ptransformId" json:"ptransform_id,omitempty"`
+	// (Required) The id of the side input.
+	SideInputId string `protobuf:"bytes,2,opt,name=side_input_id,json=sideInputId" json:"side_input_id,omitempty"`
+	// (Required) The window (after mapping the currently executing elements
+	// window into the side input windows domain) encoded in a nested context.
+	Window []byte `protobuf:"bytes,3,opt,name=window,proto3" json:"window,omitempty"`
+	// (Required) The key encoded in a nested context.
+	Key []byte `protobuf:"bytes,4,opt,name=key,proto3" json:"key,omitempty"`
+}
+
+func (m *StateKey_MultimapSideInput) Reset()                    { *m = StateKey_MultimapSideInput{} }
+func (m *StateKey_MultimapSideInput) String() string            { return proto.CompactTextString(m) }
+func (*StateKey_MultimapSideInput) ProtoMessage()               {}
+func (*StateKey_MultimapSideInput) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{20, 1} }
+
+func (m *StateKey_MultimapSideInput) GetPtransformId() string {
+	if m != nil {
+		return m.PtransformId
+	}
+	return ""
+}
+
+func (m *StateKey_MultimapSideInput) GetSideInputId() string {
+	if m != nil {
+		return m.SideInputId
+	}
+	return ""
+}
+
+func (m *StateKey_MultimapSideInput) GetWindow() []byte {
+	if m != nil {
+		return m.Window
+	}
+	return nil
+}
+
+func (m *StateKey_MultimapSideInput) GetKey() []byte {
+	if m != nil {
+		return m.Key
+	}
+	return nil
+}
+
+type StateKey_BagUserState struct {
+	// (Required) The id of the PTransform containing user state.
+	PtransformId string `protobuf:"bytes,1,opt,name=ptransform_id,json=ptransformId" json:"ptransform_id,omitempty"`
+	// (Required) The id of the user state.
+	UserStateId string `protobuf:"bytes,2,opt,name=user_state_id,json=userStateId" json:"user_state_id,omitempty"`
+	// (Required) The window encoded in a nested context.
+	Window []byte `protobuf:"bytes,3,opt,name=window,proto3" json:"window,omitempty"`
+	// (Required) The key of the currently executing element encoded in a
+	// nested context.
+	Key []byte `protobuf:"bytes,4,opt,name=key,proto3" json:"key,omitempty"`
+}
+
+func (m *StateKey_BagUserState) Reset()                    { *m = StateKey_BagUserState{} }
+func (m *StateKey_BagUserState) String() string            { return proto.CompactTextString(m) }
+func (*StateKey_BagUserState) ProtoMessage()               {}
+func (*StateKey_BagUserState) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{20, 2} }
+
+func (m *StateKey_BagUserState) GetPtransformId() string {
+	if m != nil {
+		return m.PtransformId
+	}
+	return ""
+}
+
+func (m *StateKey_BagUserState) GetUserStateId() string {
+	if m != nil {
+		return m.UserStateId
+	}
+	return ""
+}
+
+func (m *StateKey_BagUserState) GetWindow() []byte {
+	if m != nil {
+		return m.Window
+	}
+	return nil
+}
+
+func (m *StateKey_BagUserState) GetKey() []byte {
+	if m != nil {
+		return m.Key
+	}
+	return nil
+}
+
+// A request to get state.
+type StateGetRequest struct {
+	// (Optional) If specified, signals to the runner that the response
+	// should resume from the following continuation token.
+	//
+	// If unspecified, signals to the runner that the response should start
+	// from the beginning of the logical continuable stream.
+	ContinuationToken []byte `protobuf:"bytes,1,opt,name=continuation_token,json=continuationToken,proto3" json:"continuation_token,omitempty"`
+}
+
+func (m *StateGetRequest) Reset()                    { *m = StateGetRequest{} }
+func (m *StateGetRequest) String() string            { return proto.CompactTextString(m) }
+func (*StateGetRequest) ProtoMessage()               {}
+func (*StateGetRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{21} }
+
+func (m *StateGetRequest) GetContinuationToken() []byte {
+	if m != nil {
+		return m.ContinuationToken
+	}
+	return nil
+}
+
+// A response to get state representing a logical byte stream which can be
+// continued using the state API.
+type StateGetResponse struct {
+	// (Optional) If specified, represents a token which can be used with the
+	// state API to get the next chunk of this logical byte stream. The end of
+	// the logical byte stream is signalled by this field being unset.
+	ContinuationToken []byte `protobuf:"bytes,1,opt,name=continuation_token,json=continuationToken,proto3" json:"continuation_token,omitempty"`
+	// Represents a part of a logical byte stream. Elements within
+	// the logical byte stream are encoded in the nested context and
+	// concatenated together.
+	Data []byte `protobuf:"bytes,2,opt,name=data,proto3" json:"data,omitempty"`
+}
+
+func (m *StateGetResponse) Reset()                    { *m = StateGetResponse{} }
+func (m *StateGetResponse) String() string            { return proto.CompactTextString(m) }
+func (*StateGetResponse) ProtoMessage()               {}
+func (*StateGetResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{22} }
+
+func (m *StateGetResponse) GetContinuationToken() []byte {
+	if m != nil {
+		return m.ContinuationToken
+	}
+	return nil
+}
+
+func (m *StateGetResponse) GetData() []byte {
+	if m != nil {
+		return m.Data
+	}
+	return nil
+}
+
+// A request to append state.
+type StateAppendRequest struct {
+	// Represents a part of a logical byte stream. Elements within
+	// the logical byte stream are encoded in the nested context and
+	// multiple append requests are concatenated together.
+	Data []byte `protobuf:"bytes,1,opt,name=data,proto3" json:"data,omitempty"`
+}
+
+func (m *StateAppendRequest) Reset()                    { *m = StateAppendRequest{} }
+func (m *StateAppendRequest) String() string            { return proto.CompactTextString(m) }
+func (*StateAppendRequest) ProtoMessage()               {}
+func (*StateAppendRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{23} }
+
+func (m *StateAppendRequest) GetData() []byte {
+	if m != nil {
+		return m.Data
+	}
+	return nil
+}
+
+// A response to append state.
+type StateAppendResponse struct {
+}
+
+func (m *StateAppendResponse) Reset()                    { *m = StateAppendResponse{} }
+func (m *StateAppendResponse) String() string            { return proto.CompactTextString(m) }
+func (*StateAppendResponse) ProtoMessage()               {}
+func (*StateAppendResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{24} }
+
+// A request to clear state.
+type StateClearRequest struct {
+}
+
+func (m *StateClearRequest) Reset()                    { *m = StateClearRequest{} }
+func (m *StateClearRequest) String() string            { return proto.CompactTextString(m) }
+func (*StateClearRequest) ProtoMessage()               {}
+func (*StateClearRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{25} }
+
+// A response to clear state.
+type StateClearResponse struct {
+}
+
+func (m *StateClearResponse) Reset()                    { *m = StateClearResponse{} }
+func (m *StateClearResponse) String() string            { return proto.CompactTextString(m) }
+func (*StateClearResponse) ProtoMessage()               {}
+func (*StateClearResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{26} }
+
+// A log entry
+type LogEntry struct {
+	// (Required) The severity of the log statement.
+	Severity LogEntry_Severity_Enum `protobuf:"varint,1,opt,name=severity,enum=org.apache.beam.model.fn_execution.v1.LogEntry_Severity_Enum" json:"severity,omitempty"`
+	// (Required) The time at which this log statement occurred.
+	Timestamp *google_protobuf1.Timestamp `protobuf:"bytes,2,opt,name=timestamp" json:"timestamp,omitempty"`
+	// (Required) A human readable message.
+	Message string `protobuf:"bytes,3,opt,name=message" json:"message,omitempty"`
+	// (Optional) An optional trace of the functions involved. For example, in
+	// Java this can include multiple causes and multiple suppressed exceptions.
+	Trace string `protobuf:"bytes,4,opt,name=trace" json:"trace,omitempty"`
+	// (Optional) A reference to the instruction this log statement is associated
+	// with.
+	InstructionReference string `protobuf:"bytes,5,opt,name=instruction_reference,json=instructionReference" json:"instruction_reference,omitempty"`
+	// (Optional) A reference to the primitive transform this log statement is
+	// associated with.
+	PrimitiveTransformReference string `protobuf:"bytes,6,opt,name=primitive_transform_reference,json=primitiveTransformReference" json:"primitive_transform_reference,omitempty"`
+	// (Optional) Human-readable name of the function or method being invoked,
+	// with optional context such as the class or package name. The format can
+	// vary by language. For example:
+	//   qual.if.ied.Class.method (Java)
+	//   dir/package.func (Go)
+	//   module.function (Python)
+	//   file.cc:382 (C++)
+	LogLocation string `protobuf:"bytes,7,opt,name=log_location,json=logLocation" json:"log_location,omitempty"`
+	// (Optional) The name of the thread this log statement is associated with.
+	Thread string `protobuf:"bytes,8,opt,name=thread" json:"thread,omitempty"`
+}
+
+func (m *LogEntry) Reset()                    { *m = LogEntry{} }
+func (m *LogEntry) String() string            { return proto.CompactTextString(m) }
+func (*LogEntry) ProtoMessage()               {}
+func (*LogEntry) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{27} }
+
+func (m *LogEntry) GetSeverity() LogEntry_Severity_Enum {
+	if m != nil {
+		return m.Severity
+	}
+	return LogEntry_Severity_UNSPECIFIED
+}
+
+func (m *LogEntry) GetTimestamp() *google_protobuf1.Timestamp {
+	if m != nil {
+		return m.Timestamp
+	}
+	return nil
+}
+
+func (m *LogEntry) GetMessage() string {
+	if m != nil {
+		return m.Message
+	}
+	return ""
+}
+
+func (m *LogEntry) GetTrace() string {
+	if m != nil {
+		return m.Trace
+	}
+	return ""
+}
+
+func (m *LogEntry) GetInstructionReference() string {
+	if m != nil {
+		return m.InstructionReference
+	}
+	return ""
+}
+
+func (m *LogEntry) GetPrimitiveTransformReference() string {
+	if m != nil {
+		return m.PrimitiveTransformReference
+	}
+	return ""
+}
+
+func (m *LogEntry) GetLogLocation() string {
+	if m != nil {
+		return m.LogLocation
+	}
+	return ""
+}
+
+func (m *LogEntry) GetThread() string {
+	if m != nil {
+		return m.Thread
+	}
+	return ""
+}
+
+// A list of log entries, enables buffering and batching of multiple
+// log messages using the logging API.
+type LogEntry_List struct {
+	// (Required) One or or more log messages.
+	LogEntries []*LogEntry `protobuf:"bytes,1,rep,name=log_entries,json=logEntries" json:"log_entries,omitempty"`
+}
+
+func (m *LogEntry_List) Reset()                    { *m = LogEntry_List{} }
+func (m *LogEntry_List) String() string            { return proto.CompactTextString(m) }
+func (*LogEntry_List) ProtoMessage()               {}
+func (*LogEntry_List) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{27, 0} }
+
+func (m *LogEntry_List) GetLogEntries() []*LogEntry {
+	if m != nil {
+		return m.LogEntries
+	}
+	return nil
+}
+
+// The severity of the event described in a log entry, expressed as one of the
+// severity levels listed below. For your reference, the levels are
+// assigned the listed numeric values. The effect of using numeric values
+// other than those listed is undefined.
+//
+// If you are writing log entries, you should map other severity encodings to
+// one of these standard levels. For example, you might map all of
+// Java's FINE, FINER, and FINEST levels to `Severity.DEBUG`.
+//
+// This list is intentionally not comprehensive; the intent is to provide a
+// common set of "good enough" severity levels so that logging front ends
+// can provide filtering and searching across log types. Users of the API are
+// free not to use all severity levels in their log messages.
+type LogEntry_Severity struct {
+}
+
+func (m *LogEntry_Severity) Reset()                    { *m = LogEntry_Severity{} }
+func (m *LogEntry_Severity) String() string            { return proto.CompactTextString(m) }
+func (*LogEntry_Severity) ProtoMessage()               {}
+func (*LogEntry_Severity) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{27, 1} }
+
+type LogControl struct {
+}
+
+func (m *LogControl) Reset()                    { *m = LogControl{} }
+func (m *LogControl) String() string            { return proto.CompactTextString(m) }
+func (*LogControl) ProtoMessage()               {}
+func (*LogControl) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{28} }
+
+// A Docker container configuration for launching the SDK harness to execute
+// user specified functions.
+type DockerContainer struct {
+	// (Required) A pipeline level unique id which can be used as a reference to
+	// refer to this.
+	Id string `protobuf:"bytes,1,opt,name=id" json:"id,omitempty"`
+	// (Required) The Docker container URI
+	// For example "dataflow.gcr.io/v1beta3/java-batch:1.5.1"
+	Uri string `protobuf:"bytes,2,opt,name=uri" json:"uri,omitempty"`
+	// (Optional) Docker registry specification.
+	// If unspecified, the uri is expected to be able to be fetched without
+	// requiring additional configuration by a runner.
+	RegistryReference string `protobuf:"bytes,3,opt,name=registry_reference,json=registryReference" json:"registry_reference,omitempty"`
+}
+
+func (m *DockerContainer) Reset()                    { *m = DockerContainer{} }
+func (m *DockerContainer) String() string            { return proto.CompactTextString(m) }
+func (*DockerContainer) ProtoMessage()               {}
+func (*DockerContainer) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{29} }
+
+func (m *DockerContainer) GetId() string {
+	if m != nil {
+		return m.Id
+	}
+	return ""
+}
+
+func (m *DockerContainer) GetUri() string {
+	if m != nil {
+		return m.Uri
+	}
+	return ""
+}
+
+func (m *DockerContainer) GetRegistryReference() string {
+	if m != nil {
+		return m.RegistryReference
+	}
+	return ""
+}
+
+func init() {
+	proto.RegisterType((*Target)(nil), "org.apache.beam.model.fn_execution.v1.Target")
+	proto.RegisterType((*Target_List)(nil), "org.apache.beam.model.fn_execution.v1.Target.List")
+	proto.RegisterType((*RemoteGrpcPort)(nil), "org.apache.beam.model.fn_execution.v1.RemoteGrpcPort")
+	proto.RegisterType((*InstructionRequest)(nil), "org.apache.beam.model.fn_execution.v1.InstructionRequest")
+	proto.RegisterType((*InstructionResponse)(nil), "org.apache.beam.model.fn_execution.v1.InstructionResponse")
+	proto.RegisterType((*RegisterRequest)(nil), "org.apache.beam.model.fn_execution.v1.RegisterRequest")
+	proto.RegisterType((*RegisterResponse)(nil), "org.apache.beam.model.fn_execution.v1.RegisterResponse")
+	proto.RegisterType((*ProcessBundleDescriptor)(nil), "org.apache.beam.model.fn_execution.v1.ProcessBundleDescriptor")
+	proto.RegisterType((*ProcessBundleRequest)(nil), "org.apache.beam.model.fn_execution.v1.ProcessBundleRequest")
+	proto.RegisterType((*ProcessBundleResponse)(nil), "org.apache.beam.model.fn_execution.v1.ProcessBundleResponse")
+	proto.RegisterType((*ProcessBundleProgressRequest)(nil), "org.apache.beam.model.fn_execution.v1.ProcessBundleProgressRequest")
+	proto.RegisterType((*Metrics)(nil), "org.apache.beam.model.fn_execution.v1.Metrics")
+	proto.RegisterType((*Metrics_PTransform)(nil), "org.apache.beam.model.fn_execution.v1.Metrics.PTransform")
+	proto.RegisterType((*Metrics_PTransform_Measured)(nil), "org.apache.beam.model.fn_execution.v1.Metrics.PTransform.Measured")
+	proto.RegisterType((*Metrics_PTransform_ProcessedElements)(nil), "org.apache.beam.model.fn_execution.v1.Metrics.PTransform.ProcessedElements")
+	proto.RegisterType((*Metrics_PTransform_ActiveElements)(nil), "org.apache.beam.model.fn_execution.v1.Metrics.PTransform.ActiveElements")
+	proto.RegisterType((*Metrics_User)(nil), "org.apache.beam.model.fn_execution.v1.Metrics.User")
+	proto.RegisterType((*ProcessBundleProgressResponse)(nil), "org.apache.beam.model.fn_execution.v1.ProcessBundleProgressResponse")
+	proto.RegisterType((*ProcessBundleSplitRequest)(nil), "org.apache.beam.model.fn_execution.v1.ProcessBundleSplitRequest")
+	proto.RegisterType((*ElementCountRestriction)(nil), "org.apache.beam.model.fn_execution.v1.ElementCountRestriction")
+	proto.RegisterType((*ElementCountSkipRestriction)(nil), "org.apache.beam.model.fn_execution.v1.ElementCountSkipRestriction")
+	proto.RegisterType((*PrimitiveTransformSplit)(nil), "org.apache.beam.model.fn_execution.v1.PrimitiveTransformSplit")
+	proto.RegisterType((*ProcessBundleSplitResponse)(nil), "org.apache.beam.model.fn_execution.v1.ProcessBundleSplitResponse")
+	proto.RegisterType((*Elements)(nil), "org.apache.beam.model.fn_execution.v1.Elements")
+	proto.RegisterType((*Elements_Data)(nil), "org.apache.beam.model.fn_execution.v1.Elements.Data")
+	proto.RegisterType((*StateRequest)(nil), "org.apache.beam.model.fn_execution.v1.StateRequest")
+	proto.RegisterType((*StateResponse)(nil), "org.apache.beam.model.fn_execution.v1.StateResponse")
+	proto.RegisterType((*StateKey)(nil), "org.apache.beam.model.fn_execution.v1.StateKey")
+	proto.RegisterType((*StateKey_Runner)(nil), "org.apache.beam.model.fn_execution.v1.StateKey.Runner")
+	proto.RegisterType((*StateKey_MultimapSideInput)(nil), "org.apache.beam.model.fn_execution.v1.StateKey.MultimapSideInput")
+	proto.RegisterType((*StateKey_BagUserState)(nil), "org.apache.beam.model.fn_execution.v1.StateKey.BagUserState")
+	proto.RegisterType((*StateGetRequest)(nil), "org.apache.beam.model.fn_execution.v1.StateGetRequest")
+	proto.RegisterType((*StateGetResponse)(nil), "org.apache.beam.model.fn_execution.v1.StateGetResponse")
+	proto.RegisterType((*StateAppendRequest)(nil), "org.apache.beam.model.fn_execution.v1.StateAppendRequest")
+	proto.RegisterType((*StateAppendResponse)(nil), "org.apache.beam.model.fn_execution.v1.StateAppendResponse")
+	proto.RegisterType((*StateClearRequest)(nil), "org.apache.beam.model.fn_execution.v1.StateClearRequest")
+	proto.RegisterType((*StateClearResponse)(nil), "org.apache.beam.model.fn_execution.v1.StateClearResponse")
+	proto.RegisterType((*LogEntry)(nil), "org.apache.beam.model.fn_execution.v1.LogEntry")
+	proto.RegisterType((*LogEntry_List)(nil), "org.apache.beam.model.fn_execution.v1.LogEntry.List")
+	proto.RegisterType((*LogEntry_Severity)(nil), "org.apache.beam.model.fn_execution.v1.LogEntry.Severity")
+	proto.RegisterType((*LogControl)(nil), "org.apache.beam.model.fn_execution.v1.LogControl")
+	proto.RegisterType((*DockerContainer)(nil), "org.apache.beam.model.fn_execution.v1.DockerContainer")
+	proto.RegisterEnum("org.apache.beam.model.fn_execution.v1.LogEntry_Severity_Enum", LogEntry_Severity_Enum_name, LogEntry_Severity_Enum_value)
+}
+
+// Reference imports to suppress errors if they are not otherwise used.
+var _ context.Context
+var _ grpc.ClientConn
+
+// This is a compile-time assertion to ensure that this generated file
+// is compatible with the grpc package it is being compiled against.
+const _ = grpc.SupportPackageIsVersion4
+
+// Client API for BeamFnControl service
+
+type BeamFnControlClient interface {
+	// Instructions sent by the runner to the SDK requesting different types
+	// of work.
+	Control(ctx context.Context, opts ...grpc.CallOption) (BeamFnControl_ControlClient, error)
+}
+
+type beamFnControlClient struct {
+	cc *grpc.ClientConn
+}
+
+func NewBeamFnControlClient(cc *grpc.ClientConn) BeamFnControlClient {
+	return &beamFnControlClient{cc}
+}
+
+func (c *beamFnControlClient) Control(ctx context.Context, opts ...grpc.CallOption) (BeamFnControl_ControlClient, error) {
+	stream, err := grpc.NewClientStream(ctx, &_BeamFnControl_serviceDesc.Streams[0], c.cc, "/org.apache.beam.model.fn_execution.v1.BeamFnControl/Control", opts...)
+	if err != nil {
+		return nil, err
+	}
+	x := &beamFnControlControlClient{stream}
+	return x, nil
+}
+
+type BeamFnControl_ControlClient interface {
+	Send(*InstructionResponse) error
+	Recv() (*InstructionRequest, error)
+	grpc.ClientStream
+}
+
+type beamFnControlControlClient struct {
+	grpc.ClientStream
+}
+
+func (x *beamFnControlControlClient) Send(m *InstructionResponse) error {
+	return x.ClientStream.SendMsg(m)
+}
+
+func (x *beamFnControlControlClient) Recv() (*InstructionRequest, error) {
+	m := new(InstructionRequest)
+	if err := x.ClientStream.RecvMsg(m); err != nil {
+		return nil, err
+	}
+	return m, nil
+}
+
+// Server API for BeamFnControl service
+
+type BeamFnControlServer interface {
+	// Instructions sent by the runner to the SDK requesting different types
+	// of work.
+	Control(BeamFnControl_ControlServer) error
+}
+
+func RegisterBeamFnControlServer(s *grpc.Server, srv BeamFnControlServer) {
+	s.RegisterService(&_BeamFnControl_serviceDesc, srv)
+}
+
+func _BeamFnControl_Control_Handler(srv interface{}, stream grpc.ServerStream) error {
+	return srv.(BeamFnControlServer).Control(&beamFnControlControlServer{stream})
+}
+
+type BeamFnControl_ControlServer interface {
+	Send(*InstructionRequest) error
+	Recv() (*InstructionResponse, error)
+	grpc.ServerStream
+}
+
+type beamFnControlControlServer struct {
+	grpc.ServerStream
+}
+
+func (x *beamFnControlControlServer) Send(m *InstructionRequest) error {
+	return x.ServerStream.SendMsg(m)
+}
+
+func (x *beamFnControlControlServer) Recv() (*InstructionResponse, error) {
+	m := new(InstructionResponse)
+	if err := x.ServerStream.RecvMsg(m); err != nil {
+		return nil, err
+	}
+	return m, nil
+}
+
+var _BeamFnControl_serviceDesc = grpc.ServiceDesc{
+	ServiceName: "org.apache.beam.model.fn_execution.v1.BeamFnControl",
+	HandlerType: (*BeamFnControlServer)(nil),
+	Methods:     []grpc.MethodDesc{},
+	Streams: []grpc.StreamDesc{
+		{
+			StreamName:    "Control",
+			Handler:       _BeamFnControl_Control_Handler,
+			ServerStreams: true,
+			ClientStreams: true,
+		},
+	},
+	Metadata: "beam_fn_api.proto",
+}
+
+// Client API for BeamFnData service
+
+type BeamFnDataClient interface {
+	// Used to send data between harnesses.
+	Data(ctx context.Context, opts ...grpc.CallOption) (BeamFnData_DataClient, error)
+}
+
+type beamFnDataClient struct {
+	cc *grpc.ClientConn
+}
+
+func NewBeamFnDataClient(cc *grpc.ClientConn) BeamFnDataClient {
+	return &beamFnDataClient{cc}
+}
+
+func (c *beamFnDataClient) Data(ctx context.Context, opts ...grpc.CallOption) (BeamFnData_DataClient, error) {
+	stream, err := grpc.NewClientStream(ctx, &_BeamFnData_serviceDesc.Streams[0], c.cc, "/org.apache.beam.model.fn_execution.v1.BeamFnData/Data", opts...)
+	if err != nil {
+		return nil, err
+	}
+	x := &beamFnDataDataClient{stream}
+	return x, nil
+}
+
+type BeamFnData_DataClient interface {
+	Send(*Elements) error
+	Recv() (*Elements, error)
+	grpc.ClientStream
+}
+
+type beamFnDataDataClient struct {
+	grpc.ClientStream
+}
+
+func (x *beamFnDataDataClient) Send(m *Elements) error {
+	return x.ClientStream.SendMsg(m)
+}
+
+func (x *beamFnDataDataClient) Recv() (*Elements, error) {
+	m := new(Elements)
+	if err := x.ClientStream.RecvMsg(m); err != nil {
+		return nil, err
+	}
+	return m, nil
+}
+
+// Server API for BeamFnData service
+
+type BeamFnDataServer interface {
+	// Used to send data between harnesses.
+	Data(BeamFnData_DataServer) error
+}
+
+func RegisterBeamFnDataServer(s *grpc.Server, srv BeamFnDataServer) {
+	s.RegisterService(&_BeamFnData_serviceDesc, srv)
+}
+
+func _BeamFnData_Data_Handler(srv interface{}, stream grpc.ServerStream) error {
+	return srv.(BeamFnDataServer).Data(&beamFnDataDataServer{stream})
+}
+
+type BeamFnData_DataServer interface {
+	Send(*Elements) error
+	Recv() (*Elements, error)
+	grpc.ServerStream
+}
+
+type beamFnDataDataServer struct {
+	grpc.ServerStream
+}
+
+func (x *beamFnDataDataServer) Send(m *Elements) error {
+	return x.ServerStream.SendMsg(m)
+}
+
+func (x *beamFnDataDataServer) Recv() (*Elements, error) {
+	m := new(Elements)
+	if err := x.ServerStream.RecvMsg(m); err != nil {
+		return nil, err
+	}
+	return m, nil
+}
+
+var _BeamFnData_serviceDesc = grpc.ServiceDesc{
+	ServiceName: "org.apache.beam.model.fn_execution.v1.BeamFnData",
+	HandlerType: (*BeamFnDataServer)(nil),
+	Methods:     []grpc.MethodDesc{},
+	Streams: []grpc.StreamDesc{
+		{
+			StreamName:    "Data",
+			Handler:       _BeamFnData_Data_Handler,
+			ServerStreams: true,
+			ClientStreams: true,
+		},
+	},
+	Metadata: "beam_fn_api.proto",
+}
+
+// Client API for BeamFnState service
+
+type BeamFnStateClient interface {
+	// Used to get/append/clear state stored by the runner on behalf of the SDK.
+	State(ctx context.Context, opts ...grpc.CallOption) (BeamFnState_StateClient, error)
+}
+
+type beamFnStateClient struct {
+	cc *grpc.ClientConn
+}
+
+func NewBeamFnStateClient(cc *grpc.ClientConn) BeamFnStateClient {
+	return &beamFnStateC

<TRUNCATED>

[36/50] [abbrv] beam git commit: [BEAM-3113] Disable stack trace optimization in java container

Posted by ke...@apache.org.
[BEAM-3113] Disable stack trace optimization in java container


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

Branch: refs/heads/mr-runner
Commit: 15db1dd3c285db1d2800e62af8b7df00a4ce225e
Parents: 6c27b82
Author: Henning Rohde <he...@google.com>
Authored: Fri Oct 27 13:38:26 2017 -0700
Committer: Thomas Groh <tg...@google.com>
Committed: Mon Nov 6 11:02:27 2017 -0800

----------------------------------------------------------------------
 sdks/java/container/boot.go | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/15db1dd3/sdks/java/container/boot.go
----------------------------------------------------------------------
diff --git a/sdks/java/container/boot.go b/sdks/java/container/boot.go
index a6c4406..144c4ac 100644
--- a/sdks/java/container/boot.go
+++ b/sdks/java/container/boot.go
@@ -107,6 +107,7 @@ func main() {
 
 	args := []string{
 		"-Xmx" + strconv.FormatUint(heapSizeLimit(info), 10),
+		"-XX:-OmitStackTraceInFastThrow",
 		"-cp", strings.Join(cp, ":"),
 		"org.apache.beam.fn.harness.FnHarness",
 	}


[45/50] [abbrv] beam git commit: Check that bigtableWriter is non-null before calling close().

Posted by ke...@apache.org.
Check that bigtableWriter is non-null before calling close().

TearDown can be called before a bundle, so it's necessary to
check that the writer is uninitialized.


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

Branch: refs/heads/mr-runner
Commit: 255b61595bce896cfd25a815fa5b03377bd0bce2
Parents: 3dfcb44
Author: Bill Neubauer <wc...@google.com>
Authored: Mon Nov 6 14:21:18 2017 -0800
Committer: Kenneth Knowles <ke...@apache.org>
Committed: Mon Nov 6 19:16:04 2017 -0800

----------------------------------------------------------------------
 .../java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java   | 6 ++++--
 1 file changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/255b6159/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java
index ba6c0b6..29dc269 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java
@@ -639,8 +639,10 @@ public class BigtableIO {
 
       @Teardown
       public void tearDown() throws Exception {
-        bigtableWriter.close();
-        bigtableWriter = null;
+        if (bigtableWriter != null) {
+          bigtableWriter.close();
+          bigtableWriter = null;
+        }
       }
 
       @Override


[46/50] [abbrv] beam git commit: This closes #4083: Check that bigtableWriter is non-null before calling close()

Posted by ke...@apache.org.
This closes #4083: Check that bigtableWriter is non-null before calling close()


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

Branch: refs/heads/mr-runner
Commit: 269bf89463094a17f51d61d30bedd388b04dc8c0
Parents: 3dfcb44 255b615
Author: Kenneth Knowles <ke...@apache.org>
Authored: Mon Nov 6 19:16:19 2017 -0800
Committer: Kenneth Knowles <ke...@apache.org>
Committed: Mon Nov 6 19:16:50 2017 -0800

----------------------------------------------------------------------
 .../java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java   | 6 ++++--
 1 file changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------



[50/50] [abbrv] beam git commit: This closes #4091: Merge branch 'master' upto commit 269bf89 into mr-runner

Posted by ke...@apache.org.
This closes #4091: Merge branch 'master' upto commit 269bf89 into mr-runner

  mr-runner: re-enable sdks/python module.
  mr-runner: update to 2.3.0-SNAPSHOT.
  Check that bigtableWriter is non-null before calling close().
  Added a preprocessing step to the Cloud Spanner sink.
  Fix Repackaging Configuration in the the DirectRunner
  Migrate shared Fn Execution code to Java7
  update dataflow.version
  [BEAM-3114] Generate text proto config properly in container boot code
  [BEAM-3113] Disable stack trace optimization in java container
  Fix Go package comment for syscallx
  Add all portability protos to Go
  [BEAM-2728] Extension for sketch-based statistics : HyperLogLog
  Add a runners/java-fn-execution module
  Add sdks/java/fn-execution
  [BEAM-3135] Adding futures dependency to python SDK
  Updates BigQueryTableSource to consider data in streaming buffer when determining estimated size.
  Getting AutoValue 1.5.1 working in Beam.
  Add License Header to SqlTypeUtils
  [BEAM-2203] Implement TIMESTAMPADD
  Fix working dir in website precommits
  Do not relocate generated Model Classes
  Updates Python datastore wordcount example to take a dataset parameter.
  Remove obsolete extra parameter
  [BEAM-2482] - CodedValueMutationDetector should use the coders structural value
  Changed the mutation detector to be based on structural value only
  Allows to set a Cloud Spanner host. https://batch-spanner.googleapis.com/ is set as a default host name.
  [BEAM-2468] Reading Kinesis records in the background
  [BEAM-3054] Uses locale-insensitive number formatting in ESIO and tests
  Added VoidSerializer for KafkaIO. Modified KafkaIO.Write.values() to auto add the VoidSerializer for the key.serializer config for kafka producer
  [BEAM-1542] SpannerIO: mutation encoding and size estimation improvements
  [BEAM-2979] Fix a race condition in getWatermark() in KafkaIO.
  [BEAM-3112] Improve error messages in ElasticsearchIO test utils
  [BEAM-3111] Upgrade Elasticsearch to 5.6.3 and clean pom
  Temporarily disable Dataflow pipeline_url metadata
  NonNull by default in sdk/transforms/splittabledofn
  NonNull by default in sdk/transforms/join
  NonNull by default in sdk/transforms/display
  NonNull by default in sdk/transforms/reflect
  NonNull by default in sdk/transforms/windowing
  NonNull by default in sdk/transforms
  Reading spanner schema transform
  Suppress AutoValue warnings in TextIO
  Remove extraneous type arguments in Latest.java
  Remove extraneous type arguments in PAssert
  Remove coveralls invocations from all jobs
  Adds ParseResult.failure()
  Many improvements to TikaIO
  Add missing @RunWith to test.
  [BEAM-2566] Decouple SDK harness from Dataflow runner by elevating experiments and SDK harness configuration to java-sdk-core.
  Add python_requires to setup.py
  CR: [BEAM-3005] Set JVM max heap size in java container
  Update rat exclusion for python and Go protos
  Declare .go and Dockerfile as text in gitattributes
  [BEAM-3005] Set JVM max heap size in java container
  [BEAM-3005] Add resource limits to provision proto
  Stage the pipeline in Python DataflowRunner
  Add zip to the list of accepted extra package file types.
  Fix remaining nullability errors
  Increase Java postcommit timeout to 240
  Manage RAT plugin more centrally; only toggle skipping
  Fix RAT exclusions
  Rearrange .gitignore slightly
  Fix typo in seed job
  Rename seed job so it is first in glob used by prior seed job
  Fix typo in seed jobs
  Increase seed job(s) timeout to 60 minutes
  Increase seed job(s) timeout to 30 minutes
  Make the main seed job standalone
  Increase job_beam_PreCommit_Java_MavenInstall timeout from 2.5 to 4 hours.
  Implement FnApi side inputs in Python.
  Unit test for label pipeline option
  NonNull by default in metrics
  Ignore findbugs in AutoValue generated classes
  NonNull by default for sdk/testing
  NonNull by default in sdk/state
  NonNull by default in sdk/runners
  NonNull by default in sdk/annotations
  NonNull by default in sdk/coders
  Make Java core SDK root dir NonNull by default
  Add dep on Apache-licensed findbugs-annotations implementation
  [BEAM-2682] Deletes AvroIOTransformTest
  Clone source to a distinguished subdirectory of Jenkins workspace
  Adding lull tracking for python sampler
  Avoids generating proto files for Windows if grpcio-tools is not installed.
  Do not crash when RawPTransform has null spec
  Unit test to repro NPE in PTransformTranslation
  Created Java snippets file
  Add standalone version of seed job
  Pin runner harness also for official BEAM releases.
  Remove duplicate mocking in DataflowRunnerTest
  Add assertion that valid jobs must have staged pipeline
  Stage the pipeline without using a temp file
  Stage the portable pipeline in Dataflow
  Add ability to stage explicit file list
  Improve GcsFileSystem errors messages slightly
  [BEAM-2720] Update kafka client version to 0.11.0.1
  Update PipelineTest.testReplacedNames
  DirectRunner: Replace use of RawPTransform with NotSerializable.forUrn translators
  Wordcount on fnapi pipeline and IT test.
  Clearer getOrDefault style in RehydratedComponents
  ...


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

Branch: refs/heads/mr-runner
Commit: b6f22aa76ad217fff6078e9351676895ac793ecc
Parents: 5fa0b14 202a2cb
Author: Kenneth Knowles <ke...@apache.org>
Authored: Tue Nov 7 08:53:31 2017 -0800
Committer: Kenneth Knowles <ke...@apache.org>
Committed: Tue Nov 7 08:53:31 2017 -0800

----------------------------------------------------------------------
 .gitattributes                                  |    2 +
 .gitignore                                      |    9 +-
 .test-infra/jenkins/PreCommit_Pipeline.groovy   |  129 +
 .../jenkins/common_job_properties.groovy        |  205 +-
 .test-infra/jenkins/job_00_seed.groovy          |  114 +
 .test-infra/jenkins/job_beam_Java_Build.groovy  |   82 +
 .../jenkins/job_beam_Java_CodeHealth.groovy     |   39 +
 .../job_beam_Java_IntegrationTest.groovy        |   63 +
 .../jenkins/job_beam_Java_UnitTest.groovy       |   48 +
 ...job_beam_PostCommit_Java_MavenInstall.groovy |   21 +-
 ..._PostCommit_Java_MavenInstall_Windows.groovy |    2 +-
 ...ommit_Python_ValidatesRunner_Dataflow.groovy |   54 +
 .../job_beam_PreCommit_Go_MavenInstall.groovy   |   56 +
 .../job_beam_PreCommit_Java_MavenInstall.groovy |   21 +-
 .../jenkins/job_beam_PreCommit_Pipeline.groovy  |   84 +
 ...ob_beam_PreCommit_Python_MavenInstall.groovy |   56 +
 .../job_beam_PreCommit_Website_Merge.groovy     |    3 +
 .../job_beam_PreCommit_Website_Stage.groovy     |    3 +
 .../job_beam_PreCommit_Website_Test.groovy      |    3 +
 .../jenkins/job_beam_Python_UnitTest.groovy     |   40 +
 .../job_beam_Release_NightlySnapshot.groovy     |   22 +-
 .test-infra/jenkins/job_seed.groovy             |   53 -
 .test-infra/jenkins/job_seed_standalone.groovy  |  114 +
 examples/java/pom.xml                           |    8 +-
 .../apache/beam/examples/WindowedWordCount.java |    1 -
 .../examples/cookbook/BigQueryTornadoes.java    |    2 +-
 .../cookbook/CombinePerKeyExamples.java         |    2 +-
 .../beam/examples/cookbook/FilterExamples.java  |    2 +-
 .../beam/examples/cookbook/JoinExamples.java    |    6 +-
 .../examples/cookbook/MaxPerKeyExamples.java    |    2 +-
 examples/java8/pom.xml                          |   14 +-
 .../complete/game/injector/Injector.java        |    2 +-
 .../complete/game/injector/InjectorUtils.java   |    2 +-
 .../examples/website_snippets/Snippets.java     |   87 +
 .../examples/website_snippets/SnippetsTest.java |  114 +
 examples/pom.xml                                |    2 +-
 model/fn-execution/pom.xml                      |  114 +
 .../src/main/proto/beam_fn_api.proto            |  729 ++++
 .../src/main/proto/beam_provision_api.proto     |   99 +
 .../model/fnexecution/v1/standard_coders.yaml   |  195 +
 model/job-management/pom.xml                    |  114 +
 .../src/main/proto/beam_artifact_api.proto      |  134 +
 .../src/main/proto/beam_job_api.proto           |  174 +
 model/pipeline/pom.xml                          |   89 +
 .../src/main/proto/beam_runner_api.proto        |  843 +++++
 model/pipeline/src/main/proto/endpoints.proto   |   47 +
 .../src/main/proto/standard_window_fns.proto    |   54 +
 model/pom.xml                                   |   40 +
 pom.xml                                         |  368 +-
 runners/apex/pom.xml                            |    9 +-
 .../beam/runners/apex/ApexPipelineOptions.java  |    2 +-
 .../beam/runners/apex/ApexRunnerResult.java     |    4 +-
 .../apex/translation/ParDoTranslator.java       |   24 +-
 .../operators/ApexParDoOperator.java            |   22 +-
 .../translation/utils/ApexStateInternals.java   |   17 +-
 .../apex/translation/utils/ValuesSource.java    |    4 -
 .../beam/runners/apex/ApexYarnLauncherTest.java |    9 +-
 .../apex/examples/UnboundedTextSource.java      |    4 -
 .../FlattenPCollectionTranslatorTest.java       |    1 -
 .../translation/GroupByKeyTranslatorTest.java   |    4 -
 .../apex/translation/ParDoTranslatorTest.java   |    4 +-
 .../translation/utils/CollectionSource.java     |    4 -
 runners/core-construction-java/pom.xml          |   19 +-
 .../construction/ArtifactServiceStager.java     |  244 ++
 .../core/construction/CoderTranslation.java     |   22 +-
 .../core/construction/CombineTranslation.java   |  224 +-
 .../CreatePCollectionViewTranslation.java       |   28 +-
 .../construction/DisplayDataTranslation.java    |    4 +-
 .../core/construction/FlattenTranslator.java    |   12 +-
 .../construction/GroupByKeyTranslation.java     |   17 +-
 .../construction/PCollectionTranslation.java    |   10 +-
 .../construction/PTransformTranslation.java     |  292 +-
 .../core/construction/ParDoTranslation.java     |  392 +-
 .../PipelineOptionsTranslation.java             |   51 +
 .../core/construction/PipelineTranslation.java  |  137 +-
 .../core/construction/ReadTranslation.java      |  113 +-
 .../core/construction/RehydratedComponents.java |    9 +-
 .../construction/RunnerPCollectionView.java     |    2 +-
 .../core/construction/SdkComponents.java        |   17 +-
 .../core/construction/SplittableParDo.java      |    8 +
 .../construction/TestStreamTranslation.java     |  170 +-
 .../TransformPayloadTranslatorRegistrar.java    |    2 +
 .../core/construction/TriggerTranslation.java   |   10 +-
 .../construction/WindowIntoTranslation.java     |   30 +-
 .../WindowingStrategyTranslation.java           |  150 +-
 .../construction/WriteFilesTranslation.java     |  245 +-
 .../construction/ArtifactServiceStagerTest.java |  141 +
 .../core/construction/CoderTranslationTest.java |    4 +-
 .../construction/CombineTranslationTest.java    |   20 +-
 .../CreatePCollectionViewTranslationTest.java   |   10 +-
 .../InMemoryArtifactStagerService.java          |  152 +
 .../PCollectionTranslationTest.java             |    2 +-
 .../construction/PTransformMatchersTest.java    |   15 +-
 .../construction/PTransformTranslationTest.java |   39 +-
 .../core/construction/ParDoTranslationTest.java |    9 +-
 .../PipelineOptionsTranslationTest.java         |  143 +
 .../construction/PipelineTranslationTest.java   |    2 +-
 .../core/construction/ReadTranslationTest.java  |   14 +-
 .../core/construction/SdkComponentsTest.java    |    2 +-
 .../construction/TestStreamTranslationTest.java |   12 +-
 .../construction/WindowIntoTranslationTest.java |    2 +-
 .../WindowingStrategyTranslationTest.java       |    2 +-
 .../construction/WriteFilesTranslationTest.java |   15 +-
 runners/core-java/pom.xml                       |   31 +-
 .../runners/core/InMemoryStateInternals.java    |   39 +-
 .../runners/core/InMemoryTimerInternals.java    |   28 +-
 .../beam/runners/core/SideInputHandler.java     |    2 +-
 .../core/SplittableParDoViaKeyedWorkItems.java  |    7 +
 .../apache/beam/runners/core/StateTable.java    |   40 +-
 .../org/apache/beam/runners/core/StateTags.java |   13 +
 .../core/TestInMemoryStateInternals.java        |    6 +-
 .../apache/beam/runners/core/WatermarkHold.java |    6 +-
 .../runners/core/fn/FnApiControlClient.java     |  148 +
 .../core/fn/FnApiControlClientPoolService.java  |   66 +
 .../beam/runners/core/fn/FnDataReceiver.java    |   33 +
 .../beam/runners/core/fn/FnDataService.java     |   81 +
 .../beam/runners/core/fn/SdkHarnessClient.java  |  172 +
 .../runners/core/fn/SdkHarnessDoFnRunner.java   |  102 +
 .../beam/runners/core/fn/package-info.java      |   22 +
 .../core/metrics/MetricsContainerImpl.java      |   40 +
 .../triggers/AfterWatermarkStateMachine.java    |    8 +-
 .../triggers/TriggerStateMachineRunner.java     |    3 +-
 .../core/triggers/TriggerStateMachines.java     |    2 +-
 .../beam/runners/core/ReduceFnTester.java       |   22 +-
 .../beam/runners/core/StateInternalsTest.java   |   59 +
 .../fn/FnApiControlClientPoolServiceTest.java   |   65 +
 .../runners/core/fn/FnApiControlClientTest.java |  139 +
 .../runners/core/fn/SdkHarnessClientTest.java   |   96 +
 .../core/fn/SdkHarnessDoFnRunnerTest.java       |   73 +
 .../core/metrics/MetricsContainerImplTest.java  |   10 +
 .../AfterWatermarkStateMachineTest.java         |   29 +
 .../triggers/TriggerStateMachineTester.java     |   13 +-
 .../core/triggers/TriggerStateMachinesTest.java |    2 +-
 runners/direct-java/pom.xml                     |   14 +-
 .../CopyOnAccessInMemoryStateInternals.java     |   10 +-
 .../beam/runners/direct/DirectGroupByKey.java   |   16 +-
 .../beam/runners/direct/DirectRunner.java       |    5 +-
 .../GroupAlsoByWindowEvaluatorFactory.java      |    2 +-
 .../beam/runners/direct/MultiStepCombine.java   |   18 +-
 .../runners/direct/ParDoEvaluatorFactory.java   |   30 +-
 .../direct/ParDoMultiOverrideFactory.java       |    9 +-
 ...littableProcessElementsEvaluatorFactory.java |    3 +-
 .../direct/StatefulParDoEvaluatorFactory.java   |   15 +-
 .../direct/TestStreamEvaluatorFactory.java      |    9 +-
 .../direct/TransformEvaluatorRegistry.java      |   37 +-
 .../runners/direct/ViewOverrideFactory.java     |    8 +-
 .../direct/WriteWithShardingFactory.java        |   37 +-
 .../direct/BoundedReadEvaluatorFactoryTest.java |    4 -
 .../CopyOnAccessInMemoryStateInternalsTest.java |   74 +-
 .../UnboundedReadEvaluatorFactoryTest.java      |    3 -
 .../direct/WriteWithShardingFactoryTest.java    |   11 +-
 runners/flink/pom.xml                           |    4 +-
 .../runners/flink/CreateStreamingFlinkView.java |    3 +
 .../flink/FlinkBatchPipelineTranslator.java     |    3 +-
 .../runners/flink/FlinkPipelineOptions.java     |   11 +
 .../FlinkStreamingTransformTranslators.java     |  244 +-
 .../functions/FlinkAssignContext.java           |   17 +-
 .../translation/types/CoderTypeSerializer.java  |    2 +-
 .../wrappers/streaming/DoFnOperator.java        |  429 ++-
 .../streaming/SplittableDoFnOperator.java       |    4 +-
 .../wrappers/streaming/WindowDoFnOperator.java  |    4 +-
 .../state/FlinkSplitStateInternals.java         |    8 +-
 .../streaming/state/FlinkStateInternals.java    |   13 +-
 .../beam/runners/flink/PipelineOptionsTest.java |   21 +-
 .../flink/streaming/DoFnOperatorTest.java       |  161 +-
 .../flink/streaming/TestCountingSource.java     |    3 -
 .../types/CoderTypeSerializerTest.java          |   79 +
 runners/gcp/gcemd/Dockerfile                    |   30 +
 runners/gcp/gcemd/main.go                       |   85 +
 runners/gcp/gcemd/pom.xml                       |  154 +
 runners/gcp/gcsproxy/Dockerfile                 |   30 +
 runners/gcp/gcsproxy/main.go                    |   91 +
 runners/gcp/gcsproxy/pom.xml                    |  154 +
 runners/gcp/pom.xml                             |   38 +
 runners/gearpump/pom.xml                        |   21 +-
 .../gearpump/GearpumpPipelineResult.java        |    3 +-
 .../translators/GroupByKeyTranslator.java       |    2 +-
 .../gearpump/translators/io/GearpumpSource.java |    7 +-
 .../gearpump/translators/io/ValuesSource.java   |    8 -
 .../translators/utils/DoFnRunnerFactory.java    |    7 +-
 .../translators/utils/TranslatorUtils.java      |   20 -
 .../FlattenPCollectionsTranslatorTest.java      |    6 +
 runners/google-cloud-dataflow-java/pom.xml      |   22 +-
 .../beam/runners/dataflow/AssignWindows.java    |   84 -
 .../dataflow/BatchStatefulParDoOverrides.java   |   71 +-
 .../dataflow/DataflowPipelineTranslator.java    |    5 +-
 .../beam/runners/dataflow/DataflowRunner.java   |   69 +-
 .../runners/dataflow/DataflowRunnerInfo.java    |   91 +-
 .../dataflow/PrimitiveParDoSingleFactory.java   |    2 +-
 .../dataflow/options/CloudDebuggerOptions.java  |    3 +-
 .../options/DataflowPipelineDebugOptions.java   |   23 +-
 .../options/DataflowPipelineOptions.java        |    8 +
 .../options/DataflowProfilingOptions.java       |    3 +-
 .../options/DataflowWorkerLoggingOptions.java   |    6 +
 ...aultCoderCloudObjectTranslatorRegistrar.java |    2 +
 .../beam/runners/dataflow/util/DoFnInfo.java    |  104 -
 .../beam/runners/dataflow/util/GcsStager.java   |   53 +-
 .../runners/dataflow/util/MonitoringUtil.java   |    2 +-
 .../beam/runners/dataflow/util/PackageUtil.java |  581 +--
 .../beam/runners/dataflow/util/Stager.java      |   32 +-
 .../BatchStatefulParDoOverridesTest.java        |   45 +-
 .../DataflowPipelineTranslatorTest.java         |    4 +-
 .../dataflow/DataflowRunnerInfoTest.java        |    9 +
 .../runners/dataflow/DataflowRunnerTest.java    |  107 +-
 .../dataflow/util/MonitoringUtilTest.java       |    4 +-
 .../runners/dataflow/util/PackageUtilTest.java  |  124 +-
 runners/java-fn-execution/pom.xml               |   91 +
 .../beam/runners/fnexecution/ServerFactory.java |  104 +
 .../beam/runners/fnexecution/package-info.java  |   23 +
 .../runners/fnexecution/ServerFactoryTest.java  |  153 +
 runners/local-artifact-service-java/pom.xml     |  116 +
 .../LocalFileSystemArtifactStagerService.java   |  279 ++
 .../beam/artifact/local/package-info.java       |   22 +
 ...ocalFileSystemArtifactStagerServiceTest.java |  301 ++
 runners/map-reduce/pom.xml                      |    2 +-
 runners/pom.xml                                 |    6 +-
 runners/reference/job-server/pom.xml            |   82 +
 .../reference/job/ReferenceRunnerJobServer.java |   77 +
 .../job/ReferenceRunnerJobService.java          |   79 +
 .../runners/reference/job/package-info.java     |   23 +
 .../job/ReferenceRunnerJobServiceTest.java      |   34 +
 runners/reference/pom.xml                       |   39 +
 runners/spark/pom.xml                           |    6 +-
 .../apache/beam/runners/spark/SparkRunner.java  |    5 +-
 .../beam/runners/spark/io/CreateStream.java     |  104 +-
 .../SparkGroupAlsoByWindowViaWindowSet.java     |  819 ++--
 .../spark/stateful/SparkTimerInternals.java     |   21 +-
 .../spark/translation/BoundedDataset.java       |   17 +-
 .../beam/runners/spark/translation/Dataset.java |    3 +-
 .../spark/translation/EvaluationContext.java    |   23 +-
 .../spark/translation/SparkContextFactory.java  |    2 -
 .../translation/StorageLevelPTransform.java     |   37 -
 .../spark/translation/TransformTranslator.java  |   53 +-
 .../spark/translation/TranslationUtils.java     |   78 +
 .../streaming/StreamingTransformTranslator.java |   86 +-
 .../translation/streaming/UnboundedDataset.java |   27 +-
 .../streaming/WatermarkSyncedDStream.java       |  149 +
 .../spark/util/GlobalWatermarkHolder.java       |  302 +-
 .../runners/spark/SparkPipelineStateTest.java   |    4 +-
 .../runners/spark/SparkRunnerDebuggerTest.java  |   11 +-
 .../spark/translation/StorageLevelTest.java     |   75 -
 .../translation/streaming/CreateStreamTest.java |   33 +-
 .../spark/src/test/resources/log4j.properties   |   11 +-
 sdks/CONTAINERS.md                              |  162 +
 sdks/common/fn-api/pom.xml                      |  109 -
 .../fn-api/src/main/proto/beam_fn_api.proto     |  659 ----
 .../org/apache/beam/fn/v1/standard_coders.yaml  |  195 -
 sdks/common/pom.xml                             |   39 -
 sdks/common/runner-api/pom.xml                  |  109 -
 .../src/main/proto/beam_job_api.proto           |  143 -
 .../src/main/proto/beam_runner_api.proto        |  812 ----
 .../src/main/proto/standard_window_fns.proto    |   53 -
 sdks/go/BUILD.md                                |   63 +
 sdks/go/cmd/beamctl/cmd/artifact.go             |   98 +
 sdks/go/cmd/beamctl/cmd/root.go                 |   56 +
 sdks/go/cmd/beamctl/main.go                     |   31 +
 sdks/go/descriptor.xml                          |   29 +
 sdks/go/pkg/beam/artifact/gcsproxy/retrieval.go |  155 +
 sdks/go/pkg/beam/artifact/gcsproxy/staging.go   |  200 +
 sdks/go/pkg/beam/artifact/materialize.go        |  240 ++
 sdks/go/pkg/beam/artifact/materialize_test.go   |  238 ++
 sdks/go/pkg/beam/artifact/server_test.go        |  213 ++
 sdks/go/pkg/beam/artifact/stage.go              |  238 ++
 sdks/go/pkg/beam/artifact/stage_test.go         |   98 +
 .../beam/model/fnexecution_v1/beam_fn_api.pb.go | 2729 ++++++++++++++
 .../fnexecution_v1/beam_provision_api.pb.go     |  306 ++
 sdks/go/pkg/beam/model/gen.go                   |   22 +
 .../jobmanagement_v1/beam_artifact_api.pb.go    |  690 ++++
 .../model/jobmanagement_v1/beam_job_api.pb.go   |  903 +++++
 .../model/pipeline_v1/beam_runner_api.pb.go     | 3491 ++++++++++++++++++
 .../pkg/beam/model/pipeline_v1/endpoints.pb.go  |  160 +
 .../model/pipeline_v1/standard_window_fns.pb.go |  120 +
 sdks/go/pkg/beam/provision/provision_test.go    |   54 +
 sdks/go/pkg/beam/provision/provison.go          |   80 +
 sdks/go/pkg/beam/util/errorx/guarded.go         |   47 +
 sdks/go/pkg/beam/util/execx/exec.go             |   33 +
 sdks/go/pkg/beam/util/gcsx/gcs.go               |   88 +
 sdks/go/pkg/beam/util/grpcx/dial.go             |   37 +
 sdks/go/pkg/beam/util/grpcx/metadata.go         |   55 +
 sdks/go/pkg/beam/util/syscallx/syscall.go       |   27 +
 .../pkg/beam/util/syscallx/syscall_default.go   |   28 +
 sdks/go/pkg/beam/util/syscallx/syscall_linux.go |   38 +
 sdks/go/pom.xml                                 |  163 +
 sdks/java/build-tools/pom.xml                   |    2 +-
 .../src/main/resources/beam/findbugs-filter.xml |   51 +-
 .../resources/docker/file/openjdk7/Dockerfile   |   49 +
 .../docker/file/openjdk7/docker-entrypoint.sh   |   24 +
 .../resources/docker/file/openjdk8/Dockerfile   |   49 +
 .../docker/file/openjdk8/docker-entrypoint.sh   |   24 +
 .../resources/docker/git/openjdk8/Dockerfile    |   53 +
 .../docker/git/openjdk8/docker-entrypoint.sh    |   22 +
 .../resources/docker/release/python2/Dockerfile |   21 +
 sdks/java/container/Dockerfile                  |   28 +
 sdks/java/container/boot.go                     |  134 +
 sdks/java/container/pom.xml                     |  184 +
 sdks/java/core/pom.xml                          |   15 +-
 .../main/java/org/apache/beam/sdk/Pipeline.java |   12 +-
 .../org/apache/beam/sdk/PipelineResult.java     |   14 +-
 .../beam/sdk/annotations/Experimental.java      |    8 +-
 .../beam/sdk/annotations/package-info.java      |    4 +
 .../apache/beam/sdk/coders/BeamRecordCoder.java |  111 +
 .../apache/beam/sdk/coders/CoderRegistry.java   |    6 +
 .../apache/beam/sdk/coders/DefaultCoder.java    |   17 +-
 .../beam/sdk/coders/LengthPrefixCoder.java      |    3 +-
 .../beam/sdk/coders/SerializableCoder.java      |    5 +-
 .../apache/beam/sdk/coders/StructuredCoder.java |   42 +-
 .../org/apache/beam/sdk/coders/VoidCoder.java   |   12 +-
 .../apache/beam/sdk/coders/package-info.java    |    4 +
 .../java/org/apache/beam/sdk/io/AvroIO.java     |  544 ++-
 .../java/org/apache/beam/sdk/io/AvroSink.java   |    2 +-
 .../java/org/apache/beam/sdk/io/AvroSource.java |   32 +-
 .../apache/beam/sdk/io/BlockBasedSource.java    |   27 +-
 .../apache/beam/sdk/io/CompressedSource.java    |  292 +-
 .../org/apache/beam/sdk/io/Compression.java     |  228 ++
 .../org/apache/beam/sdk/io/CountingSource.java  |    3 -
 .../beam/sdk/io/DefaultFilenamePolicy.java      |   25 +-
 .../org/apache/beam/sdk/io/FileBasedSink.java   |  138 +-
 .../org/apache/beam/sdk/io/FileBasedSource.java |   29 +-
 .../java/org/apache/beam/sdk/io/FileIO.java     |  450 +++
 .../apache/beam/sdk/io/GenerateSequence.java    |    3 +-
 .../org/apache/beam/sdk/io/LocalFileSystem.java |    9 +-
 .../main/java/org/apache/beam/sdk/io/Match.java |  156 -
 .../beam/sdk/io/ReadAllViaFileBasedSource.java  |  100 +-
 .../apache/beam/sdk/io/ReadableFileCoder.java   |   50 +
 .../java/org/apache/beam/sdk/io/Source.java     |    2 +-
 .../java/org/apache/beam/sdk/io/TFRecordIO.java |  171 +-
 .../java/org/apache/beam/sdk/io/TextIO.java     |  590 +--
 .../java/org/apache/beam/sdk/io/TextSource.java |  117 +-
 .../java/org/apache/beam/sdk/io/WriteFiles.java |  127 +-
 .../apache/beam/sdk/io/WriteFilesResult.java    |   81 +
 .../org/apache/beam/sdk/io/package-info.java    |    2 +-
 .../beam/sdk/io/range/ByteKeyRangeTracker.java  |    5 +
 .../beam/sdk/io/range/OffsetRangeTracker.java   |   22 +-
 .../org/apache/beam/sdk/metrics/MetricName.java |    7 +
 .../apache/beam/sdk/metrics/package-info.java   |    4 +
 .../DefaultPipelineOptionsRegistrar.java        |    2 +
 .../beam/sdk/options/ExperimentalOptions.java   |   38 +
 .../beam/sdk/options/PipelineOptions.java       |   36 +
 .../sdk/options/PipelineOptionsFactory.java     |   45 +
 .../beam/sdk/options/SdkHarnessOptions.java     |  173 +
 .../apache/beam/sdk/options/ValueProvider.java  |   28 +-
 .../apache/beam/sdk/options/ValueProviders.java |   15 +-
 .../java/org/apache/beam/sdk/package-info.java  |    4 +
 .../beam/sdk/runners/TransformHierarchy.java    |   19 +-
 .../apache/beam/sdk/runners/package-info.java   |    4 +
 .../org/apache/beam/sdk/state/BagState.java     |    6 +
 .../apache/beam/sdk/state/CombiningState.java   |    5 +
 .../apache/beam/sdk/state/GroupingState.java    |   12 +-
 .../org/apache/beam/sdk/state/MapState.java     |   20 +-
 .../apache/beam/sdk/state/ReadableState.java    |    6 +
 .../apache/beam/sdk/state/ReadableStates.java   |    4 +-
 .../org/apache/beam/sdk/state/SetState.java     |   10 +-
 .../org/apache/beam/sdk/state/StateSpecs.java   |    4 +-
 .../org/apache/beam/sdk/state/package-info.java |    4 +
 .../beam/sdk/testing/FileChecksumMatcher.java   |   41 +-
 .../org/apache/beam/sdk/testing/PAssert.java    |   74 +-
 .../apache/beam/sdk/testing/PaneExtractors.java |   25 +-
 .../beam/sdk/testing/SourceTestUtils.java       |   11 +
 .../beam/sdk/testing/SuccessOrFailure.java      |   24 +-
 .../apache/beam/sdk/testing/TestPipeline.java   |   53 +-
 .../beam/sdk/testing/WindowFnTestUtils.java     |  141 +-
 .../apache/beam/sdk/testing/WindowSupplier.java |    4 +-
 .../apache/beam/sdk/testing/package-info.java   |    5 +
 .../sdk/transforms/ApproximateQuantiles.java    |   10 +-
 .../beam/sdk/transforms/ApproximateUnique.java  |    4 +-
 .../org/apache/beam/sdk/transforms/Combine.java |   10 +-
 .../apache/beam/sdk/transforms/CombineFns.java  |    2 +
 .../apache/beam/sdk/transforms/Contextful.java  |  127 +
 .../org/apache/beam/sdk/transforms/Create.java  |    5 +-
 .../apache/beam/sdk/transforms/Distinct.java    |    2 +
 .../apache/beam/sdk/transforms/DoFnTester.java  |   27 +-
 .../beam/sdk/transforms/FlatMapElements.java    |  150 +-
 .../org/apache/beam/sdk/transforms/Latest.java  |    2 +-
 .../apache/beam/sdk/transforms/MapElements.java |   77 +-
 .../org/apache/beam/sdk/transforms/Max.java     |    5 +-
 .../org/apache/beam/sdk/transforms/Min.java     |    8 +-
 .../apache/beam/sdk/transforms/PTransform.java  |    7 +-
 .../org/apache/beam/sdk/transforms/ParDo.java   |    8 +-
 .../beam/sdk/transforms/Requirements.java       |   61 +
 .../apache/beam/sdk/transforms/Reshuffle.java   |   47 +
 .../org/apache/beam/sdk/transforms/Top.java     |    5 +-
 .../org/apache/beam/sdk/transforms/View.java    |    9 +-
 .../org/apache/beam/sdk/transforms/Watch.java   |   82 +-
 .../apache/beam/sdk/transforms/WithKeys.java    |   12 +-
 .../sdk/transforms/display/DisplayData.java     |   25 +-
 .../sdk/transforms/display/package-info.java    |    4 +
 .../beam/sdk/transforms/join/CoGbkResult.java   |   15 +-
 .../transforms/join/KeyedPCollectionTuple.java  |   10 +-
 .../beam/sdk/transforms/join/RawUnionValue.java |    8 +-
 .../beam/sdk/transforms/join/package-info.java  |    4 +
 .../beam/sdk/transforms/package-info.java       |    4 +
 .../reflect/ByteBuddyDoFnInvokerFactory.java    |    3 +-
 .../sdk/transforms/reflect/DoFnInvoker.java     |   50 +-
 .../sdk/transforms/reflect/DoFnSignatures.java  |    1 +
 .../sdk/transforms/reflect/package-info.java    |    3 +
 .../splittabledofn/OffsetRangeTracker.java      |    5 +-
 .../transforms/splittabledofn/package-info.java |    4 +
 .../windowing/IncompatibleWindowException.java  |    2 +-
 .../MergeOverlappingIntervalWindows.java        |    4 +-
 .../beam/sdk/transforms/windowing/Trigger.java  |   20 +-
 .../beam/sdk/transforms/windowing/Window.java   |    1 +
 .../sdk/transforms/windowing/package-info.java  |    4 +
 .../org/apache/beam/sdk/util/ApiSurface.java    |    2 +
 .../org/apache/beam/sdk/util/CoderUtils.java    |    2 +-
 .../java/org/apache/beam/sdk/util/DoFnInfo.java |  104 +
 .../apache/beam/sdk/util/MutationDetectors.java |   79 +-
 .../beam/sdk/util/SerializableThrowable.java    |   49 +
 .../apache/beam/sdk/util/SerializableUtils.java |   69 +-
 .../org/apache/beam/sdk/values/BeamRecord.java  |  319 ++
 .../apache/beam/sdk/values/BeamRecordType.java  |   96 +
 .../apache/beam/sdk/values/TypeDescriptors.java |   37 +-
 .../java/org/apache/beam/sdk/PipelineTest.java  |   51 +-
 .../apache/beam/sdk/coders/AvroCoderTest.java   |   35 +-
 .../apache/beam/sdk/coders/CommonCoderTest.java |    4 +-
 .../apache/beam/sdk/coders/VoidCoderTest.java   |   11 +-
 .../java/org/apache/beam/sdk/io/AvroIOTest.java |  417 ++-
 .../apache/beam/sdk/io/AvroIOTransformTest.java |  324 --
 .../beam/sdk/io/CompressedSourceTest.java       |   96 +-
 .../apache/beam/sdk/io/FileBasedSinkTest.java   |   43 +-
 .../apache/beam/sdk/io/FileBasedSourceTest.java |    3 -
 .../java/org/apache/beam/sdk/io/FileIOTest.java |  313 ++
 .../org/apache/beam/sdk/io/FileSystemsTest.java |   15 +-
 .../beam/sdk/io/OffsetBasedSourceTest.java      |    3 -
 .../java/org/apache/beam/sdk/io/ReadTest.java   |    6 -
 .../java/org/apache/beam/sdk/io/SimpleSink.java |   23 +-
 .../org/apache/beam/sdk/io/TFRecordIOTest.java  |   35 +-
 .../org/apache/beam/sdk/io/TextIOReadTest.java  | 1305 ++++---
 .../org/apache/beam/sdk/io/TextIOWriteTest.java |   40 +-
 .../org/apache/beam/sdk/io/WriteFilesTest.java  |   11 +-
 .../sdk/io/range/ByteKeyRangeTrackerTest.java   |   23 +
 .../apache/beam/sdk/metrics/MetricsTest.java    |   28 +
 .../sdk/options/PipelineOptionsFactoryTest.java |   39 +
 .../beam/sdk/options/PipelineOptionsTest.java   |   11 +
 .../sdk/options/ProxyInvocationHandlerTest.java |    4 +-
 .../beam/sdk/options/SdkHarnessOptionsTest.java |   76 +
 .../beam/sdk/options/ValueProviderTest.java     |   38 +-
 .../runners/dataflow/TestCountingSource.java    |    3 -
 .../sdk/testing/InterceptingUrlClassLoader.java |   57 +
 .../apache/beam/sdk/testing/PAssertTest.java    |   41 +-
 .../beam/sdk/testing/PaneExtractorsTest.java    |    7 +-
 .../beam/sdk/testing/TestPipelineTest.java      |   37 +-
 .../transforms/ApproximateQuantilesTest.java    |  528 +--
 .../apache/beam/sdk/transforms/CreateTest.java  |   22 +-
 .../sdk/transforms/FlatMapElementsTest.java     |   35 +-
 .../apache/beam/sdk/transforms/FlattenTest.java |   35 +
 .../beam/sdk/transforms/MapElementsTest.java    |   42 +-
 .../apache/beam/sdk/transforms/ParDoTest.java   |   44 +-
 .../apache/beam/sdk/transforms/WatchTest.java   |   46 +-
 .../beam/sdk/transforms/WithKeysTest.java       |   30 +-
 .../transforms/reflect/DoFnInvokersTest.java    |   12 +-
 .../sdk/transforms/windowing/WindowTest.java    |  125 +-
 .../beam/sdk/util/MutationDetectorsTest.java    |   56 +
 .../beam/sdk/util/SerializableUtilsTest.java    |   60 +
 .../beam/sdk/values/TypeDescriptorsTest.java    |   17 +-
 .../google-cloud-platform-core/pom.xml          |    2 +-
 .../extensions/gcp/storage/GcsFileSystem.java   |    5 +-
 .../java/org/apache/beam/sdk/util/GcsUtil.java  |    6 +-
 .../org/apache/beam/sdk/util/GcsUtilTest.java   |   45 +
 sdks/java/extensions/jackson/pom.xml            |    2 +-
 sdks/java/extensions/join-library/pom.xml       |    2 +-
 sdks/java/extensions/pom.xml                    |    4 +-
 sdks/java/extensions/protobuf/pom.xml           |    2 +-
 sdks/java/extensions/sketching/pom.xml          |  104 +
 .../sketching/ApproximateDistinct.java          |  573 +++
 .../sdk/extensions/sketching/package-info.java  |   22 +
 .../sketching/ApproximateDistinctTest.java      |  209 ++
 sdks/java/extensions/sorter/pom.xml             |    2 +-
 sdks/java/extensions/sql/NOTICE                 |   45 +
 sdks/java/extensions/sql/pom.xml                |  275 ++
 .../sdk/extensions/sql/BeamRecordSqlType.java   |  186 +
 .../apache/beam/sdk/extensions/sql/BeamSql.java |  250 ++
 .../sdk/extensions/sql/BeamSqlRecordHelper.java |  217 ++
 .../beam/sdk/extensions/sql/BeamSqlUdf.java     |   43 +
 .../extensions/sql/example/BeamSqlExample.java  |  104 +
 .../extensions/sql/example/package-info.java    |   23 +
 .../sdk/extensions/sql/impl/BeamSqlCli.java     |   65 +
 .../sdk/extensions/sql/impl/BeamSqlEnv.java     |  135 +
 .../interpreter/BeamSqlExpressionExecutor.java  |   44 +
 .../sql/impl/interpreter/BeamSqlFnExecutor.java |  458 +++
 .../operator/BeamSqlCaseExpression.java         |   64 +
 .../operator/BeamSqlCastExpression.java         |  132 +
 .../interpreter/operator/BeamSqlExpression.java |   79 +
 .../operator/BeamSqlInputRefExpression.java     |   48 +
 .../interpreter/operator/BeamSqlPrimitive.java  |  157 +
 .../operator/BeamSqlReinterpretExpression.java  |   55 +
 .../operator/BeamSqlUdfExpression.java          |   92 +
 .../operator/BeamSqlWindowEndExpression.java    |   48 +
 .../operator/BeamSqlWindowExpression.java       |   51 +
 .../operator/BeamSqlWindowStartExpression.java  |   49 +
 .../sql/impl/interpreter/operator/UdafImpl.java |   87 +
 .../arithmetic/BeamSqlArithmeticExpression.java |  124 +
 .../arithmetic/BeamSqlDivideExpression.java     |   37 +
 .../arithmetic/BeamSqlMinusExpression.java      |   36 +
 .../arithmetic/BeamSqlModExpression.java        |   36 +
 .../arithmetic/BeamSqlMultiplyExpression.java   |   36 +
 .../arithmetic/BeamSqlPlusExpression.java       |   36 +
 .../operator/arithmetic/package-info.java       |   22 +
 .../comparison/BeamSqlCompareExpression.java    |   97 +
 .../comparison/BeamSqlEqualsExpression.java     |   49 +
 .../BeamSqlGreaterThanExpression.java           |   49 +
 .../BeamSqlGreaterThanOrEqualsExpression.java   |   49 +
 .../comparison/BeamSqlIsNotNullExpression.java  |   54 +
 .../comparison/BeamSqlIsNullExpression.java     |   54 +
 .../comparison/BeamSqlLessThanExpression.java   |   49 +
 .../BeamSqlLessThanOrEqualsExpression.java      |   49 +
 .../comparison/BeamSqlNotEqualsExpression.java  |   49 +
 .../operator/comparison/package-info.java       |   22 +
 .../date/BeamSqlCurrentDateExpression.java      |   45 +
 .../date/BeamSqlCurrentTimeExpression.java      |   53 +
 .../date/BeamSqlCurrentTimestampExpression.java |   49 +
 .../date/BeamSqlDateCeilExpression.java         |   55 +
 .../date/BeamSqlDateFloorExpression.java        |   55 +
 .../date/BeamSqlDatetimePlusExpression.java     |  129 +
 .../operator/date/BeamSqlExtractExpression.java |  102 +
 .../date/BeamSqlIntervalMultiplyExpression.java |  103 +
 .../operator/date/TimeUnitUtils.java            |   54 +
 .../interpreter/operator/date/package-info.java |   22 +
 .../operator/logical/BeamSqlAndExpression.java  |   48 +
 .../logical/BeamSqlLogicalExpression.java       |   46 +
 .../operator/logical/BeamSqlNotExpression.java  |   54 +
 .../operator/logical/BeamSqlOrExpression.java   |   48 +
 .../operator/logical/package-info.java          |   22 +
 .../operator/math/BeamSqlAbsExpression.java     |   74 +
 .../operator/math/BeamSqlAcosExpression.java    |   40 +
 .../operator/math/BeamSqlAsinExpression.java    |   40 +
 .../operator/math/BeamSqlAtan2Expression.java   |   42 +
 .../operator/math/BeamSqlAtanExpression.java    |   40 +
 .../operator/math/BeamSqlCeilExpression.java    |   45 +
 .../operator/math/BeamSqlCosExpression.java     |   40 +
 .../operator/math/BeamSqlCotExpression.java     |   40 +
 .../operator/math/BeamSqlDegreesExpression.java |   40 +
 .../operator/math/BeamSqlExpExpression.java     |   40 +
 .../operator/math/BeamSqlFloorExpression.java   |   45 +
 .../operator/math/BeamSqlLnExpression.java      |   40 +
 .../operator/math/BeamSqlLogExpression.java     |   40 +
 .../math/BeamSqlMathBinaryExpression.java       |   65 +
 .../math/BeamSqlMathUnaryExpression.java        |   60 +
 .../operator/math/BeamSqlPiExpression.java      |   43 +
 .../operator/math/BeamSqlPowerExpression.java   |   44 +
 .../operator/math/BeamSqlRadiansExpression.java |   40 +
 .../operator/math/BeamSqlRandExpression.java    |   55 +
 .../math/BeamSqlRandIntegerExpression.java      |   59 +
 .../operator/math/BeamSqlRoundExpression.java   |  107 +
 .../operator/math/BeamSqlSignExpression.java    |   72 +
 .../operator/math/BeamSqlSinExpression.java     |   40 +
 .../operator/math/BeamSqlTanExpression.java     |   40 +
 .../math/BeamSqlTruncateExpression.java         |   75 +
 .../interpreter/operator/math/package-info.java |   22 +
 .../impl/interpreter/operator/package-info.java |   22 +
 .../string/BeamSqlCharLengthExpression.java     |   40 +
 .../string/BeamSqlConcatExpression.java         |   63 +
 .../string/BeamSqlInitCapExpression.java        |   56 +
 .../operator/string/BeamSqlLowerExpression.java |   40 +
 .../string/BeamSqlOverlayExpression.java        |   77 +
 .../string/BeamSqlPositionExpression.java       |   73 +
 .../string/BeamSqlStringUnaryExpression.java    |   44 +
 .../string/BeamSqlSubstringExpression.java      |   83 +
 .../operator/string/BeamSqlTrimExpression.java  |  102 +
 .../operator/string/BeamSqlUpperExpression.java |   40 +
 .../operator/string/package-info.java           |   22 +
 .../sql/impl/interpreter/package-info.java      |   22 +
 .../sdk/extensions/sql/impl/package-info.java   |   22 +
 .../sql/impl/planner/BeamQueryPlanner.java      |  168 +
 .../sql/impl/planner/BeamRelDataTypeSystem.java |   40 +
 .../sql/impl/planner/BeamRuleSets.java          |   75 +
 .../sql/impl/planner/package-info.java          |   24 +
 .../sql/impl/rel/BeamAggregationRel.java        |  182 +
 .../extensions/sql/impl/rel/BeamFilterRel.java  |   69 +
 .../extensions/sql/impl/rel/BeamIOSinkRel.java  |   75 +
 .../sql/impl/rel/BeamIOSourceRel.java           |   62 +
 .../sql/impl/rel/BeamIntersectRel.java          |   58 +
 .../extensions/sql/impl/rel/BeamJoinRel.java    |  298 ++
 .../sql/impl/rel/BeamLogicalConvention.java     |   72 +
 .../extensions/sql/impl/rel/BeamMinusRel.java   |   56 +
 .../extensions/sql/impl/rel/BeamProjectRel.java |   80 +
 .../extensions/sql/impl/rel/BeamRelNode.java    |   39 +
 .../sql/impl/rel/BeamSetOperatorRelBase.java    |   98 +
 .../extensions/sql/impl/rel/BeamSortRel.java    |  235 ++
 .../sql/impl/rel/BeamSqlRelUtils.java           |   72 +
 .../extensions/sql/impl/rel/BeamUnionRel.java   |   88 +
 .../extensions/sql/impl/rel/BeamValuesRel.java  |   79 +
 .../extensions/sql/impl/rel/package-info.java   |   23 +
 .../sql/impl/rule/BeamAggregationRule.java      |  162 +
 .../sql/impl/rule/BeamFilterRule.java           |   49 +
 .../sql/impl/rule/BeamIOSinkRule.java           |   81 +
 .../sql/impl/rule/BeamIOSourceRule.java         |   49 +
 .../sql/impl/rule/BeamIntersectRule.java        |   50 +
 .../extensions/sql/impl/rule/BeamJoinRule.java  |   53 +
 .../extensions/sql/impl/rule/BeamMinusRule.java |   50 +
 .../sql/impl/rule/BeamProjectRule.java          |   50 +
 .../extensions/sql/impl/rule/BeamSortRule.java  |   51 +
 .../extensions/sql/impl/rule/BeamUnionRule.java |   50 +
 .../sql/impl/rule/BeamValuesRule.java           |   48 +
 .../extensions/sql/impl/rule/package-info.java  |   23 +
 .../sql/impl/schema/BaseBeamTable.java          |   35 +
 .../extensions/sql/impl/schema/BeamIOType.java  |   28 +
 .../sql/impl/schema/BeamPCollectionTable.java   |   63 +
 .../sql/impl/schema/BeamSqlTable.java           |   54 +
 .../sql/impl/schema/BeamTableUtils.java         |  118 +
 .../impl/schema/kafka/BeamKafkaCSVTable.java    |  109 +
 .../sql/impl/schema/kafka/BeamKafkaTable.java   |  109 +
 .../sql/impl/schema/kafka/package-info.java     |   22 +
 .../sql/impl/schema/package-info.java           |   22 +
 .../sql/impl/schema/text/BeamTextCSVTable.java  |   70 +
 .../schema/text/BeamTextCSVTableIOReader.java   |   58 +
 .../schema/text/BeamTextCSVTableIOWriter.java   |   58 +
 .../sql/impl/schema/text/BeamTextTable.java     |   41 +
 .../sql/impl/schema/text/package-info.java      |   22 +
 .../transform/BeamAggregationTransforms.java    |  311 ++
 .../impl/transform/BeamBuiltinAggregations.java |  557 +++
 .../sql/impl/transform/BeamJoinTransforms.java  |  161 +
 .../transform/BeamSetOperatorsTransforms.java   |  111 +
 .../sql/impl/transform/BeamSqlFilterFn.java     |   63 +
 .../transform/BeamSqlOutputToConsoleFn.java     |   41 +
 .../sql/impl/transform/BeamSqlProjectFn.java    |   72 +
 .../sql/impl/transform/package-info.java        |   22 +
 .../extensions/sql/impl/utils/CalciteUtils.java |  113 +
 .../extensions/sql/impl/utils/SqlTypeUtils.java |   63 +
 .../extensions/sql/impl/utils/package-info.java |   22 +
 .../beam/sdk/extensions/sql/package-info.java   |   22 +
 .../sql/src/main/resources/log4j.properties     |   23 +
 ...ged.org.codehaus.commons.compiler.properties |   18 +
 .../extensions/sql/BeamSqlApiSurfaceTest.java   |   57 +
 .../sql/BeamSqlDslAggregationTest.java          |  400 ++
 .../beam/sdk/extensions/sql/BeamSqlDslBase.java |  136 +
 .../extensions/sql/BeamSqlDslFilterTest.java    |  155 +
 .../sdk/extensions/sql/BeamSqlDslJoinTest.java  |  188 +
 .../extensions/sql/BeamSqlDslProjectTest.java   |  227 ++
 .../extensions/sql/BeamSqlDslUdfUdafTest.java   |  139 +
 .../beam/sdk/extensions/sql/TestUtils.java      |  190 +
 .../impl/interpreter/BeamSqlFnExecutorTest.java |  446 +++
 .../interpreter/BeamSqlFnExecutorTestBase.java  |   88 +
 .../operator/BeamNullExperssionTest.java        |   55 +
 .../operator/BeamSqlAndOrExpressionTest.java    |   61 +
 .../operator/BeamSqlCaseExpressionTest.java     |   93 +
 .../operator/BeamSqlCastExpressionTest.java     |  129 +
 .../operator/BeamSqlCompareExpressionTest.java  |  115 +
 .../operator/BeamSqlInputRefExpressionTest.java |   57 +
 .../operator/BeamSqlPrimitiveTest.java          |   59 +
 .../BeamSqlReinterpretExpressionTest.java       |   75 +
 .../operator/BeamSqlUdfExpressionTest.java      |   51 +
 .../BeamSqlArithmeticExpressionTest.java        |  237 ++
 .../date/BeamSqlCurrentDateExpressionTest.java  |   38 +
 .../date/BeamSqlCurrentTimeExpressionTest.java  |   39 +
 .../BeamSqlCurrentTimestampExpressionTest.java  |   39 +
 .../date/BeamSqlDateCeilExpressionTest.java     |   50 +
 .../date/BeamSqlDateExpressionTestBase.java     |   52 +
 .../date/BeamSqlDateFloorExpressionTest.java    |   49 +
 .../date/BeamSqlDatetimePlusExpressionTest.java |  155 +
 .../date/BeamSqlExtractExpressionTest.java      |  103 +
 .../BeamSqlIntervalMultiplyExpressionTest.java  |  107 +
 .../operator/date/TimeUnitUtilsTest.java        |   54 +
 .../logical/BeamSqlNotExpressionTest.java       |   47 +
 .../math/BeamSqlMathBinaryExpressionTest.java   |  215 ++
 .../math/BeamSqlMathUnaryExpressionTest.java    |  312 ++
 .../string/BeamSqlCharLengthExpressionTest.java |   44 +
 .../string/BeamSqlConcatExpressionTest.java     |   66 +
 .../string/BeamSqlInitCapExpressionTest.java    |   54 +
 .../string/BeamSqlLowerExpressionTest.java      |   44 +
 .../string/BeamSqlOverlayExpressionTest.java    |   87 +
 .../string/BeamSqlPositionExpressionTest.java   |   84 +
 .../BeamSqlStringUnaryExpressionTest.java       |   52 +
 .../string/BeamSqlSubstringExpressionTest.java  |  101 +
 .../string/BeamSqlTrimExpressionTest.java       |  103 +
 .../string/BeamSqlUpperExpressionTest.java      |   44 +
 .../extensions/sql/impl/rel/BaseRelTest.java    |   34 +
 .../sql/impl/rel/BeamIntersectRelTest.java      |  118 +
 .../rel/BeamJoinRelBoundedVsBoundedTest.java    |  203 +
 .../rel/BeamJoinRelUnboundedVsBoundedTest.java  |  240 ++
 .../BeamJoinRelUnboundedVsUnboundedTest.java    |  218 ++
 .../sql/impl/rel/BeamMinusRelTest.java          |  117 +
 .../impl/rel/BeamSetOperatorRelBaseTest.java    |  105 +
 .../sql/impl/rel/BeamSortRelTest.java           |  257 ++
 .../sql/impl/rel/BeamUnionRelTest.java          |  103 +
 .../sql/impl/rel/BeamValuesRelTest.java         |  104 +
 .../sdk/extensions/sql/impl/rel/CheckSize.java  |   41 +
 .../sql/impl/schema/BeamSqlRowCoderTest.java    |   77 +
 .../schema/kafka/BeamKafkaCSVTableTest.java     |  107 +
 .../impl/schema/text/BeamTextCSVTableTest.java  |  176 +
 .../transform/BeamAggregationTransformTest.java |  453 +++
 .../schema/transform/BeamTransformBaseTest.java |   97 +
 .../sql/impl/utils/SqlTypeUtilsTest.java        |   76 +
 ...amSqlArithmeticOperatorsIntegrationTest.java |  165 +
 ...mSqlBuiltinFunctionsIntegrationTestBase.java |  168 +
 ...amSqlComparisonOperatorsIntegrationTest.java |  329 ++
 ...mSqlConditionalFunctionsIntegrationTest.java |   60 +
 .../BeamSqlDateFunctionsIntegrationTest.java    |  125 +
 .../BeamSqlLogicalFunctionsIntegrationTest.java |   43 +
 .../BeamSqlMathFunctionsIntegrationTest.java    |  351 ++
 .../BeamSqlStringFunctionsIntegrationTest.java  |   51 +
 .../extensions/sql/mock/MockedBoundedTable.java |  134 +
 .../sdk/extensions/sql/mock/MockedTable.java    |   42 +
 .../sql/mock/MockedUnboundedTable.java          |  110 +
 sdks/java/fn-execution/pom.xml                  |   82 +
 .../harness/channel/ManagedChannelFactory.java  |   82 +
 .../harness/channel/SocketAddressFactory.java   |   64 +
 .../beam/harness/channel/package-info.java      |   22 +
 .../channel/ManagedChannelFactoryTest.java      |   71 +
 .../channel/SocketAddressFactoryTest.java       |   56 +
 .../org/apache/beam/harness/test/Consumer.java  |   26 +
 .../org/apache/beam/harness/test/Supplier.java  |   26 +
 .../apache/beam/harness/test/TestExecutors.java |   93 +
 .../beam/harness/test/TestExecutorsTest.java    |  175 +
 .../apache/beam/harness/test/TestStreams.java   |  185 +
 .../beam/harness/test/TestStreamsTest.java      |  109 +
 sdks/java/harness/pom.xml                       |  128 +-
 .../beam/fn/harness/BeamFnDataReadRunner.java   |   15 +-
 .../beam/fn/harness/BeamFnDataWriteRunner.java  |   15 +-
 .../beam/fn/harness/BoundedSourceRunner.java    |   14 +-
 .../apache/beam/fn/harness/FnApiDoFnRunner.java |  398 +-
 .../org/apache/beam/fn/harness/FnHarness.java   |   64 +-
 .../org/apache/beam/fn/harness/IdGenerator.java |   33 +
 .../fn/harness/PTransformRunnerFactory.java     |    7 +-
 .../harness/channel/ManagedChannelFactory.java  |   86 -
 .../harness/channel/SocketAddressFactory.java   |   64 -
 .../beam/fn/harness/channel/package-info.java   |   22 -
 .../fn/harness/control/BeamFnControlClient.java |   15 +-
 .../harness/control/ProcessBundleHandler.java   |  154 +-
 .../fn/harness/control/RegisterHandler.java     |    8 +-
 .../BeamFnDataBufferingOutboundObserver.java    |    8 +-
 .../beam/fn/harness/data/BeamFnDataClient.java  |    7 +-
 .../fn/harness/data/BeamFnDataGrpcClient.java   |   19 +-
 .../harness/data/BeamFnDataGrpcMultiplexer.java |   24 +-
 .../harness/data/BeamFnDataInboundObserver.java |    4 +-
 .../beam/fn/harness/fake/FakeStepContext.java   |   39 -
 .../beam/fn/harness/fake/package-info.java      |   22 -
 .../beam/fn/harness/fn/ThrowingBiConsumer.java  |   33 +
 .../fn/harness/logging/BeamFnLoggingClient.java |  202 +-
 .../beam/fn/harness/state/BagUserState.java     |  121 +
 .../fn/harness/state/BeamFnStateClient.java     |   39 +
 .../state/BeamFnStateGrpcClientCache.java       |  173 +
 .../state/LazyCachingIteratorToIterable.java    |   72 +
 .../harness/state/StateFetchingIterators.java   |  126 +
 .../beam/fn/harness/state/package-info.java     |   22 +
 .../harness/stream/BufferingStreamObserver.java |   16 +-
 .../beam/fn/harness/stream/DataStreams.java     |   73 +-
 .../harness/stream/StreamObserverFactory.java   |    4 +-
 .../fn/harness/BeamFnDataReadRunnerTest.java    |   19 +-
 .../fn/harness/BeamFnDataWriteRunnerTest.java   |   15 +-
 .../fn/harness/BoundedSourceRunnerTest.java     |   21 +-
 .../beam/fn/harness/FnApiDoFnRunnerTest.java    |  247 +-
 .../apache/beam/fn/harness/FnHarnessTest.java   |   25 +-
 .../apache/beam/fn/harness/IdGeneratorTest.java |   40 +
 .../channel/ManagedChannelFactoryTest.java      |   74 -
 .../channel/SocketAddressFactoryTest.java       |   56 -
 .../control/BeamFnControlClientTest.java        |   13 +-
 .../control/ProcessBundleHandlerTest.java       |  160 +-
 .../fn/harness/control/RegisterHandlerTest.java |   10 +-
 ...BeamFnDataBufferingOutboundObserverTest.java |    4 +-
 .../harness/data/BeamFnDataGrpcClientTest.java  |   30 +-
 .../data/BeamFnDataGrpcMultiplexerTest.java     |    9 +-
 .../data/BeamFnDataInboundObserverTest.java     |    2 +-
 .../logging/BeamFnLoggingClientTest.java        |  136 +-
 .../beam/fn/harness/state/BagUserStateTest.java |  106 +
 .../state/BeamFnStateGrpcClientCacheTest.java   |  234 ++
 .../fn/harness/state/FakeBeamFnStateClient.java |  110 +
 .../LazyCachingIteratorToIterableTest.java      |   76 +
 .../state/StateFetchingIteratorsTest.java       |   99 +
 .../stream/BufferingStreamObserverTest.java     |   12 +-
 .../beam/fn/harness/stream/DataStreamsTest.java |  165 +-
 .../stream/DirectStreamObserverTest.java        |    8 +-
 .../beam/fn/harness/test/TestExecutors.java     |   85 -
 .../beam/fn/harness/test/TestExecutorsTest.java |  160 -
 .../beam/fn/harness/test/TestStreams.java       |  162 -
 .../beam/fn/harness/test/TestStreamsTest.java   |   84 -
 sdks/java/io/amqp/pom.xml                       |   24 +-
 .../org/apache/beam/sdk/io/amqp/AmqpIO.java     |   33 +-
 .../org/apache/beam/sdk/io/amqp/AmqpIOTest.java |  112 +-
 sdks/java/io/cassandra/pom.xml                  |    2 +-
 .../beam/sdk/io/cassandra/CassandraIO.java      |   64 +-
 sdks/java/io/common/pom.xml                     |    2 +-
 .../sdk/io/common/IOTestPipelineOptions.java    |    6 +
 .../elasticsearch-tests-2/pom.xml               |   60 +
 .../src/test/contrib/create_elk_container.sh    |   24 +
 .../sdk/io/elasticsearch/ElasticsearchIOIT.java |  123 +
 .../io/elasticsearch/ElasticsearchIOTest.java   |  185 +
 .../elasticsearch-tests-5/pom.xml               |  124 +
 .../src/test/contrib/create_elk_container.sh    |   24 +
 .../sdk/io/elasticsearch/ElasticsearchIOIT.java |  122 +
 .../io/elasticsearch/ElasticsearchIOTest.java   |  185 +
 .../org/elasticsearch/bootstrap/JarHell.java    |   39 +
 .../elasticsearch-tests-common/pom.xml          |   77 +
 .../elasticsearch/ElasticSearchIOTestUtils.java |  141 +
 .../elasticsearch/ElasticsearchIOITCommon.java  |   92 +
 .../ElasticsearchIOTestCommon.java              |  306 ++
 sdks/java/io/elasticsearch-tests/pom.xml        |  144 +
 sdks/java/io/elasticsearch/pom.xml              |  228 +-
 .../sdk/io/elasticsearch/ElasticsearchIO.java   |  448 ++-
 .../beam/sdk/io/elasticsearch/package-info.java |    1 -
 .../src/test/contrib/create_elk_container.sh    |   24 -
 .../elasticsearch/ElasticSearchIOTestUtils.java |  138 -
 .../sdk/io/elasticsearch/ElasticsearchIOIT.java |  155 -
 .../io/elasticsearch/ElasticsearchIOTest.java   |  355 --
 .../elasticsearch/ElasticsearchTestDataSet.java |   97 -
 sdks/java/io/google-cloud-platform/pom.xml      |    9 +-
 .../beam/sdk/io/gcp/bigquery/BatchLoads.java    |  473 ++-
 .../sdk/io/gcp/bigquery/BigQueryHelpers.java    |   28 +-
 .../beam/sdk/io/gcp/bigquery/BigQueryIO.java    |  755 +++-
 .../io/gcp/bigquery/BigQueryQuerySource.java    |   49 +-
 .../sdk/io/gcp/bigquery/BigQueryServices.java   |   53 +-
 .../io/gcp/bigquery/BigQueryServicesImpl.java   |   82 +-
 .../sdk/io/gcp/bigquery/BigQuerySourceBase.java |  131 +-
 .../gcp/bigquery/BigQueryTableRowIterator.java  |  501 ---
 .../io/gcp/bigquery/BigQueryTableSource.java    |   40 +-
 .../sdk/io/gcp/bigquery/CalculateSchemas.java   |   78 -
 .../beam/sdk/io/gcp/bigquery/CreateTables.java  |   23 +-
 .../io/gcp/bigquery/DynamicDestinations.java    |   13 +-
 .../bigquery/DynamicDestinationsHelpers.java    |   27 +-
 .../sdk/io/gcp/bigquery/ReifyAsIterable.java    |   51 +
 .../sdk/io/gcp/bigquery/SchemaAndRecord.java    |   43 +
 .../io/gcp/bigquery/StreamingWriteTables.java   |   10 +-
 .../sdk/io/gcp/bigquery/TableDestination.java   |   43 +-
 .../io/gcp/bigquery/TableDestinationCoder.java  |    2 +
 .../gcp/bigquery/TableDestinationCoderV2.java   |   59 +
 .../io/gcp/bigquery/WriteBundlesToFiles.java    |   43 +-
 .../bigquery/WriteGroupedRecordsToFiles.java    |    7 +-
 .../sdk/io/gcp/bigquery/WritePartition.java     |   13 +-
 .../beam/sdk/io/gcp/bigquery/WriteRename.java   |  112 +-
 .../beam/sdk/io/gcp/bigquery/WriteTables.java   |  189 +-
 .../beam/sdk/io/gcp/bigtable/BigtableIO.java    |  135 +-
 .../io/gcp/bigtable/BigtableServiceImpl.java    |   16 +-
 .../beam/sdk/io/gcp/datastore/DatastoreV1.java  |  111 +-
 .../apache/beam/sdk/io/gcp/pubsub/PubsubIO.java |   35 +-
 .../sdk/io/gcp/pubsub/PubsubUnboundedSink.java  |    6 +-
 .../io/gcp/pubsub/PubsubUnboundedSource.java    |   41 +-
 .../sdk/io/gcp/spanner/AbstractSpannerFn.java   |   71 -
 .../sdk/io/gcp/spanner/CreateTransactionFn.java |   22 +-
 .../io/gcp/spanner/MutationGroupEncoder.java    |  660 ++++
 .../io/gcp/spanner/MutationSizeEstimator.java   |   48 +
 .../sdk/io/gcp/spanner/NaiveSpannerReadFn.java  |   19 +-
 .../beam/sdk/io/gcp/spanner/OrderedCode.java    |  764 ++++
 .../sdk/io/gcp/spanner/ReadSpannerSchema.java   |   94 +
 .../sdk/io/gcp/spanner/SerializedMutation.java  |   35 +
 .../io/gcp/spanner/SerializedMutationCoder.java |   60 +
 .../sdk/io/gcp/spanner/SpannerAccessor.java     |   43 +
 .../beam/sdk/io/gcp/spanner/SpannerConfig.java  |   41 +-
 .../beam/sdk/io/gcp/spanner/SpannerIO.java      |  384 +-
 .../beam/sdk/io/gcp/spanner/SpannerSchema.java  |  144 +
 .../sdk/io/gcp/spanner/SpannerWriteGroupFn.java |  125 -
 .../beam/sdk/io/gcp/GcpApiSurfaceTest.java      |    2 +
 .../sdk/io/gcp/bigquery/BigQueryIOTest.java     |  811 ++--
 .../bigquery/BigQueryTableRowIteratorTest.java  |  358 --
 .../sdk/io/gcp/bigquery/BigQueryUtilTest.java   |  187 -
 .../io/gcp/bigquery/FakeBigQueryServices.java   |   78 -
 .../sdk/io/gcp/bigquery/FakeDatasetService.java |   21 +-
 .../sdk/io/gcp/bigquery/FakeJobService.java     |   46 +-
 .../sdk/io/gcp/bigquery/TableContainer.java     |    2 +
 .../sdk/io/gcp/bigtable/BigtableIOTest.java     |   30 +-
 .../sdk/io/gcp/datastore/DatastoreV1Test.java   |  167 +-
 .../sdk/io/gcp/datastore/SplitQueryFnIT.java    |    5 +-
 .../beam/sdk/io/gcp/pubsub/PubsubIOTest.java    |   55 +
 .../gcp/spanner/MutationGroupEncoderTest.java   |  636 ++++
 .../sdk/io/gcp/spanner/OrderedCodeTest.java     |  890 +++++
 .../io/gcp/spanner/ReadSpannerSchemaTest.java   |  134 +
 .../sdk/io/gcp/spanner/SpannerIOReadTest.java   |   69 -
 .../sdk/io/gcp/spanner/SpannerIOWriteTest.java  |  453 ++-
 .../sdk/io/gcp/spanner/SpannerSchemaTest.java   |   61 +
 .../beam/sdk/io/gcp/spanner/SpannerWriteIT.java |    5 +-
 sdks/java/io/hadoop-common/pom.xml              |    2 +-
 sdks/java/io/hadoop-file-system/pom.xml         |    2 +-
 .../beam/sdk/io/hdfs/HadoopFileSystem.java      |   20 +-
 .../beam/sdk/io/hdfs/HadoopFileSystemTest.java  |   66 +-
 sdks/java/io/hadoop/input-format/pom.xml        |    7 +-
 .../hadoop/inputformat/HadoopInputFormatIO.java |   48 +-
 .../inputformat/HadoopInputFormatIOTest.java    |   14 +-
 sdks/java/io/hadoop/jdk1.8-tests/pom.xml        |    2 +-
 sdks/java/io/hadoop/pom.xml                     |    2 +-
 sdks/java/io/hbase/pom.xml                      |    2 +-
 .../io/hbase/HBaseCoderProviderRegistrar.java   |    8 +-
 .../org/apache/beam/sdk/io/hbase/HBaseIO.java   | 1001 ++---
 .../beam/sdk/io/hbase/HBaseMutationCoder.java   |   27 +-
 .../beam/sdk/io/hbase/HBaseResultCoder.java     |    6 +-
 .../beam/sdk/io/hbase/SerializableScan.java     |   37 +-
 .../hbase/HBaseCoderProviderRegistrarTest.java  |    4 +-
 .../apache/beam/sdk/io/hbase/HBaseIOTest.java   |  754 ++--
 .../sdk/io/hbase/HBaseMutationCoderTest.java    |    4 +-
 .../beam/sdk/io/hbase/HBaseResultCoderTest.java |    4 +-
 .../beam/sdk/io/hbase/SerializableScanTest.java |    6 +-
 sdks/java/io/hcatalog/pom.xml                   |    2 +-
 .../apache/beam/sdk/io/hcatalog/HCatalogIO.java |   23 +-
 .../beam/sdk/io/hcatalog/HCatalogIOTest.java    |   24 +-
 sdks/java/io/jdbc/pom.xml                       |    2 +-
 .../org/apache/beam/sdk/io/jdbc/JdbcIO.java     |  313 +-
 .../org/apache/beam/sdk/io/jdbc/JdbcIOTest.java |    6 -
 sdks/java/io/jms/pom.xml                        |    2 +-
 .../java/org/apache/beam/sdk/io/jms/JmsIO.java  |   71 +-
 sdks/java/io/kafka/README.md                    |   36 +
 sdks/java/io/kafka/pom.xml                      |   19 +-
 .../org/apache/beam/sdk/io/kafka/KafkaIO.java   |  790 +++-
 .../apache/beam/sdk/io/kafka/ProducerSpEL.java  |  135 +
 .../apache/beam/sdk/io/kafka/KafkaIOTest.java   |  233 +-
 sdks/java/io/kinesis/pom.xml                    |    8 +-
 .../beam/sdk/io/kinesis/AWSClientsProvider.java |   36 +
 .../sdk/io/kinesis/GetKinesisRecordsResult.java |    8 +-
 .../sdk/io/kinesis/KinesisClientProvider.java   |   33 -
 .../apache/beam/sdk/io/kinesis/KinesisIO.java   |  125 +-
 .../beam/sdk/io/kinesis/KinesisReader.java      |  146 +-
 .../sdk/io/kinesis/KinesisReaderCheckpoint.java |   18 -
 .../beam/sdk/io/kinesis/KinesisSource.java      |   40 +-
 .../apache/beam/sdk/io/kinesis/RoundRobin.java  |   54 -
 .../beam/sdk/io/kinesis/ShardCheckpoint.java    |    8 +-
 .../beam/sdk/io/kinesis/ShardReadersPool.java   |  162 +
 .../sdk/io/kinesis/ShardRecordsIterator.java    |   88 +-
 .../sdk/io/kinesis/SimplifiedKinesisClient.java |   98 +-
 .../io/kinesis/TransientKinesisException.java   |    4 +-
 .../beam/sdk/io/kinesis/AmazonKinesisMock.java  |   19 +-
 .../sdk/io/kinesis/KinesisMockReadTest.java     |    5 +-
 .../beam/sdk/io/kinesis/KinesisReaderIT.java    |    5 +-
 .../beam/sdk/io/kinesis/KinesisReaderTest.java  |  152 +-
 .../beam/sdk/io/kinesis/RoundRobinTest.java     |   59 -
 .../sdk/io/kinesis/ShardReadersPoolTest.java    |  185 +
 .../io/kinesis/ShardRecordsIteratorTest.java    |   35 +-
 .../io/kinesis/SimplifiedKinesisClientTest.java |  107 +
 sdks/java/io/mongodb/pom.xml                    |    2 +-
 .../beam/sdk/io/mongodb/MongoDbGridFSIO.java    |    4 -
 .../apache/beam/sdk/io/mongodb/MongoDbIO.java   |   62 +-
 .../sdk/io/mongodb/MongoDBGridFSIOTest.java     |    2 +-
 .../beam/sdk/io/mongodb/MongoDbIOTest.java      |   11 +
 sdks/java/io/mqtt/pom.xml                       |    2 +-
 .../org/apache/beam/sdk/io/mqtt/MqttIO.java     |   49 +-
 sdks/java/io/pom.xml                            |   12 +-
 sdks/java/io/redis/pom.xml                      |   90 +
 .../io/redis/RedisConnectionConfiguration.java  |  122 +
 .../org/apache/beam/sdk/io/redis/RedisIO.java   |  451 +++
 .../apache/beam/sdk/io/redis/package-info.java  |   22 +
 .../apache/beam/sdk/io/redis/RedisIOTest.java   |  109 +
 sdks/java/io/solr/pom.xml                       |  145 +
 .../beam/sdk/io/solr/AuthorizedSolrClient.java  |   91 +
 .../beam/sdk/io/solr/JavaBinCodecCoder.java     |   98 +
 .../org/apache/beam/sdk/io/solr/SolrIO.java     |  705 ++++
 .../apache/beam/sdk/io/solr/package-info.java   |   20 +
 .../beam/sdk/io/solr/JavaBinCodecCoderTest.java |   81 +
 .../org/apache/beam/sdk/io/solr/SolrIOTest.java |  269 ++
 .../beam/sdk/io/solr/SolrIOTestUtils.java       |  132 +
 .../resources/cloud-minimal/conf/schema.xml     |   29 +
 .../resources/cloud-minimal/conf/solrconfig.xml |   48 +
 sdks/java/io/tika/pom.xml                       |  113 +
 .../apache/beam/sdk/io/tika/ParseResult.java    |  144 +
 .../org/apache/beam/sdk/io/tika/TikaIO.java     |  284 ++
 .../apache/beam/sdk/io/tika/package-info.java   |   22 +
 .../beam/sdk/io/tika/ParseResultTest.java       |   83 +
 .../org/apache/beam/sdk/io/tika/TikaIOTest.java |  149 +
 .../java/io/tika/src/test/resources/damaged.pdf |    2 +
 .../resources/valid/apache-beam-tika-pdf.zip    |  Bin 0 -> 11685 bytes
 .../test/resources/valid/apache-beam-tika.odt   |  Bin 0 -> 12540 bytes
 sdks/java/io/xml/pom.xml                        |    2 +-
 .../java/org/apache/beam/sdk/io/xml/XmlIO.java  |  375 +-
 .../org/apache/beam/sdk/io/xml/XmlSink.java     |   10 -
 .../org/apache/beam/sdk/io/xml/XmlSource.java   |   63 +-
 .../org/apache/beam/sdk/io/xml/XmlSinkTest.java |   12 +-
 .../apache/beam/sdk/io/xml/XmlSourceTest.java   |   74 +-
 sdks/java/java8tests/pom.xml                    |    2 +-
 sdks/java/javadoc/ant.xml                       |    1 +
 sdks/java/javadoc/pom.xml                       |   22 +-
 .../maven-archetypes/examples-java8/pom.xml     |    2 +-
 .../main/resources/archetype-resources/pom.xml  |   13 +
 sdks/java/maven-archetypes/examples/pom.xml     |    2 +-
 sdks/java/maven-archetypes/pom.xml              |    2 +-
 sdks/java/maven-archetypes/starter/pom.xml      |    2 +-
 sdks/java/nexmark/README.md                     |  340 ++
 sdks/java/nexmark/pom.xml                       |  265 ++
 .../java/org/apache/beam/sdk/nexmark/Main.java  |  303 ++
 .../org/apache/beam/sdk/nexmark/Monitor.java    |   78 +
 .../beam/sdk/nexmark/NexmarkConfiguration.java  |  721 ++++
 .../beam/sdk/nexmark/NexmarkLauncher.java       | 1157 ++++++
 .../apache/beam/sdk/nexmark/NexmarkOptions.java |  403 ++
 .../apache/beam/sdk/nexmark/NexmarkPerf.java    |  207 ++
 .../apache/beam/sdk/nexmark/NexmarkSuite.java   |  112 +
 .../apache/beam/sdk/nexmark/NexmarkUtils.java   |  674 ++++
 .../apache/beam/sdk/nexmark/model/Auction.java  |  187 +
 .../beam/sdk/nexmark/model/AuctionBid.java      |   85 +
 .../beam/sdk/nexmark/model/AuctionCount.java    |   84 +
 .../beam/sdk/nexmark/model/AuctionPrice.java    |   88 +
 .../org/apache/beam/sdk/nexmark/model/Bid.java  |  177 +
 .../beam/sdk/nexmark/model/BidsPerSession.java  |   87 +
 .../beam/sdk/nexmark/model/CategoryPrice.java   |   97 +
 .../org/apache/beam/sdk/nexmark/model/Done.java |   80 +
 .../apache/beam/sdk/nexmark/model/Event.java    |  171 +
 .../beam/sdk/nexmark/model/IdNameReserve.java   |   98 +
 .../beam/sdk/nexmark/model/KnownSize.java       |   26 +
 .../beam/sdk/nexmark/model/NameCityStateId.java |  103 +
 .../apache/beam/sdk/nexmark/model/Person.java   |  163 +
 .../beam/sdk/nexmark/model/SellerPrice.java     |   89 +
 .../beam/sdk/nexmark/model/package-info.java    |   22 +
 .../apache/beam/sdk/nexmark/package-info.java   |   21 +
 .../sdk/nexmark/queries/AbstractSimulator.java  |  211 ++
 .../beam/sdk/nexmark/queries/NexmarkQuery.java  |  270 ++
 .../sdk/nexmark/queries/NexmarkQueryModel.java  |  117 +
 .../apache/beam/sdk/nexmark/queries/Query0.java |   70 +
 .../beam/sdk/nexmark/queries/Query0Model.java   |   64 +
 .../apache/beam/sdk/nexmark/queries/Query1.java |   67 +
 .../beam/sdk/nexmark/queries/Query10.java       |  367 ++
 .../beam/sdk/nexmark/queries/Query11.java       |   79 +
 .../beam/sdk/nexmark/queries/Query12.java       |   80 +
 .../beam/sdk/nexmark/queries/Query1Model.java   |   76 +
 .../apache/beam/sdk/nexmark/queries/Query2.java |   79 +
 .../beam/sdk/nexmark/queries/Query2Model.java   |   80 +
 .../apache/beam/sdk/nexmark/queries/Query3.java |  301 ++
 .../beam/sdk/nexmark/queries/Query3Model.java   |  124 +
 .../apache/beam/sdk/nexmark/queries/Query4.java |  116 +
 .../beam/sdk/nexmark/queries/Query4Model.java   |  186 +
 .../apache/beam/sdk/nexmark/queries/Query5.java |  138 +
 .../beam/sdk/nexmark/queries/Query5Model.java   |  176 +
 .../apache/beam/sdk/nexmark/queries/Query6.java |  155 +
 .../beam/sdk/nexmark/queries/Query6Model.java   |  133 +
 .../apache/beam/sdk/nexmark/queries/Query7.java |   90 +
 .../beam/sdk/nexmark/queries/Query7Model.java   |  130 +
 .../apache/beam/sdk/nexmark/queries/Query8.java |   98 +
 .../beam/sdk/nexmark/queries/Query8Model.java   |  148 +
 .../apache/beam/sdk/nexmark/queries/Query9.java |   44 +
 .../beam/sdk/nexmark/queries/Query9Model.java   |   44 +
 .../beam/sdk/nexmark/queries/WinningBids.java   |  418 +++
 .../nexmark/queries/WinningBidsSimulator.java   |  206 ++
 .../beam/sdk/nexmark/queries/package-info.java  |   22 +
 .../sdk/nexmark/sources/BoundedEventSource.java |  190 +
 .../beam/sdk/nexmark/sources/Generator.java     |  609 +++
 .../sdk/nexmark/sources/GeneratorConfig.java    |  298 ++
 .../nexmark/sources/UnboundedEventSource.java   |  329 ++
 .../beam/sdk/nexmark/sources/package-info.java  |   22 +
 .../nexmark/src/main/resources/log4j.properties |   55 +
 .../beam/sdk/nexmark/queries/QueryTest.java     |  185 +
 .../nexmark/sources/BoundedEventSourceTest.java |   70 +
 .../beam/sdk/nexmark/sources/GeneratorTest.java |  110 +
 .../sources/UnboundedEventSourceTest.java       |  105 +
 sdks/java/pom.xml                               |    5 +-
 sdks/pom.xml                                    |    6 +-
 sdks/python/apache_beam/__init__.py             |   47 +-
 sdks/python/apache_beam/coders/coder_impl.py    |   22 +-
 sdks/python/apache_beam/coders/coders.py        |  118 +-
 sdks/python/apache_beam/coders/coders_test.py   |    2 +-
 .../apache_beam/coders/coders_test_common.py    |   17 +-
 .../apache_beam/coders/observable_test.py       |    1 -
 .../coders/proto2_coder_test_messages_pb2.py    |    6 +-
 .../apache_beam/coders/standard_coders_test.py  |   11 +-
 sdks/python/apache_beam/coders/stream_test.py   |    1 -
 sdks/python/apache_beam/coders/typecoders.py    |    1 -
 .../examples/complete/autocomplete.py           |    6 +-
 .../examples/complete/autocomplete_test.py      |    2 +-
 .../examples/complete/estimate_pi.py            |    5 +-
 .../examples/complete/estimate_pi_test.py       |    2 +-
 .../examples/complete/game/game_stats.py        |  393 ++
 .../examples/complete/game/game_stats_test.py   |   81 +
 .../examples/complete/game/hourly_team_score.py |  247 +-
 .../examples/complete/game/leader_board.py      |  349 ++
 .../examples/complete/game/leader_board_test.py |   69 +
 .../examples/complete/game/user_score.py        |  186 +-
 .../complete/juliaset/juliaset/juliaset.py      |    8 +-
 .../complete/juliaset/juliaset/juliaset_test.py |    1 -
 .../examples/complete/juliaset/juliaset_main.py |    2 -
 .../examples/complete/juliaset/setup.py         |    7 +-
 .../apache_beam/examples/complete/tfidf.py      |   17 +-
 .../apache_beam/examples/complete/tfidf_test.py |    7 +-
 .../examples/complete/top_wikipedia_sessions.py |    5 +-
 .../complete/top_wikipedia_sessions_test.py     |    1 -
 .../examples/cookbook/bigquery_side_input.py    |    5 +-
 .../examples/cookbook/bigquery_tornadoes.py     |    2 +-
 .../cookbook/bigquery_tornadoes_it_test.py      |    2 +-
 .../examples/cookbook/custom_ptransform.py      |    2 +-
 .../examples/cookbook/datastore_wordcount.py    |   39 +-
 .../examples/cookbook/group_with_coder.py       |    6 +-
 .../examples/cookbook/group_with_coder_test.py  |    1 -
 .../examples/cookbook/mergecontacts.py          |   36 +-
 .../examples/cookbook/multiple_output_pardo.py  |   14 +-
 .../apache_beam/examples/snippets/snippets.py   |   82 +-
 .../examples/snippets/snippets_test.py          |   70 +-
 .../apache_beam/examples/streaming_wordcount.py |    9 +-
 .../apache_beam/examples/windowed_wordcount.py  |    7 +-
 sdks/python/apache_beam/examples/wordcount.py   |   12 +-
 .../apache_beam/examples/wordcount_debugging.py |   12 +-
 .../apache_beam/examples/wordcount_fnapi.py     |  151 +
 .../apache_beam/examples/wordcount_it_test.py   |   16 +
 .../apache_beam/examples/wordcount_minimal.py   |    6 +-
 sdks/python/apache_beam/internal/gcp/auth.py    |    1 -
 .../apache_beam/internal/gcp/json_value.py      |   46 +-
 .../apache_beam/internal/gcp/json_value_test.py |    3 +-
 sdks/python/apache_beam/internal/pickler.py     |    2 +-
 sdks/python/apache_beam/internal/util.py        |    6 +-
 sdks/python/apache_beam/io/avroio.py            |  164 +-
 sdks/python/apache_beam/io/avroio_test.py       |   54 +-
 .../python/apache_beam/io/concat_source_test.py |    1 -
 sdks/python/apache_beam/io/filebasedsink.py     |   22 +-
 .../python/apache_beam/io/filebasedsink_test.py |    3 +-
 sdks/python/apache_beam/io/filebasedsource.py   |  234 +-
 .../apache_beam/io/filebasedsource_test.py      |    8 +-
 sdks/python/apache_beam/io/filesystem.py        |   31 +-
 sdks/python/apache_beam/io/filesystem_test.py   |    5 +-
 sdks/python/apache_beam/io/filesystems.py       |    1 -
 sdks/python/apache_beam/io/filesystems_test.py  |    4 +-
 sdks/python/apache_beam/io/gcp/bigquery.py      |  264 +-
 sdks/python/apache_beam/io/gcp/bigquery_test.py |    4 +-
 .../io/gcp/datastore/v1/adaptive_throttler.py   |   94 +
 .../gcp/datastore/v1/adaptive_throttler_test.py |   95 +
 .../io/gcp/datastore/v1/datastoreio.py          |   55 +-
 .../io/gcp/datastore/v1/datastoreio_test.py     |   11 +-
 .../apache_beam/io/gcp/datastore/v1/helper.py   |   31 +-
 .../io/gcp/datastore/v1/helper_test.py          |    7 +-
 .../apache_beam/io/gcp/gcsfilesystem_test.py    |    1 +
 sdks/python/apache_beam/io/gcp/gcsio.py         |   17 +-
 .../clients/bigquery/bigquery_v2_messages.py    |    3 -
 .../clients/storage/storage_v1_messages.py      |    3 -
 sdks/python/apache_beam/io/gcp/pubsub.py        |    5 +-
 sdks/python/apache_beam/io/gcp/pubsub_test.py   |    5 +-
 .../io/gcp/tests/bigquery_matcher_test.py       |    3 +-
 sdks/python/apache_beam/io/gcp/tests/utils.py   |    8 +-
 .../apache_beam/io/gcp/tests/utils_test.py      |   70 +-
 sdks/python/apache_beam/io/iobase.py            |   83 +-
 .../apache_beam/io/localfilesystem_test.py      |    4 +-
 sdks/python/apache_beam/io/range_trackers.py    |   54 +-
 .../apache_beam/io/range_trackers_test.py       |   37 +
 sdks/python/apache_beam/io/source_test_utils.py |   92 +-
 .../apache_beam/io/source_test_utils_test.py    |    2 +-
 sdks/python/apache_beam/io/sources_test.py      |    1 -
 sdks/python/apache_beam/io/textio.py            |  178 +-
 sdks/python/apache_beam/io/textio_test.py       |  109 +-
 sdks/python/apache_beam/io/tfrecordio.py        |    5 +-
 sdks/python/apache_beam/io/tfrecordio_test.py   |    8 +-
 sdks/python/apache_beam/metrics/cells_test.py   |    2 +-
 sdks/python/apache_beam/metrics/execution.py    |    5 +-
 .../apache_beam/metrics/execution_test.py       |    8 +-
 sdks/python/apache_beam/metrics/metric.py       |    3 +-
 sdks/python/apache_beam/metrics/metric_test.py  |   18 +-
 sdks/python/apache_beam/metrics/metricbase.py   |    4 +
 .../apache_beam/options/pipeline_options.py     |   32 +-
 .../options/pipeline_options_test.py            |    7 +-
 .../options/pipeline_options_validator_test.py  |    3 +-
 .../apache_beam/options/value_provider.py       |    1 -
 sdks/python/apache_beam/pipeline.py             |  136 +-
 sdks/python/apache_beam/pipeline_test.py        |   92 +-
 sdks/python/apache_beam/pvalue.py               |  111 +-
 sdks/python/apache_beam/pvalue_test.py          |    8 +
 sdks/python/apache_beam/runners/common.py       |    6 +-
 sdks/python/apache_beam/runners/common_test.py  |    2 +-
 .../runners/dataflow/dataflow_metrics.py        |    2 +-
 .../runners/dataflow/dataflow_runner.py         |  122 +-
 .../runners/dataflow/dataflow_runner_test.py    |   89 +-
 .../runners/dataflow/internal/apiclient.py      |  106 +-
 .../runners/dataflow/internal/apiclient_test.py |  112 +-
 .../clients/dataflow/dataflow_v1b3_client.py    |  259 +-
 .../clients/dataflow/dataflow_v1b3_messages.py  |   92 +-
 .../clients/dataflow/message_matchers.py        |    1 -
 .../clients/dataflow/message_matchers_test.py   |    3 +-
 .../runners/dataflow/internal/dependency.py     |   58 +-
 .../dataflow/internal/dependency_test.py        |   10 +-
 .../runners/dataflow/internal/names.py          |    2 +
 .../runners/dataflow/native_io/iobase_test.py   |   22 +-
 .../dataflow/native_io/streaming_create.py      |    2 +-
 .../runners/dataflow/template_runner_test.py    |    5 +-
 .../runners/dataflow/test_dataflow_runner.py    |    5 +-
 .../consumer_tracking_pipeline_visitor_test.py  |    2 +-
 .../runners/direct/direct_metrics.py            |    2 +-
 .../runners/direct/direct_metrics_test.py       |    8 +-
 .../apache_beam/runners/direct/direct_runner.py |   19 +-
 .../runners/direct/direct_runner_test.py        |   41 +
 .../runners/direct/evaluation_context.py        |   31 +-
 .../apache_beam/runners/direct/executor.py      |  127 +-
 .../runners/direct/helper_transforms.py         |    2 +-
 .../runners/direct/transform_evaluator.py       |   67 +-
 sdks/python/apache_beam/runners/direct/util.py  |    7 +-
 .../runners/experimental/__init__.py            |   16 +
 .../experimental/python_rpc_direct/__init__.py  |   22 +
 .../python_rpc_direct_runner.py                 |  110 +
 .../experimental/python_rpc_direct/server.py    |  111 +
 sdks/python/apache_beam/runners/job/__init__.py |   16 +
 sdks/python/apache_beam/runners/job/manager.py  |   52 +
 sdks/python/apache_beam/runners/job/utils.py    |   32 +
 .../apache_beam/runners/pipeline_context.py     |    2 +-
 .../runners/portability/fn_api_runner.py        |  393 +-
 .../runners/portability/fn_api_runner_test.py   |  115 +-
 .../portability/maptask_executor_runner.py      |   19 +-
 .../portability/maptask_executor_runner_test.py |   12 +-
 .../portability/universal_local_runner.py       |  409 ++
 .../portability/universal_local_runner_main.py  |   44 +
 .../portability/universal_local_runner_test.py  |   85 +
 sdks/python/apache_beam/runners/runner.py       |   61 +-
 sdks/python/apache_beam/runners/runner_test.py  |    2 +-
 .../runners/worker/bundle_processor.py          |  235 +-
 .../apache_beam/runners/worker/data_plane.py    |   35 +-
 .../runners/worker/data_plane_test.py           |    7 +-
 .../apache_beam/runners/worker/log_handler.py   |   17 +-
 .../runners/worker/log_handler_test.py          |   20 +-
 .../apache_beam/runners/worker/opcounters.py    |   10 +-
 .../runners/worker/opcounters_test.py           |    1 -
 .../apache_beam/runners/worker/operations.pxd   |    4 +-
 .../apache_beam/runners/worker/operations.py    |   84 +-
 .../apache_beam/runners/worker/sdk_worker.py    |  206 +-
 .../runners/worker/sdk_worker_main.py           |   30 +-
 .../runners/worker/sdk_worker_test.py           |   11 +-
 .../apache_beam/runners/worker/statesampler.pyx |   68 +-
 .../runners/worker/statesampler_fake.py         |   17 +-
 .../runners/worker/statesampler_test.py         |    5 +-
 .../apache_beam/testing/pipeline_verifiers.py   |    1 -
 .../testing/pipeline_verifiers_test.py          |    5 +-
 .../python/apache_beam/testing/test_pipeline.py |   54 +-
 .../apache_beam/testing/test_pipeline_test.py   |    4 +-
 sdks/python/apache_beam/testing/test_stream.py  |    1 -
 .../apache_beam/testing/test_stream_test.py     |    3 +-
 sdks/python/apache_beam/testing/test_utils.py   |   23 +-
 .../apache_beam/testing/test_utils_test.py      |   62 +
 sdks/python/apache_beam/testing/util.py         |    5 +-
 sdks/python/apache_beam/testing/util_test.py    |    4 +-
 sdks/python/apache_beam/transforms/combiners.py |    9 +-
 .../apache_beam/transforms/combiners_test.py    |    5 +-
 sdks/python/apache_beam/transforms/core.py      |  397 +-
 .../apache_beam/transforms/create_test.py       |    6 +-
 sdks/python/apache_beam/transforms/display.py   |   91 +-
 .../apache_beam/transforms/display_test.py      |    6 +-
 .../python/apache_beam/transforms/ptransform.py |  147 +-
 .../apache_beam/transforms/ptransform_test.py   |   78 +-
 .../apache_beam/transforms/sideinputs_test.py   |    5 +-
 sdks/python/apache_beam/transforms/timeutil.py  |    1 -
 sdks/python/apache_beam/transforms/trigger.py   |   32 +-
 .../apache_beam/transforms/trigger_test.py      |   19 +-
 sdks/python/apache_beam/transforms/util.py      |  273 +-
 sdks/python/apache_beam/transforms/util_test.py |  108 +
 sdks/python/apache_beam/transforms/window.py    |    9 +-
 .../apache_beam/transforms/window_test.py       |   17 +-
 .../transforms/write_ptransform_test.py         |    4 +-
 sdks/python/apache_beam/typehints/decorators.py |  129 +-
 .../typehints/native_type_compatibility.py      |  166 +
 .../typehints/native_type_compatibility_test.py |   92 +
 sdks/python/apache_beam/typehints/opcodes.py    |   19 +-
 .../apache_beam/typehints/trivial_inference.py  |   80 +-
 .../typehints/trivial_inference_test.py         |    9 +-
 sdks/python/apache_beam/typehints/typecheck.py  |    6 +-
 .../typehints/typed_pipeline_test.py            |   33 +-
 sdks/python/apache_beam/typehints/typehints.py  |   79 +-
 .../apache_beam/typehints/typehints_test.py     |    3 +-
 .../apache_beam/utils/annotations_test.py       |    1 +
 sdks/python/apache_beam/utils/counters.py       |  107 +-
 sdks/python/apache_beam/utils/counters_test.py  |   78 +
 sdks/python/apache_beam/utils/processes_test.py |    1 -
 sdks/python/apache_beam/utils/proto_utils.py    |   11 +
 sdks/python/apache_beam/utils/retry.py          |    9 +-
 sdks/python/apache_beam/utils/retry_test.py     |    5 +-
 sdks/python/apache_beam/utils/urns.py           |   22 +-
 sdks/python/apache_beam/version.py              |    2 +-
 sdks/python/container/Dockerfile                |   27 +
 sdks/python/container/boot.go                   |  123 +
 sdks/python/container/pom.xml                   |  154 +
 sdks/python/gen_protos.py                       |   30 +-
 sdks/python/generate_pydoc.sh                   |  134 +-
 sdks/python/pom.xml                             |    9 +-
 sdks/python/run_postcommit.sh                   |   20 -
 sdks/python/run_pylint.sh                       |   45 +-
 sdks/python/run_validatesrunner.sh              |   71 +
 sdks/python/setup.py                            |   10 +-
 sdks/python/tox.ini                             |   21 +-
 1247 files changed, 92590 insertions(+), 17747 deletions(-)
----------------------------------------------------------------------



[03/50] [abbrv] beam git commit: [BEAM-2482] - CodedValueMutationDetector should use the coders structural value

Posted by ke...@apache.org.
[BEAM-2482] - CodedValueMutationDetector should use the coders structural value

This closes #4062


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

Branch: refs/heads/mr-runner
Commit: b743ab10dd9200eb89e4ee6bc4621ce7bb5a721d
Parents: e3f6d6f 84da6ca
Author: Luke Cwik <lc...@google.com>
Authored: Wed Nov 1 10:43:53 2017 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Wed Nov 1 10:43:53 2017 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/beam/sdk/util/MutationDetectors.java    | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------



[32/50] [abbrv] beam git commit: Add all portability protos to Go

Posted by ke...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/42100456/sdks/go/pkg/beam/model/fnexecution_v1/beam_provision_api.pb.go
----------------------------------------------------------------------
diff --git a/sdks/go/pkg/beam/model/fnexecution_v1/beam_provision_api.pb.go b/sdks/go/pkg/beam/model/fnexecution_v1/beam_provision_api.pb.go
index 0087fcf..a472885 100644
--- a/sdks/go/pkg/beam/model/fnexecution_v1/beam_provision_api.pb.go
+++ b/sdks/go/pkg/beam/model/fnexecution_v1/beam_provision_api.pb.go
@@ -1,39 +1,12 @@
-// Licensed to the Apache Software Foundation (ASF) under one or more
-// contributor license agreements.  See the NOTICE file distributed with
-// this work for additional information regarding copyright ownership.
-// The ASF licenses this file to You under the Apache License, Version 2.0
-// (the "License"); you may not use this file except in compliance with
-// the License.  You may obtain a copy of the License at
-//
-//    http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing, software
-// distributed under the License is distributed on an "AS IS" BASIS,
-// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// See the License for the specific language governing permissions and
-// limitations under the License.
-
 // Code generated by protoc-gen-go. DO NOT EDIT.
 // source: beam_provision_api.proto
 
-/*
-Package fnexecution_v1 is a generated protocol buffer package.
-
-It is generated from these files:
-	beam_provision_api.proto
-
-It has these top-level messages:
-	GetProvisionInfoRequest
-	GetProvisionInfoResponse
-	ProvisionInfo
-	Resources
-*/
 package fnexecution_v1
 
 import proto "github.com/golang/protobuf/proto"
 import fmt "fmt"
 import math "math"
-import google_protobuf "github.com/golang/protobuf/ptypes/struct"
+import google_protobuf2 "github.com/golang/protobuf/ptypes/struct"
 
 import (
 	context "golang.org/x/net/context"
@@ -45,12 +18,6 @@ var _ = proto.Marshal
 var _ = fmt.Errorf
 var _ = math.Inf
 
-// This is a compile-time assertion to ensure that this generated file
-// is compatible with the proto package it is being compiled against.
-// A compilation error at this line likely means your copy of the
-// proto package needs to be updated.
-const _ = proto.ProtoPackageIsVersion2 // please upgrade the proto package
-
 // A request to get the provision info of a SDK harness worker instance.
 type GetProvisionInfoRequest struct {
 }
@@ -58,7 +25,7 @@ type GetProvisionInfoRequest struct {
 func (m *GetProvisionInfoRequest) Reset()                    { *m = GetProvisionInfoRequest{} }
 func (m *GetProvisionInfoRequest) String() string            { return proto.CompactTextString(m) }
 func (*GetProvisionInfoRequest) ProtoMessage()               {}
-func (*GetProvisionInfoRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{0} }
+func (*GetProvisionInfoRequest) Descriptor() ([]byte, []int) { return fileDescriptor1, []int{0} }
 
 // A response containing the provision info of a SDK harness worker instance.
 type GetProvisionInfoResponse struct {
@@ -68,7 +35,7 @@ type GetProvisionInfoResponse struct {
 func (m *GetProvisionInfoResponse) Reset()                    { *m = GetProvisionInfoResponse{} }
 func (m *GetProvisionInfoResponse) String() string            { return proto.CompactTextString(m) }
 func (*GetProvisionInfoResponse) ProtoMessage()               {}
-func (*GetProvisionInfoResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{1} }
+func (*GetProvisionInfoResponse) Descriptor() ([]byte, []int) { return fileDescriptor1, []int{1} }
 
 func (m *GetProvisionInfoResponse) GetInfo() *ProvisionInfo {
 	if m != nil {
@@ -86,7 +53,7 @@ type ProvisionInfo struct {
 	JobName string `protobuf:"bytes,2,opt,name=job_name,json=jobName" json:"job_name,omitempty"`
 	// (required) Pipeline options. For non-template jobs, the options are
 	// identical to what is passed to job submission.
-	PipelineOptions *google_protobuf.Struct `protobuf:"bytes,3,opt,name=pipeline_options,json=pipelineOptions" json:"pipeline_options,omitempty"`
+	PipelineOptions *google_protobuf2.Struct `protobuf:"bytes,3,opt,name=pipeline_options,json=pipelineOptions" json:"pipeline_options,omitempty"`
 	// (optional) Resource limits that the SDK harness worker should respect.
 	// Runners may -- but are not required to -- enforce any limits provided.
 	ResourceLimits *Resources `protobuf:"bytes,4,opt,name=resource_limits,json=resourceLimits" json:"resource_limits,omitempty"`
@@ -95,7 +62,7 @@ type ProvisionInfo struct {
 func (m *ProvisionInfo) Reset()                    { *m = ProvisionInfo{} }
 func (m *ProvisionInfo) String() string            { return proto.CompactTextString(m) }
 func (*ProvisionInfo) ProtoMessage()               {}
-func (*ProvisionInfo) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{2} }
+func (*ProvisionInfo) Descriptor() ([]byte, []int) { return fileDescriptor1, []int{2} }
 
 func (m *ProvisionInfo) GetJobId() string {
 	if m != nil {
@@ -111,7 +78,7 @@ func (m *ProvisionInfo) GetJobName() string {
 	return ""
 }
 
-func (m *ProvisionInfo) GetPipelineOptions() *google_protobuf.Struct {
+func (m *ProvisionInfo) GetPipelineOptions() *google_protobuf2.Struct {
 	if m != nil {
 		return m.PipelineOptions
 	}
@@ -140,7 +107,7 @@ type Resources struct {
 func (m *Resources) Reset()                    { *m = Resources{} }
 func (m *Resources) String() string            { return proto.CompactTextString(m) }
 func (*Resources) ProtoMessage()               {}
-func (*Resources) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{3} }
+func (*Resources) Descriptor() ([]byte, []int) { return fileDescriptor1, []int{3} }
 
 func (m *Resources) GetMemory() *Resources_Memory {
 	if m != nil {
@@ -172,7 +139,7 @@ type Resources_Memory struct {
 func (m *Resources_Memory) Reset()                    { *m = Resources_Memory{} }
 func (m *Resources_Memory) String() string            { return proto.CompactTextString(m) }
 func (*Resources_Memory) ProtoMessage()               {}
-func (*Resources_Memory) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{3, 0} }
+func (*Resources_Memory) Descriptor() ([]byte, []int) { return fileDescriptor1, []int{3, 0} }
 
 func (m *Resources_Memory) GetSize() uint64 {
 	if m != nil {
@@ -191,7 +158,7 @@ type Resources_Cpu struct {
 func (m *Resources_Cpu) Reset()                    { *m = Resources_Cpu{} }
 func (m *Resources_Cpu) String() string            { return proto.CompactTextString(m) }
 func (*Resources_Cpu) ProtoMessage()               {}
-func (*Resources_Cpu) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{3, 1} }
+func (*Resources_Cpu) Descriptor() ([]byte, []int) { return fileDescriptor1, []int{3, 1} }
 
 func (m *Resources_Cpu) GetShares() float32 {
 	if m != nil {
@@ -209,7 +176,7 @@ type Resources_Disk struct {
 func (m *Resources_Disk) Reset()                    { *m = Resources_Disk{} }
 func (m *Resources_Disk) String() string            { return proto.CompactTextString(m) }
 func (*Resources_Disk) ProtoMessage()               {}
-func (*Resources_Disk) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{3, 2} }
+func (*Resources_Disk) Descriptor() ([]byte, []int) { return fileDescriptor1, []int{3, 2} }
 
 func (m *Resources_Disk) GetSize() uint64 {
 	if m != nil {
@@ -302,9 +269,9 @@ var _ProvisionService_serviceDesc = grpc.ServiceDesc{
 	Metadata: "beam_provision_api.proto",
 }
 
-func init() { proto.RegisterFile("beam_provision_api.proto", fileDescriptor0) }
+func init() { proto.RegisterFile("beam_provision_api.proto", fileDescriptor1) }
 
-var fileDescriptor0 = []byte{
+var fileDescriptor1 = []byte{
 	// 469 bytes of a gzipped FileDescriptorProto
 	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x93, 0xcf, 0x6e, 0xd3, 0x40,
 	0x10, 0xc6, 0xe5, 0xc6, 0x18, 0x3a, 0x40, 0x1b, 0xad, 0x80, 0xba, 0x56, 0x91, 0x50, 0x04, 0x12,

http://git-wip-us.apache.org/repos/asf/beam/blob/42100456/sdks/go/pkg/beam/model/gen.go
----------------------------------------------------------------------
diff --git a/sdks/go/pkg/beam/model/gen.go b/sdks/go/pkg/beam/model/gen.go
index d20007d..2c6de37 100644
--- a/sdks/go/pkg/beam/model/gen.go
+++ b/sdks/go/pkg/beam/model/gen.go
@@ -17,5 +17,6 @@ package model
 
 // TODO(herohde) 9/1/2017: for now, install protoc as described on grpc.io before running go generate.
 
-//go:generate protoc -I../../../../../model/job-management/src/main/proto ../../../../../model/job-management/src/main/proto/beam_artifact_api.proto --go_out=jobmanagement_v1,plugins=grpc:jobmanagement_v1
-//go:generate protoc -I../../../../../model/fn-execution/src/main/proto ../../../../../model/fn-execution/src/main/proto/beam_provision_api.proto --go_out=fnexecution_v1,plugins=grpc:fnexecution_v1
+//go:generate protoc -I../../../../../model/pipeline/src/main/proto ../../../../../model/pipeline/src/main/proto/beam_runner_api.proto ../../../../../model/pipeline/src/main/proto/endpoints.proto ../../../../../model/pipeline/src/main/proto/standard_window_fns.proto --go_out=pipeline_v1,plugins=grpc:pipeline_v1
+//go:generate protoc -I../../../../../model/pipeline/src/main/proto -I../../../../../model/job-management/src/main/proto ../../../../../model/job-management/src/main/proto/beam_job_api.proto ../../../../../model/job-management/src/main/proto/beam_artifact_api.proto --go_out=Mbeam_runner_api.proto=github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1,Mendpoints.proto=github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1,jobmanagement_v1,plugins=grpc:jobmanagement_v1
+//go:generate protoc -I../../../../../model/pipeline/src/main/proto -I../../../../../model/fn-execution/src/main/proto ../../../../../model/fn-execution/src/main/proto/beam_fn_api.proto ../../../../../model/fn-execution/src/main/proto/beam_provision_api.proto --go_out=Mbeam_runner_api.proto=github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1,Mendpoints.proto=github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1,fnexecution_v1,plugins=grpc:fnexecution_v1

http://git-wip-us.apache.org/repos/asf/beam/blob/42100456/sdks/go/pkg/beam/model/jobmanagement_v1/beam_artifact_api.pb.go
----------------------------------------------------------------------
diff --git a/sdks/go/pkg/beam/model/jobmanagement_v1/beam_artifact_api.pb.go b/sdks/go/pkg/beam/model/jobmanagement_v1/beam_artifact_api.pb.go
index fd83ae9..3a4940e 100644
--- a/sdks/go/pkg/beam/model/jobmanagement_v1/beam_artifact_api.pb.go
+++ b/sdks/go/pkg/beam/model/jobmanagement_v1/beam_artifact_api.pb.go
@@ -1,40 +1,6 @@
-// Licensed to the Apache Software Foundation (ASF) under one or more
-// contributor license agreements.  See the NOTICE file distributed with
-// this work for additional information regarding copyright ownership.
-// The ASF licenses this file to You under the Apache License, Version 2.0
-// (the "License"); you may not use this file except in compliance with
-// the License.  You may obtain a copy of the License at
-//
-//    http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing, software
-// distributed under the License is distributed on an "AS IS" BASIS,
-// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// See the License for the specific language governing permissions and
-// limitations under the License.
-
 // Code generated by protoc-gen-go. DO NOT EDIT.
 // source: beam_artifact_api.proto
 
-/*
-Package jobmanagement_v1 is a generated protocol buffer package.
-
-It is generated from these files:
-	beam_artifact_api.proto
-
-It has these top-level messages:
-	ArtifactMetadata
-	Manifest
-	ProxyManifest
-	GetManifestRequest
-	GetManifestResponse
-	GetArtifactRequest
-	ArtifactChunk
-	PutArtifactRequest
-	PutArtifactResponse
-	CommitManifestRequest
-	CommitManifestResponse
-*/
 package jobmanagement_v1
 
 import proto "github.com/golang/protobuf/proto"
@@ -51,12 +17,6 @@ var _ = proto.Marshal
 var _ = fmt.Errorf
 var _ = math.Inf
 
-// This is a compile-time assertion to ensure that this generated file
-// is compatible with the proto package it is being compiled against.
-// A compilation error at this line likely means your copy of the
-// proto package needs to be updated.
-const _ = proto.ProtoPackageIsVersion2 // please upgrade the proto package
-
 // An artifact identifier and associated metadata.
 type ArtifactMetadata struct {
 	// (Required) The name of the artifact.
@@ -71,7 +31,7 @@ type ArtifactMetadata struct {
 func (m *ArtifactMetadata) Reset()                    { *m = ArtifactMetadata{} }
 func (m *ArtifactMetadata) String() string            { return proto.CompactTextString(m) }
 func (*ArtifactMetadata) ProtoMessage()               {}
-func (*ArtifactMetadata) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{0} }
+func (*ArtifactMetadata) Descriptor() ([]byte, []int) { return fileDescriptor1, []int{0} }
 
 func (m *ArtifactMetadata) GetName() string {
 	if m != nil {
@@ -102,7 +62,7 @@ type Manifest struct {
 func (m *Manifest) Reset()                    { *m = Manifest{} }
 func (m *Manifest) String() string            { return proto.CompactTextString(m) }
 func (*Manifest) ProtoMessage()               {}
-func (*Manifest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{1} }
+func (*Manifest) Descriptor() ([]byte, []int) { return fileDescriptor1, []int{1} }
 
 func (m *Manifest) GetArtifact() []*ArtifactMetadata {
 	if m != nil {
@@ -120,7 +80,7 @@ type ProxyManifest struct {
 func (m *ProxyManifest) Reset()                    { *m = ProxyManifest{} }
 func (m *ProxyManifest) String() string            { return proto.CompactTextString(m) }
 func (*ProxyManifest) ProtoMessage()               {}
-func (*ProxyManifest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{2} }
+func (*ProxyManifest) Descriptor() ([]byte, []int) { return fileDescriptor1, []int{2} }
 
 func (m *ProxyManifest) GetManifest() *Manifest {
 	if m != nil {
@@ -144,7 +104,7 @@ type ProxyManifest_Location struct {
 func (m *ProxyManifest_Location) Reset()                    { *m = ProxyManifest_Location{} }
 func (m *ProxyManifest_Location) String() string            { return proto.CompactTextString(m) }
 func (*ProxyManifest_Location) ProtoMessage()               {}
-func (*ProxyManifest_Location) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{2, 0} }
+func (*ProxyManifest_Location) Descriptor() ([]byte, []int) { return fileDescriptor1, []int{2, 0} }
 
 func (m *ProxyManifest_Location) GetName() string {
 	if m != nil {
@@ -167,7 +127,7 @@ type GetManifestRequest struct {
 func (m *GetManifestRequest) Reset()                    { *m = GetManifestRequest{} }
 func (m *GetManifestRequest) String() string            { return proto.CompactTextString(m) }
 func (*GetManifestRequest) ProtoMessage()               {}
-func (*GetManifestRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{3} }
+func (*GetManifestRequest) Descriptor() ([]byte, []int) { return fileDescriptor1, []int{3} }
 
 // A response containing a job manifest.
 type GetManifestResponse struct {
@@ -177,7 +137,7 @@ type GetManifestResponse struct {
 func (m *GetManifestResponse) Reset()                    { *m = GetManifestResponse{} }
 func (m *GetManifestResponse) String() string            { return proto.CompactTextString(m) }
 func (*GetManifestResponse) ProtoMessage()               {}
-func (*GetManifestResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{4} }
+func (*GetManifestResponse) Descriptor() ([]byte, []int) { return fileDescriptor1, []int{4} }
 
 func (m *GetManifestResponse) GetManifest() *Manifest {
 	if m != nil {
@@ -195,7 +155,7 @@ type GetArtifactRequest struct {
 func (m *GetArtifactRequest) Reset()                    { *m = GetArtifactRequest{} }
 func (m *GetArtifactRequest) String() string            { return proto.CompactTextString(m) }
 func (*GetArtifactRequest) ProtoMessage()               {}
-func (*GetArtifactRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{5} }
+func (*GetArtifactRequest) Descriptor() ([]byte, []int) { return fileDescriptor1, []int{5} }
 
 func (m *GetArtifactRequest) GetName() string {
 	if m != nil {
@@ -212,7 +172,7 @@ type ArtifactChunk struct {
 func (m *ArtifactChunk) Reset()                    { *m = ArtifactChunk{} }
 func (m *ArtifactChunk) String() string            { return proto.CompactTextString(m) }
 func (*ArtifactChunk) ProtoMessage()               {}
-func (*ArtifactChunk) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{6} }
+func (*ArtifactChunk) Descriptor() ([]byte, []int) { return fileDescriptor1, []int{6} }
 
 func (m *ArtifactChunk) GetData() []byte {
 	if m != nil {
@@ -234,7 +194,7 @@ type PutArtifactRequest struct {
 func (m *PutArtifactRequest) Reset()                    { *m = PutArtifactRequest{} }
 func (m *PutArtifactRequest) String() string            { return proto.CompactTextString(m) }
 func (*PutArtifactRequest) ProtoMessage()               {}
-func (*PutArtifactRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{7} }
+func (*PutArtifactRequest) Descriptor() ([]byte, []int) { return fileDescriptor1, []int{7} }
 
 type isPutArtifactRequest_Content interface {
 	isPutArtifactRequest_Content()
@@ -351,7 +311,7 @@ type PutArtifactResponse struct {
 func (m *PutArtifactResponse) Reset()                    { *m = PutArtifactResponse{} }
 func (m *PutArtifactResponse) String() string            { return proto.CompactTextString(m) }
 func (*PutArtifactResponse) ProtoMessage()               {}
-func (*PutArtifactResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{8} }
+func (*PutArtifactResponse) Descriptor() ([]byte, []int) { return fileDescriptor1, []int{8} }
 
 // A request to commit the manifest for a Job. All artifacts must have been successfully uploaded
 // before this call is made.
@@ -363,7 +323,7 @@ type CommitManifestRequest struct {
 func (m *CommitManifestRequest) Reset()                    { *m = CommitManifestRequest{} }
 func (m *CommitManifestRequest) String() string            { return proto.CompactTextString(m) }
 func (*CommitManifestRequest) ProtoMessage()               {}
-func (*CommitManifestRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{9} }
+func (*CommitManifestRequest) Descriptor() ([]byte, []int) { return fileDescriptor1, []int{9} }
 
 func (m *CommitManifestRequest) GetManifest() *Manifest {
 	if m != nil {
@@ -381,7 +341,7 @@ type CommitManifestResponse struct {
 func (m *CommitManifestResponse) Reset()                    { *m = CommitManifestResponse{} }
 func (m *CommitManifestResponse) String() string            { return proto.CompactTextString(m) }
 func (*CommitManifestResponse) ProtoMessage()               {}
-func (*CommitManifestResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{10} }
+func (*CommitManifestResponse) Descriptor() ([]byte, []int) { return fileDescriptor1, []int{10} }
 
 func (m *CommitManifestResponse) GetStagingToken() string {
 	if m != nil {
@@ -688,9 +648,9 @@ var _ArtifactRetrievalService_serviceDesc = grpc.ServiceDesc{
 	Metadata: "beam_artifact_api.proto",
 }
 
-func init() { proto.RegisterFile("beam_artifact_api.proto", fileDescriptor0) }
+func init() { proto.RegisterFile("beam_artifact_api.proto", fileDescriptor1) }
 
-var fileDescriptor0 = []byte{
+var fileDescriptor1 = []byte{
 	// 557 bytes of a gzipped FileDescriptorProto
 	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x55, 0xcf, 0x6e, 0xd3, 0x4e,
 	0x10, 0xee, 0x26, 0x3f, 0xfd, 0x70, 0xc6, 0x0d, 0x8a, 0xb6, 0xb4, 0x58, 0x39, 0x45, 0x5b, 0x09,

http://git-wip-us.apache.org/repos/asf/beam/blob/42100456/sdks/go/pkg/beam/model/jobmanagement_v1/beam_job_api.pb.go
----------------------------------------------------------------------
diff --git a/sdks/go/pkg/beam/model/jobmanagement_v1/beam_job_api.pb.go b/sdks/go/pkg/beam/model/jobmanagement_v1/beam_job_api.pb.go
new file mode 100644
index 0000000..575dbd9
--- /dev/null
+++ b/sdks/go/pkg/beam/model/jobmanagement_v1/beam_job_api.pb.go
@@ -0,0 +1,903 @@
+// Code generated by protoc-gen-go. DO NOT EDIT.
+// source: beam_job_api.proto
+
+/*
+Package jobmanagement_v1 is a generated protocol buffer package.
+
+It is generated from these files:
+	beam_job_api.proto
+	beam_artifact_api.proto
+
+It has these top-level messages:
+	PrepareJobRequest
+	PrepareJobResponse
+	RunJobRequest
+	RunJobResponse
+	CancelJobRequest
+	CancelJobResponse
+	GetJobStateRequest
+	GetJobStateResponse
+	JobMessagesRequest
+	JobMessage
+	JobMessagesResponse
+	JobState
+	ArtifactMetadata
+	Manifest
+	ProxyManifest
+	GetManifestRequest
+	GetManifestResponse
+	GetArtifactRequest
+	ArtifactChunk
+	PutArtifactRequest
+	PutArtifactResponse
+	CommitManifestRequest
+	CommitManifestResponse
+*/
+package jobmanagement_v1
+
+import proto "github.com/golang/protobuf/proto"
+import fmt "fmt"
+import math "math"
+import org_apache_beam_model_pipeline_v1 "github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1"
+import org_apache_beam_model_pipeline_v11 "github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1"
+import google_protobuf1 "github.com/golang/protobuf/ptypes/struct"
+
+import (
+	context "golang.org/x/net/context"
+	grpc "google.golang.org/grpc"
+)
+
+// Reference imports to suppress errors if they are not otherwise used.
+var _ = proto.Marshal
+var _ = fmt.Errorf
+var _ = math.Inf
+
+// This is a compile-time assertion to ensure that this generated file
+// is compatible with the proto package it is being compiled against.
+// A compilation error at this line likely means your copy of the
+// proto package needs to be updated.
+const _ = proto.ProtoPackageIsVersion2 // please upgrade the proto package
+
+type JobMessage_MessageImportance int32
+
+const (
+	JobMessage_MESSAGE_IMPORTANCE_UNSPECIFIED JobMessage_MessageImportance = 0
+	JobMessage_JOB_MESSAGE_DEBUG              JobMessage_MessageImportance = 1
+	JobMessage_JOB_MESSAGE_DETAILED           JobMessage_MessageImportance = 2
+	JobMessage_JOB_MESSAGE_BASIC              JobMessage_MessageImportance = 3
+	JobMessage_JOB_MESSAGE_WARNING            JobMessage_MessageImportance = 4
+	JobMessage_JOB_MESSAGE_ERROR              JobMessage_MessageImportance = 5
+)
+
+var JobMessage_MessageImportance_name = map[int32]string{
+	0: "MESSAGE_IMPORTANCE_UNSPECIFIED",
+	1: "JOB_MESSAGE_DEBUG",
+	2: "JOB_MESSAGE_DETAILED",
+	3: "JOB_MESSAGE_BASIC",
+	4: "JOB_MESSAGE_WARNING",
+	5: "JOB_MESSAGE_ERROR",
+}
+var JobMessage_MessageImportance_value = map[string]int32{
+	"MESSAGE_IMPORTANCE_UNSPECIFIED": 0,
+	"JOB_MESSAGE_DEBUG":              1,
+	"JOB_MESSAGE_DETAILED":           2,
+	"JOB_MESSAGE_BASIC":              3,
+	"JOB_MESSAGE_WARNING":            4,
+	"JOB_MESSAGE_ERROR":              5,
+}
+
+func (x JobMessage_MessageImportance) String() string {
+	return proto.EnumName(JobMessage_MessageImportance_name, int32(x))
+}
+func (JobMessage_MessageImportance) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor0, []int{9, 0}
+}
+
+type JobState_Enum int32
+
+const (
+	JobState_UNSPECIFIED JobState_Enum = 0
+	JobState_STOPPED     JobState_Enum = 1
+	JobState_RUNNING     JobState_Enum = 2
+	JobState_DONE        JobState_Enum = 3
+	JobState_FAILED      JobState_Enum = 4
+	JobState_CANCELLED   JobState_Enum = 5
+	JobState_UPDATED     JobState_Enum = 6
+	JobState_DRAINING    JobState_Enum = 7
+	JobState_DRAINED     JobState_Enum = 8
+	JobState_STARTING    JobState_Enum = 9
+	JobState_CANCELLING  JobState_Enum = 10
+)
+
+var JobState_Enum_name = map[int32]string{
+	0:  "UNSPECIFIED",
+	1:  "STOPPED",
+	2:  "RUNNING",
+	3:  "DONE",
+	4:  "FAILED",
+	5:  "CANCELLED",
+	6:  "UPDATED",
+	7:  "DRAINING",
+	8:  "DRAINED",
+	9:  "STARTING",
+	10: "CANCELLING",
+}
+var JobState_Enum_value = map[string]int32{
+	"UNSPECIFIED": 0,
+	"STOPPED":     1,
+	"RUNNING":     2,
+	"DONE":        3,
+	"FAILED":      4,
+	"CANCELLED":   5,
+	"UPDATED":     6,
+	"DRAINING":    7,
+	"DRAINED":     8,
+	"STARTING":    9,
+	"CANCELLING":  10,
+}
+
+func (x JobState_Enum) String() string {
+	return proto.EnumName(JobState_Enum_name, int32(x))
+}
+func (JobState_Enum) EnumDescriptor() ([]byte, []int) { return fileDescriptor0, []int{11, 0} }
+
+// Prepare is a synchronous request that returns a preparationId back
+// Throws error GRPC_STATUS_UNAVAILABLE if server is down
+// Throws error ALREADY_EXISTS if the jobName is reused. Runners are permitted to deduplicate based on the name of the job.
+// Throws error UNKNOWN for all other issues
+type PrepareJobRequest struct {
+	Pipeline        *org_apache_beam_model_pipeline_v1.Pipeline `protobuf:"bytes,1,opt,name=pipeline" json:"pipeline,omitempty"`
+	PipelineOptions *google_protobuf1.Struct                    `protobuf:"bytes,2,opt,name=pipeline_options,json=pipelineOptions" json:"pipeline_options,omitempty"`
+	JobName         string                                      `protobuf:"bytes,3,opt,name=job_name,json=jobName" json:"job_name,omitempty"`
+}
+
+func (m *PrepareJobRequest) Reset()                    { *m = PrepareJobRequest{} }
+func (m *PrepareJobRequest) String() string            { return proto.CompactTextString(m) }
+func (*PrepareJobRequest) ProtoMessage()               {}
+func (*PrepareJobRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{0} }
+
+func (m *PrepareJobRequest) GetPipeline() *org_apache_beam_model_pipeline_v1.Pipeline {
+	if m != nil {
+		return m.Pipeline
+	}
+	return nil
+}
+
+func (m *PrepareJobRequest) GetPipelineOptions() *google_protobuf1.Struct {
+	if m != nil {
+		return m.PipelineOptions
+	}
+	return nil
+}
+
+func (m *PrepareJobRequest) GetJobName() string {
+	if m != nil {
+		return m.JobName
+	}
+	return ""
+}
+
+type PrepareJobResponse struct {
+	// (required) The ID used to associate calls made while preparing the job. preparationId is used
+	// to run the job, as well as in other pre-execution APIs such as Artifact staging.
+	PreparationId string `protobuf:"bytes,1,opt,name=preparation_id,json=preparationId" json:"preparation_id,omitempty"`
+	// An endpoint which exposes the Beam Artifact Staging API. Artifacts used by the job should be
+	// staged to this endpoint, and will be available during job execution.
+	ArtifactStagingEndpoint *org_apache_beam_model_pipeline_v11.ApiServiceDescriptor `protobuf:"bytes,2,opt,name=artifact_staging_endpoint,json=artifactStagingEndpoint" json:"artifact_staging_endpoint,omitempty"`
+}
+
+func (m *PrepareJobResponse) Reset()                    { *m = PrepareJobResponse{} }
+func (m *PrepareJobResponse) String() string            { return proto.CompactTextString(m) }
+func (*PrepareJobResponse) ProtoMessage()               {}
+func (*PrepareJobResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{1} }
+
+func (m *PrepareJobResponse) GetPreparationId() string {
+	if m != nil {
+		return m.PreparationId
+	}
+	return ""
+}
+
+func (m *PrepareJobResponse) GetArtifactStagingEndpoint() *org_apache_beam_model_pipeline_v11.ApiServiceDescriptor {
+	if m != nil {
+		return m.ArtifactStagingEndpoint
+	}
+	return nil
+}
+
+// Run is a synchronous request that returns a jobId back.
+// Throws error GRPC_STATUS_UNAVAILABLE if server is down
+// Throws error NOT_FOUND if the preparation ID does not exist
+// Throws error UNKNOWN for all other issues
+type RunJobRequest struct {
+	// (required) The ID provided by an earlier call to prepare. Runs the job. All prerequisite tasks
+	// must have been completed.
+	PreparationId string `protobuf:"bytes,1,opt,name=preparation_id,json=preparationId" json:"preparation_id,omitempty"`
+	// (optional) If any artifacts have been staged for this job, contains the staging_token returned
+	// from the CommitManifestResponse.
+	StagingToken string `protobuf:"bytes,2,opt,name=staging_token,json=stagingToken" json:"staging_token,omitempty"`
+}
+
+func (m *RunJobRequest) Reset()                    { *m = RunJobRequest{} }
+func (m *RunJobRequest) String() string            { return proto.CompactTextString(m) }
+func (*RunJobRequest) ProtoMessage()               {}
+func (*RunJobRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{2} }
+
+func (m *RunJobRequest) GetPreparationId() string {
+	if m != nil {
+		return m.PreparationId
+	}
+	return ""
+}
+
+func (m *RunJobRequest) GetStagingToken() string {
+	if m != nil {
+		return m.StagingToken
+	}
+	return ""
+}
+
+type RunJobResponse struct {
+	JobId string `protobuf:"bytes,1,opt,name=job_id,json=jobId" json:"job_id,omitempty"`
+}
+
+func (m *RunJobResponse) Reset()                    { *m = RunJobResponse{} }
+func (m *RunJobResponse) String() string            { return proto.CompactTextString(m) }
+func (*RunJobResponse) ProtoMessage()               {}
+func (*RunJobResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{3} }
+
+func (m *RunJobResponse) GetJobId() string {
+	if m != nil {
+		return m.JobId
+	}
+	return ""
+}
+
+// Cancel is a synchronus request that returns a job state back
+// Throws error GRPC_STATUS_UNAVAILABLE if server is down
+// Throws error NOT_FOUND if the jobId is not found
+type CancelJobRequest struct {
+	JobId string `protobuf:"bytes,1,opt,name=job_id,json=jobId" json:"job_id,omitempty"`
+}
+
+func (m *CancelJobRequest) Reset()                    { *m = CancelJobRequest{} }
+func (m *CancelJobRequest) String() string            { return proto.CompactTextString(m) }
+func (*CancelJobRequest) ProtoMessage()               {}
+func (*CancelJobRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{4} }
+
+func (m *CancelJobRequest) GetJobId() string {
+	if m != nil {
+		return m.JobId
+	}
+	return ""
+}
+
+// Valid responses include any terminal state or CANCELLING
+type CancelJobResponse struct {
+	State JobState_Enum `protobuf:"varint,1,opt,name=state,enum=org.apache.beam.model.job_management.v1.JobState_Enum" json:"state,omitempty"`
+}
+
+func (m *CancelJobResponse) Reset()                    { *m = CancelJobResponse{} }
+func (m *CancelJobResponse) String() string            { return proto.CompactTextString(m) }
+func (*CancelJobResponse) ProtoMessage()               {}
+func (*CancelJobResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{5} }
+
+func (m *CancelJobResponse) GetState() JobState_Enum {
+	if m != nil {
+		return m.State
+	}
+	return JobState_UNSPECIFIED
+}
+
+// GetState is a synchronus request that returns a job state back
+// Throws error GRPC_STATUS_UNAVAILABLE if server is down
+// Throws error NOT_FOUND if the jobId is not found
+type GetJobStateRequest struct {
+	JobId string `protobuf:"bytes,1,opt,name=job_id,json=jobId" json:"job_id,omitempty"`
+}
+
+func (m *GetJobStateRequest) Reset()                    { *m = GetJobStateRequest{} }
+func (m *GetJobStateRequest) String() string            { return proto.CompactTextString(m) }
+func (*GetJobStateRequest) ProtoMessage()               {}
+func (*GetJobStateRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{6} }
+
+func (m *GetJobStateRequest) GetJobId() string {
+	if m != nil {
+		return m.JobId
+	}
+	return ""
+}
+
+type GetJobStateResponse struct {
+	State JobState_Enum `protobuf:"varint,1,opt,name=state,enum=org.apache.beam.model.job_management.v1.JobState_Enum" json:"state,omitempty"`
+}
+
+func (m *GetJobStateResponse) Reset()                    { *m = GetJobStateResponse{} }
+func (m *GetJobStateResponse) String() string            { return proto.CompactTextString(m) }
+func (*GetJobStateResponse) ProtoMessage()               {}
+func (*GetJobStateResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{7} }
+
+func (m *GetJobStateResponse) GetState() JobState_Enum {
+	if m != nil {
+		return m.State
+	}
+	return JobState_UNSPECIFIED
+}
+
+// GetJobMessages is a streaming api for streaming job messages from the service
+// One request will connect you to the job and you'll get a stream of job state
+// and job messages back; one is used for logging and the other for detecting
+// the job ended.
+type JobMessagesRequest struct {
+	JobId string `protobuf:"bytes,1,opt,name=job_id,json=jobId" json:"job_id,omitempty"`
+}
+
+func (m *JobMessagesRequest) Reset()                    { *m = JobMessagesRequest{} }
+func (m *JobMessagesRequest) String() string            { return proto.CompactTextString(m) }
+func (*JobMessagesRequest) ProtoMessage()               {}
+func (*JobMessagesRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{8} }
+
+func (m *JobMessagesRequest) GetJobId() string {
+	if m != nil {
+		return m.JobId
+	}
+	return ""
+}
+
+type JobMessage struct {
+	MessageId   string                       `protobuf:"bytes,1,opt,name=message_id,json=messageId" json:"message_id,omitempty"`
+	Time        string                       `protobuf:"bytes,2,opt,name=time" json:"time,omitempty"`
+	Importance  JobMessage_MessageImportance `protobuf:"varint,3,opt,name=importance,enum=org.apache.beam.model.job_management.v1.JobMessage_MessageImportance" json:"importance,omitempty"`
+	MessageText string                       `protobuf:"bytes,4,opt,name=message_text,json=messageText" json:"message_text,omitempty"`
+}
+
+func (m *JobMessage) Reset()                    { *m = JobMessage{} }
+func (m *JobMessage) String() string            { return proto.CompactTextString(m) }
+func (*JobMessage) ProtoMessage()               {}
+func (*JobMessage) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{9} }
+
+func (m *JobMessage) GetMessageId() string {
+	if m != nil {
+		return m.MessageId
+	}
+	return ""
+}
+
+func (m *JobMessage) GetTime() string {
+	if m != nil {
+		return m.Time
+	}
+	return ""
+}
+
+func (m *JobMessage) GetImportance() JobMessage_MessageImportance {
+	if m != nil {
+		return m.Importance
+	}
+	return JobMessage_MESSAGE_IMPORTANCE_UNSPECIFIED
+}
+
+func (m *JobMessage) GetMessageText() string {
+	if m != nil {
+		return m.MessageText
+	}
+	return ""
+}
+
+type JobMessagesResponse struct {
+	// Types that are valid to be assigned to Response:
+	//	*JobMessagesResponse_MessageResponse
+	//	*JobMessagesResponse_StateResponse
+	Response isJobMessagesResponse_Response `protobuf_oneof:"response"`
+}
+
+func (m *JobMessagesResponse) Reset()                    { *m = JobMessagesResponse{} }
+func (m *JobMessagesResponse) String() string            { return proto.CompactTextString(m) }
+func (*JobMessagesResponse) ProtoMessage()               {}
+func (*JobMessagesResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{10} }
+
+type isJobMessagesResponse_Response interface {
+	isJobMessagesResponse_Response()
+}
+
+type JobMessagesResponse_MessageResponse struct {
+	MessageResponse *JobMessage `protobuf:"bytes,1,opt,name=message_response,json=messageResponse,oneof"`
+}
+type JobMessagesResponse_StateResponse struct {
+	StateResponse *GetJobStateResponse `protobuf:"bytes,2,opt,name=state_response,json=stateResponse,oneof"`
+}
+
+func (*JobMessagesResponse_MessageResponse) isJobMessagesResponse_Response() {}
+func (*JobMessagesResponse_StateResponse) isJobMessagesResponse_Response()   {}
+
+func (m *JobMessagesResponse) GetResponse() isJobMessagesResponse_Response {
+	if m != nil {
+		return m.Response
+	}
+	return nil
+}
+
+func (m *JobMessagesResponse) GetMessageResponse() *JobMessage {
+	if x, ok := m.GetResponse().(*JobMessagesResponse_MessageResponse); ok {
+		return x.MessageResponse
+	}
+	return nil
+}
+
+func (m *JobMessagesResponse) GetStateResponse() *GetJobStateResponse {
+	if x, ok := m.GetResponse().(*JobMessagesResponse_StateResponse); ok {
+		return x.StateResponse
+	}
+	return nil
+}
+
+// XXX_OneofFuncs is for the internal use of the proto package.
+func (*JobMessagesResponse) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) error, func(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error), func(msg proto.Message) (n int), []interface{}) {
+	return _JobMessagesResponse_OneofMarshaler, _JobMessagesResponse_OneofUnmarshaler, _JobMessagesResponse_OneofSizer, []interface{}{
+		(*JobMessagesResponse_MessageResponse)(nil),
+		(*JobMessagesResponse_StateResponse)(nil),
+	}
+}
+
+func _JobMessagesResponse_OneofMarshaler(msg proto.Message, b *proto.Buffer) error {
+	m := msg.(*JobMessagesResponse)
+	// response
+	switch x := m.Response.(type) {
+	case *JobMessagesResponse_MessageResponse:
+		b.EncodeVarint(1<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.MessageResponse); err != nil {
+			return err
+		}
+	case *JobMessagesResponse_StateResponse:
+		b.EncodeVarint(2<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.StateResponse); err != nil {
+			return err
+		}
+	case nil:
+	default:
+		return fmt.Errorf("JobMessagesResponse.Response has unexpected type %T", x)
+	}
+	return nil
+}
+
+func _JobMessagesResponse_OneofUnmarshaler(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error) {
+	m := msg.(*JobMessagesResponse)
+	switch tag {
+	case 1: // response.message_response
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(JobMessage)
+		err := b.DecodeMessage(msg)
+		m.Response = &JobMessagesResponse_MessageResponse{msg}
+		return true, err
+	case 2: // response.state_response
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(GetJobStateResponse)
+		err := b.DecodeMessage(msg)
+		m.Response = &JobMessagesResponse_StateResponse{msg}
+		return true, err
+	default:
+		return false, nil
+	}
+}
+
+func _JobMessagesResponse_OneofSizer(msg proto.Message) (n int) {
+	m := msg.(*JobMessagesResponse)
+	// response
+	switch x := m.Response.(type) {
+	case *JobMessagesResponse_MessageResponse:
+		s := proto.Size(x.MessageResponse)
+		n += proto.SizeVarint(1<<3 | proto.WireBytes)
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case *JobMessagesResponse_StateResponse:
+		s := proto.Size(x.StateResponse)
+		n += proto.SizeVarint(2<<3 | proto.WireBytes)
+		n += proto.SizeVarint(uint64(s))
+		n += s
+	case nil:
+	default:
+		panic(fmt.Sprintf("proto: unexpected type %T in oneof", x))
+	}
+	return n
+}
+
+// Enumeration of all JobStates
+type JobState struct {
+}
+
+func (m *JobState) Reset()                    { *m = JobState{} }
+func (m *JobState) String() string            { return proto.CompactTextString(m) }
+func (*JobState) ProtoMessage()               {}
+func (*JobState) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{11} }
+
+func init() {
+	proto.RegisterType((*PrepareJobRequest)(nil), "org.apache.beam.model.job_management.v1.PrepareJobRequest")
+	proto.RegisterType((*PrepareJobResponse)(nil), "org.apache.beam.model.job_management.v1.PrepareJobResponse")
+	proto.RegisterType((*RunJobRequest)(nil), "org.apache.beam.model.job_management.v1.RunJobRequest")
+	proto.RegisterType((*RunJobResponse)(nil), "org.apache.beam.model.job_management.v1.RunJobResponse")
+	proto.RegisterType((*CancelJobRequest)(nil), "org.apache.beam.model.job_management.v1.CancelJobRequest")
+	proto.RegisterType((*CancelJobResponse)(nil), "org.apache.beam.model.job_management.v1.CancelJobResponse")
+	proto.RegisterType((*GetJobStateRequest)(nil), "org.apache.beam.model.job_management.v1.GetJobStateRequest")
+	proto.RegisterType((*GetJobStateResponse)(nil), "org.apache.beam.model.job_management.v1.GetJobStateResponse")
+	proto.RegisterType((*JobMessagesRequest)(nil), "org.apache.beam.model.job_management.v1.JobMessagesRequest")
+	proto.RegisterType((*JobMessage)(nil), "org.apache.beam.model.job_management.v1.JobMessage")
+	proto.RegisterType((*JobMessagesResponse)(nil), "org.apache.beam.model.job_management.v1.JobMessagesResponse")
+	proto.RegisterType((*JobState)(nil), "org.apache.beam.model.job_management.v1.JobState")
+	proto.RegisterEnum("org.apache.beam.model.job_management.v1.JobMessage_MessageImportance", JobMessage_MessageImportance_name, JobMessage_MessageImportance_value)
+	proto.RegisterEnum("org.apache.beam.model.job_management.v1.JobState_Enum", JobState_Enum_name, JobState_Enum_value)
+}
+
+// Reference imports to suppress errors if they are not otherwise used.
+var _ context.Context
+var _ grpc.ClientConn
+
+// This is a compile-time assertion to ensure that this generated file
+// is compatible with the grpc package it is being compiled against.
+const _ = grpc.SupportPackageIsVersion4
+
+// Client API for JobService service
+
+type JobServiceClient interface {
+	// Prepare a job for execution. The job will not be executed until a call is made to run with the
+	// returned preparationId.
+	Prepare(ctx context.Context, in *PrepareJobRequest, opts ...grpc.CallOption) (*PrepareJobResponse, error)
+	// Submit the job for execution
+	Run(ctx context.Context, in *RunJobRequest, opts ...grpc.CallOption) (*RunJobResponse, error)
+	// Get the current state of the job
+	GetState(ctx context.Context, in *GetJobStateRequest, opts ...grpc.CallOption) (*GetJobStateResponse, error)
+	// Cancel the job
+	Cancel(ctx context.Context, in *CancelJobRequest, opts ...grpc.CallOption) (*CancelJobResponse, error)
+	// Subscribe to a stream of state changes of the job, will immediately return the current state of the job as the first response.
+	GetStateStream(ctx context.Context, in *GetJobStateRequest, opts ...grpc.CallOption) (JobService_GetStateStreamClient, error)
+	// Subscribe to a stream of state changes and messages from the job
+	GetMessageStream(ctx context.Context, in *JobMessagesRequest, opts ...grpc.CallOption) (JobService_GetMessageStreamClient, error)
+}
+
+type jobServiceClient struct {
+	cc *grpc.ClientConn
+}
+
+func NewJobServiceClient(cc *grpc.ClientConn) JobServiceClient {
+	return &jobServiceClient{cc}
+}
+
+func (c *jobServiceClient) Prepare(ctx context.Context, in *PrepareJobRequest, opts ...grpc.CallOption) (*PrepareJobResponse, error) {
+	out := new(PrepareJobResponse)
+	err := grpc.Invoke(ctx, "/org.apache.beam.model.job_management.v1.JobService/Prepare", in, out, c.cc, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *jobServiceClient) Run(ctx context.Context, in *RunJobRequest, opts ...grpc.CallOption) (*RunJobResponse, error) {
+	out := new(RunJobResponse)
+	err := grpc.Invoke(ctx, "/org.apache.beam.model.job_management.v1.JobService/Run", in, out, c.cc, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *jobServiceClient) GetState(ctx context.Context, in *GetJobStateRequest, opts ...grpc.CallOption) (*GetJobStateResponse, error) {
+	out := new(GetJobStateResponse)
+	err := grpc.Invoke(ctx, "/org.apache.beam.model.job_management.v1.JobService/GetState", in, out, c.cc, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *jobServiceClient) Cancel(ctx context.Context, in *CancelJobRequest, opts ...grpc.CallOption) (*CancelJobResponse, error) {
+	out := new(CancelJobResponse)
+	err := grpc.Invoke(ctx, "/org.apache.beam.model.job_management.v1.JobService/Cancel", in, out, c.cc, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *jobServiceClient) GetStateStream(ctx context.Context, in *GetJobStateRequest, opts ...grpc.CallOption) (JobService_GetStateStreamClient, error) {
+	stream, err := grpc.NewClientStream(ctx, &_JobService_serviceDesc.Streams[0], c.cc, "/org.apache.beam.model.job_management.v1.JobService/GetStateStream", opts...)
+	if err != nil {
+		return nil, err
+	}
+	x := &jobServiceGetStateStreamClient{stream}
+	if err := x.ClientStream.SendMsg(in); err != nil {
+		return nil, err
+	}
+	if err := x.ClientStream.CloseSend(); err != nil {
+		return nil, err
+	}
+	return x, nil
+}
+
+type JobService_GetStateStreamClient interface {
+	Recv() (*GetJobStateResponse, error)
+	grpc.ClientStream
+}
+
+type jobServiceGetStateStreamClient struct {
+	grpc.ClientStream
+}
+
+func (x *jobServiceGetStateStreamClient) Recv() (*GetJobStateResponse, error) {
+	m := new(GetJobStateResponse)
+	if err := x.ClientStream.RecvMsg(m); err != nil {
+		return nil, err
+	}
+	return m, nil
+}
+
+func (c *jobServiceClient) GetMessageStream(ctx context.Context, in *JobMessagesRequest, opts ...grpc.CallOption) (JobService_GetMessageStreamClient, error) {
+	stream, err := grpc.NewClientStream(ctx, &_JobService_serviceDesc.Streams[1], c.cc, "/org.apache.beam.model.job_management.v1.JobService/GetMessageStream", opts...)
+	if err != nil {
+		return nil, err
+	}
+	x := &jobServiceGetMessageStreamClient{stream}
+	if err := x.ClientStream.SendMsg(in); err != nil {
+		return nil, err
+	}
+	if err := x.ClientStream.CloseSend(); err != nil {
+		return nil, err
+	}
+	return x, nil
+}
+
+type JobService_GetMessageStreamClient interface {
+	Recv() (*JobMessagesResponse, error)
+	grpc.ClientStream
+}
+
+type jobServiceGetMessageStreamClient struct {
+	grpc.ClientStream
+}
+
+func (x *jobServiceGetMessageStreamClient) Recv() (*JobMessagesResponse, error) {
+	m := new(JobMessagesResponse)
+	if err := x.ClientStream.RecvMsg(m); err != nil {
+		return nil, err
+	}
+	return m, nil
+}
+
+// Server API for JobService service
+
+type JobServiceServer interface {
+	// Prepare a job for execution. The job will not be executed until a call is made to run with the
+	// returned preparationId.
+	Prepare(context.Context, *PrepareJobRequest) (*PrepareJobResponse, error)
+	// Submit the job for execution
+	Run(context.Context, *RunJobRequest) (*RunJobResponse, error)
+	// Get the current state of the job
+	GetState(context.Context, *GetJobStateRequest) (*GetJobStateResponse, error)
+	// Cancel the job
+	Cancel(context.Context, *CancelJobRequest) (*CancelJobResponse, error)
+	// Subscribe to a stream of state changes of the job, will immediately return the current state of the job as the first response.
+	GetStateStream(*GetJobStateRequest, JobService_GetStateStreamServer) error
+	// Subscribe to a stream of state changes and messages from the job
+	GetMessageStream(*JobMessagesRequest, JobService_GetMessageStreamServer) error
+}
+
+func RegisterJobServiceServer(s *grpc.Server, srv JobServiceServer) {
+	s.RegisterService(&_JobService_serviceDesc, srv)
+}
+
+func _JobService_Prepare_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(PrepareJobRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(JobServiceServer).Prepare(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/org.apache.beam.model.job_management.v1.JobService/Prepare",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(JobServiceServer).Prepare(ctx, req.(*PrepareJobRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _JobService_Run_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(RunJobRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(JobServiceServer).Run(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/org.apache.beam.model.job_management.v1.JobService/Run",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(JobServiceServer).Run(ctx, req.(*RunJobRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _JobService_GetState_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(GetJobStateRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(JobServiceServer).GetState(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/org.apache.beam.model.job_management.v1.JobService/GetState",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(JobServiceServer).GetState(ctx, req.(*GetJobStateRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _JobService_Cancel_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(CancelJobRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(JobServiceServer).Cancel(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/org.apache.beam.model.job_management.v1.JobService/Cancel",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(JobServiceServer).Cancel(ctx, req.(*CancelJobRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _JobService_GetStateStream_Handler(srv interface{}, stream grpc.ServerStream) error {
+	m := new(GetJobStateRequest)
+	if err := stream.RecvMsg(m); err != nil {
+		return err
+	}
+	return srv.(JobServiceServer).GetStateStream(m, &jobServiceGetStateStreamServer{stream})
+}
+
+type JobService_GetStateStreamServer interface {
+	Send(*GetJobStateResponse) error
+	grpc.ServerStream
+}
+
+type jobServiceGetStateStreamServer struct {
+	grpc.ServerStream
+}
+
+func (x *jobServiceGetStateStreamServer) Send(m *GetJobStateResponse) error {
+	return x.ServerStream.SendMsg(m)
+}
+
+func _JobService_GetMessageStream_Handler(srv interface{}, stream grpc.ServerStream) error {
+	m := new(JobMessagesRequest)
+	if err := stream.RecvMsg(m); err != nil {
+		return err
+	}
+	return srv.(JobServiceServer).GetMessageStream(m, &jobServiceGetMessageStreamServer{stream})
+}
+
+type JobService_GetMessageStreamServer interface {
+	Send(*JobMessagesResponse) error
+	grpc.ServerStream
+}
+
+type jobServiceGetMessageStreamServer struct {
+	grpc.ServerStream
+}
+
+func (x *jobServiceGetMessageStreamServer) Send(m *JobMessagesResponse) error {
+	return x.ServerStream.SendMsg(m)
+}
+
+var _JobService_serviceDesc = grpc.ServiceDesc{
+	ServiceName: "org.apache.beam.model.job_management.v1.JobService",
+	HandlerType: (*JobServiceServer)(nil),
+	Methods: []grpc.MethodDesc{
+		{
+			MethodName: "Prepare",
+			Handler:    _JobService_Prepare_Handler,
+		},
+		{
+			MethodName: "Run",
+			Handler:    _JobService_Run_Handler,
+		},
+		{
+			MethodName: "GetState",
+			Handler:    _JobService_GetState_Handler,
+		},
+		{
+			MethodName: "Cancel",
+			Handler:    _JobService_Cancel_Handler,
+		},
+	},
+	Streams: []grpc.StreamDesc{
+		{
+			StreamName:    "GetStateStream",
+			Handler:       _JobService_GetStateStream_Handler,
+			ServerStreams: true,
+		},
+		{
+			StreamName:    "GetMessageStream",
+			Handler:       _JobService_GetMessageStream_Handler,
+			ServerStreams: true,
+		},
+	},
+	Metadata: "beam_job_api.proto",
+}
+
+func init() { proto.RegisterFile("beam_job_api.proto", fileDescriptor0) }
+
+var fileDescriptor0 = []byte{
+	// 931 bytes of a gzipped FileDescriptorProto
+	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x56, 0x41, 0x6f, 0xe3, 0x44,
+	0x14, 0xae, 0xdb, 0x34, 0x4d, 0x5e, 0x9b, 0xd4, 0x9d, 0x52, 0x35, 0x1b, 0x01, 0x5a, 0x8c, 0x60,
+	0x17, 0xad, 0xe4, 0xdd, 0x76, 0x25, 0x56, 0xec, 0x72, 0x71, 0x62, 0x6f, 0xd6, 0x51, 0x9b, 0x44,
+	0xe3, 0x54, 0x48, 0x70, 0x30, 0xe3, 0x64, 0x36, 0xb8, 0xd4, 0x1e, 0x63, 0x4f, 0xa2, 0xbd, 0x21,
+	0x21, 0x71, 0x44, 0xfc, 0x01, 0xfe, 0x00, 0x27, 0x38, 0x70, 0xe3, 0x1f, 0xf1, 0x17, 0xb8, 0xa0,
+	0x19, 0x8f, 0xdb, 0xa4, 0xed, 0xaa, 0x69, 0x11, 0xe2, 0x94, 0x99, 0xf7, 0xde, 0xf7, 0xcd, 0x37,
+	0xef, 0x3d, 0xbf, 0x09, 0xa0, 0x80, 0x92, 0xc8, 0x3f, 0x65, 0x81, 0x4f, 0x92, 0xd0, 0x4c, 0x52,
+	0xc6, 0x19, 0x7a, 0xc0, 0xd2, 0x89, 0x49, 0x12, 0x32, 0xfa, 0x86, 0x9a, 0xc2, 0x6d, 0x46, 0x6c,
+	0x4c, 0xcf, 0x4c, 0x11, 0x14, 0x91, 0x98, 0x4c, 0x68, 0x44, 0x63, 0x6e, 0xce, 0x0e, 0x9a, 0x7b,
+	0x12, 0x9c, 0x4e, 0xe3, 0x98, 0xa6, 0x17, 0xf8, 0xe6, 0x36, 0x8d, 0xc7, 0x09, 0x0b, 0x63, 0x9e,
+	0x29, 0xc3, 0xbb, 0x13, 0xc6, 0x26, 0x67, 0xf4, 0xb1, 0xdc, 0x05, 0xd3, 0xd7, 0x8f, 0x33, 0x9e,
+	0x4e, 0x47, 0x3c, 0xf7, 0x1a, 0x7f, 0x6a, 0xb0, 0x33, 0x48, 0x69, 0x42, 0x52, 0xda, 0x65, 0x01,
+	0xa6, 0xdf, 0x4d, 0x69, 0xc6, 0x51, 0x07, 0x2a, 0x49, 0x98, 0xd0, 0xb3, 0x30, 0xa6, 0x0d, 0xed,
+	0xbe, 0xf6, 0x70, 0xf3, 0xf0, 0x91, 0x79, 0xbd, 0xae, 0x22, 0xcc, 0x9c, 0x1d, 0x98, 0x03, 0xb5,
+	0xc6, 0xe7, 0x60, 0xd4, 0x02, 0xbd, 0x58, 0xfb, 0x2c, 0xe1, 0x21, 0x8b, 0xb3, 0xc6, 0xaa, 0x24,
+	0xdc, 0x37, 0x73, 0x5d, 0x66, 0xa1, 0xcb, 0xf4, 0xa4, 0x2e, 0xbc, 0x5d, 0x00, 0xfa, 0x79, 0x3c,
+	0xba, 0x07, 0x15, 0x71, 0xfb, 0x98, 0x44, 0xb4, 0xb1, 0x76, 0x5f, 0x7b, 0x58, 0xc5, 0x1b, 0xa7,
+	0x2c, 0xe8, 0x91, 0x88, 0x1a, 0xbf, 0x6b, 0x80, 0xe6, 0xd5, 0x67, 0x09, 0x8b, 0x33, 0x8a, 0x3e,
+	0x82, 0x7a, 0x22, 0xad, 0x44, 0x30, 0xf8, 0xe1, 0x58, 0x5e, 0xa2, 0x8a, 0x6b, 0x73, 0x56, 0x77,
+	0x8c, 0x32, 0xb8, 0x47, 0x52, 0x1e, 0xbe, 0x26, 0x23, 0xee, 0x67, 0x9c, 0x4c, 0xc2, 0x78, 0xe2,
+	0x17, 0xd9, 0x53, 0x2a, 0x9f, 0x2d, 0x71, 0x6d, 0x2b, 0x09, 0x3d, 0x9a, 0xce, 0xc2, 0x11, 0xb5,
+	0x69, 0x36, 0x4a, 0xc3, 0x84, 0xb3, 0x14, 0xef, 0x17, 0xcc, 0x5e, 0x4e, 0xec, 0x28, 0x5e, 0xe3,
+	0x2b, 0xa8, 0xe1, 0x69, 0x3c, 0x97, 0xeb, 0x25, 0xc5, 0x7e, 0x08, 0xb5, 0x42, 0x23, 0x67, 0xdf,
+	0xd2, 0x58, 0x0a, 0xac, 0xe2, 0x2d, 0x65, 0x1c, 0x0a, 0x9b, 0xf1, 0x00, 0xea, 0x05, 0xb9, 0x4a,
+	0xc5, 0x1e, 0x94, 0x45, 0xf2, 0xce, 0x59, 0xd7, 0x4f, 0x59, 0xe0, 0x8e, 0x8d, 0x4f, 0x40, 0x6f,
+	0x93, 0x78, 0x44, 0xcf, 0xe6, 0x84, 0xbc, 0x25, 0x94, 0xc0, 0xce, 0x5c, 0xa8, 0xa2, 0x3d, 0x82,
+	0xf5, 0x8c, 0x13, 0x9e, 0x77, 0x47, 0xfd, 0xf0, 0x53, 0x73, 0xc9, 0xae, 0x35, 0xbb, 0x2c, 0xf0,
+	0x04, 0xd0, 0x74, 0xe2, 0x69, 0x84, 0x73, 0x12, 0xe3, 0x11, 0xa0, 0x0e, 0xe5, 0x85, 0xeb, 0x06,
+	0x3d, 0x23, 0xd8, 0x5d, 0x08, 0xfe, 0xaf, 0x14, 0x75, 0x59, 0x70, 0x4c, 0xb3, 0x8c, 0x4c, 0x68,
+	0x76, 0x83, 0xa2, 0xbf, 0x57, 0x01, 0x2e, 0xa2, 0xd1, 0x7b, 0x00, 0x51, 0xbe, 0xbc, 0x88, 0xac,
+	0x2a, 0x8b, 0x3b, 0x46, 0x08, 0x4a, 0x3c, 0x8c, 0xa8, 0xaa, 0x9f, 0x5c, 0x23, 0x0a, 0x10, 0x46,
+	0x09, 0x4b, 0xb9, 0x48, 0xb4, 0x6c, 0xf2, 0xfa, 0xa1, 0x73, 0x9b, 0x1b, 0xa8, 0xb3, 0x4d, 0xf5,
+	0xeb, 0x9e, 0x93, 0xe1, 0x39, 0x62, 0xf4, 0x01, 0x6c, 0x15, 0xca, 0x38, 0x7d, 0xc3, 0x1b, 0x25,
+	0x29, 0x61, 0x53, 0xd9, 0x86, 0xf4, 0x0d, 0x37, 0x7e, 0xd3, 0x60, 0xe7, 0x0a, 0x09, 0x32, 0xe0,
+	0xfd, 0x63, 0xc7, 0xf3, 0xac, 0x8e, 0xe3, 0xbb, 0xc7, 0x83, 0x3e, 0x1e, 0x5a, 0xbd, 0xb6, 0xe3,
+	0x9f, 0xf4, 0xbc, 0x81, 0xd3, 0x76, 0x5f, 0xba, 0x8e, 0xad, 0xaf, 0xa0, 0x3d, 0xd8, 0xe9, 0xf6,
+	0x5b, 0x7e, 0x11, 0x67, 0x3b, 0xad, 0x93, 0x8e, 0xae, 0xa1, 0x06, 0xbc, 0xb3, 0x68, 0x1e, 0x5a,
+	0xee, 0x91, 0x63, 0xeb, 0xab, 0x97, 0x01, 0x2d, 0xcb, 0x73, 0xdb, 0xfa, 0x1a, 0xda, 0x87, 0xdd,
+	0x79, 0xf3, 0x17, 0x16, 0xee, 0xb9, 0xbd, 0x8e, 0x5e, 0xba, 0x1c, 0xef, 0x60, 0xdc, 0xc7, 0xfa,
+	0xba, 0xf1, 0x97, 0x06, 0xbb, 0x0b, 0xb5, 0x52, 0x0d, 0xf1, 0x35, 0xe8, 0xc5, 0x65, 0x53, 0x65,
+	0x53, 0xb3, 0xec, 0xe9, 0x1d, 0x32, 0xfb, 0x6a, 0x05, 0x6f, 0x2b, 0xba, 0xf3, 0x13, 0x28, 0xd4,
+	0x65, 0xb7, 0x5c, 0xf0, 0xe7, 0x43, 0xe3, 0xf3, 0xa5, 0xf9, 0xaf, 0x69, 0xe4, 0x57, 0x2b, 0xb8,
+	0x96, 0xcd, 0x1b, 0x5a, 0x00, 0x95, 0xe2, 0x00, 0xe3, 0x57, 0x0d, 0x2a, 0x05, 0xc2, 0xf8, 0x45,
+	0x83, 0x92, 0x68, 0x5a, 0xb4, 0x0d, 0x9b, 0x8b, 0xb5, 0xd8, 0x84, 0x0d, 0x6f, 0xd8, 0x1f, 0x0c,
+	0x1c, 0x5b, 0xd7, 0xc4, 0x06, 0x9f, 0xf4, 0x64, 0x12, 0x57, 0x51, 0x05, 0x4a, 0x76, 0xbf, 0xe7,
+	0xe8, 0x6b, 0x08, 0xa0, 0xfc, 0x32, 0x2f, 0x45, 0x09, 0xd5, 0xa0, 0xda, 0x16, 0x25, 0x3d, 0x12,
+	0xdb, 0x75, 0x81, 0x38, 0x19, 0xd8, 0xd6, 0xd0, 0xb1, 0xf5, 0x32, 0xda, 0x82, 0x8a, 0x8d, 0x2d,
+	0x57, 0xe2, 0x37, 0x84, 0x4b, 0xee, 0x1c, 0x5b, 0xaf, 0x08, 0x97, 0x37, 0xb4, 0xf0, 0x50, 0xb8,
+	0xaa, 0xa8, 0x0e, 0xa0, 0x48, 0xc4, 0x1e, 0x0e, 0xff, 0x28, 0xcb, 0xcf, 0x42, 0xcd, 0x46, 0xf4,
+	0x83, 0x06, 0x1b, 0x6a, 0x56, 0xa3, 0xe7, 0x4b, 0x67, 0xe8, 0xca, 0xdb, 0xd4, 0x7c, 0x71, 0x27,
+	0xac, 0x2a, 0xd9, 0x0c, 0xd6, 0xf0, 0x34, 0x46, 0xcb, 0x4f, 0x87, 0x85, 0x59, 0xdd, 0x7c, 0x76,
+	0x6b, 0x9c, 0x3a, 0xf7, 0x47, 0x0d, 0x2a, 0x1d, 0xca, 0x65, 0xdd, 0xd0, 0x8b, 0xbb, 0xf5, 0x47,
+	0x2e, 0xe1, 0x5f, 0x35, 0x17, 0xfa, 0x1e, 0xca, 0xf9, 0x30, 0x47, 0x9f, 0x2d, 0xcd, 0x73, 0xf9,
+	0xa1, 0x68, 0x3e, 0xbf, 0x0b, 0x54, 0x09, 0xf8, 0x49, 0x83, 0x7a, 0x91, 0x08, 0x8f, 0xa7, 0x94,
+	0x44, 0xff, 0x63, 0x3a, 0x9e, 0x68, 0xe8, 0x67, 0x0d, 0xf4, 0x0e, 0xe5, 0xea, 0x2b, 0xbf, 0xb5,
+	0xa2, 0xab, 0x8f, 0xc4, 0x2d, 0x14, 0x5d, 0x33, 0xb5, 0x9e, 0x68, 0xad, 0x16, 0x7c, 0xfc, 0x56,
+	0x82, 0x05, 0x7c, 0xab, 0xdc, 0x65, 0x81, 0x95, 0x84, 0x5f, 0xea, 0x0b, 0x1e, 0x7f, 0x76, 0x10,
+	0x94, 0xe5, 0x9f, 0xaa, 0xa7, 0xff, 0x04, 0x00, 0x00, 0xff, 0xff, 0x20, 0x71, 0x77, 0xfc, 0x61,
+	0x0a, 0x00, 0x00,
+}


[12/50] [abbrv] beam git commit: [BEAM-2203] Implement TIMESTAMPADD

Posted by ke...@apache.org.
[BEAM-2203] Implement TIMESTAMPADD

Add support for TIMESTAMPADD(interval, multiplier, TIMESTAMP)

fixup! [BEAM-2203] Implement TIMESTAMPADD


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

Branch: refs/heads/mr-runner
Commit: 820f8aff944d1eda69c9226086848d8b39fcf62f
Parents: a33c717
Author: Anton Kedin <ke...@kedin-macbookpro.roam.corp.google.com>
Authored: Fri Oct 27 10:15:08 2017 -0700
Committer: mingmxu <mi...@ebay.com>
Committed: Wed Nov 1 16:29:07 2017 -0700

----------------------------------------------------------------------
 .../sql/impl/interpreter/BeamSqlFnExecutor.java |  19 ++-
 .../interpreter/operator/BeamSqlPrimitive.java  |   8 +-
 .../date/BeamSqlDatetimePlusExpression.java     | 129 +++++++++++++++
 .../date/BeamSqlIntervalMultiplyExpression.java | 103 ++++++++++++
 .../operator/date/TimeUnitUtils.java            |  54 +++++++
 .../extensions/sql/impl/utils/SqlTypeUtils.java |  46 ++++++
 .../impl/interpreter/BeamSqlFnExecutorTest.java |  30 ++++
 .../date/BeamSqlDateExpressionTestBase.java     |   5 +-
 .../date/BeamSqlDatetimePlusExpressionTest.java | 155 +++++++++++++++++++
 .../BeamSqlIntervalMultiplyExpressionTest.java  | 107 +++++++++++++
 .../operator/date/TimeUnitUtilsTest.java        |  54 +++++++
 .../sql/impl/utils/SqlTypeUtilsTest.java        |  76 +++++++++
 .../BeamSqlDateFunctionsIntegrationTest.java    |  39 ++++-
 13 files changed, 818 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/820f8aff/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/BeamSqlFnExecutor.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/BeamSqlFnExecutor.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/BeamSqlFnExecutor.java
index 8f9797b..8770055 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/BeamSqlFnExecutor.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/BeamSqlFnExecutor.java
@@ -21,6 +21,7 @@ import java.math.BigDecimal;
 import java.util.ArrayList;
 import java.util.Calendar;
 import java.util.List;
+
 import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlCaseExpression;
 import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlCastExpression;
 import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
@@ -49,7 +50,9 @@ import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.date.BeamSql
 import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.date.BeamSqlCurrentTimestampExpression;
 import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.date.BeamSqlDateCeilExpression;
 import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.date.BeamSqlDateFloorExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.date.BeamSqlDatetimePlusExpression;
 import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.date.BeamSqlExtractExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.date.BeamSqlIntervalMultiplyExpression;
 import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.logical.BeamSqlAndExpression;
 import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.logical.BeamSqlNotExpression;
 import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.logical.BeamSqlOrExpression;
@@ -143,7 +146,7 @@ public class BeamSqlFnExecutor implements BeamSqlExpressionExecutor {
         // NlsString is not serializable, we need to convert
         // it to string explicitly.
         return BeamSqlPrimitive.of(type, ((NlsString) value).getValue());
-      } else if (type == SqlTypeName.DATE && value instanceof Calendar) {
+      } else if (isDateNode(type, value)) {
         // does this actually make sense?
         // Calcite actually treat Calendar as the java type of Date Literal
         return BeamSqlPrimitive.of(type, ((Calendar) value).getTime());
@@ -235,7 +238,11 @@ public class BeamSqlFnExecutor implements BeamSqlExpressionExecutor {
           ret = new BeamSqlMinusExpression(subExps);
           break;
         case "*":
-          ret = new BeamSqlMultiplyExpression(subExps);
+          if (SqlTypeName.NUMERIC_TYPES.contains(node.type.getSqlTypeName())) {
+            ret = new BeamSqlMultiplyExpression(subExps);
+          } else {
+            ret = new BeamSqlIntervalMultiplyExpression(subExps);
+          }
           break;
         case "/":
         case "/INT":
@@ -369,6 +376,9 @@ public class BeamSqlFnExecutor implements BeamSqlExpressionExecutor {
         case "CURRENT_DATE":
           return new BeamSqlCurrentDateExpression();
 
+        case "DATETIME_PLUS":
+          return new BeamSqlDatetimePlusExpression(subExps);
+
 
         case "CASE":
           ret = new BeamSqlCaseExpression(subExps);
@@ -423,6 +433,11 @@ public class BeamSqlFnExecutor implements BeamSqlExpressionExecutor {
     return ret;
   }
 
+  private static boolean isDateNode(SqlTypeName type, Object value) {
+    return (type == SqlTypeName.DATE || type == SqlTypeName.TIMESTAMP)
+        && value instanceof Calendar;
+  }
+
   @Override
   public void prepare() {
   }

http://git-wip-us.apache.org/repos/asf/beam/blob/820f8aff/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlPrimitive.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlPrimitive.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlPrimitive.java
index 9175caa..21cbc80 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlPrimitive.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlPrimitive.java
@@ -21,6 +21,7 @@ import java.math.BigDecimal;
 import java.util.Date;
 import java.util.GregorianCalendar;
 import java.util.List;
+
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.values.BeamRecord;
 import org.apache.calcite.sql.type.SqlTypeName;
@@ -133,9 +134,12 @@ public class BeamSqlPrimitive<T> extends BeamSqlExpression {
     case TIMESTAMP:
     case DATE:
       return value instanceof Date;
-    case INTERVAL_HOUR:
-      return value instanceof BigDecimal;
+    case INTERVAL_SECOND:
     case INTERVAL_MINUTE:
+    case INTERVAL_HOUR:
+    case INTERVAL_DAY:
+    case INTERVAL_MONTH:
+    case INTERVAL_YEAR:
       return value instanceof BigDecimal;
     case SYMBOL:
       // for SYMBOL, it supports anything...

http://git-wip-us.apache.org/repos/asf/beam/blob/820f8aff/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlDatetimePlusExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlDatetimePlusExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlDatetimePlusExpression.java
new file mode 100644
index 0000000..426cda0
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlDatetimePlusExpression.java
@@ -0,0 +1,129 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.date;
+
+import static org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.date.TimeUnitUtils.timeUnitInternalMultiplier;
+import static org.apache.beam.sdk.extensions.sql.impl.utils.SqlTypeUtils.findExpressionOfType;
+
+import com.google.common.collect.ImmutableSet;
+
+import java.math.BigDecimal;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.values.BeamRecord;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.joda.time.DateTime;
+
+/**
+ * DATETIME_PLUS operation.
+ * Calcite converts 'TIMESTAMPADD(..)' or 'DATE + INTERVAL' from the user input
+ * into DATETIME_PLUS.
+ *
+ * <p>Input and output are expected to be of type TIMESTAMP.
+ */
+public class BeamSqlDatetimePlusExpression extends BeamSqlExpression {
+
+  private static final Set<SqlTypeName> SUPPORTED_INTERVAL_TYPES = ImmutableSet.of(
+      SqlTypeName.INTERVAL_SECOND,
+      SqlTypeName.INTERVAL_MINUTE,
+      SqlTypeName.INTERVAL_HOUR,
+      SqlTypeName.INTERVAL_DAY,
+      SqlTypeName.INTERVAL_MONTH,
+      SqlTypeName.INTERVAL_YEAR);
+
+  public BeamSqlDatetimePlusExpression(List<BeamSqlExpression> operands) {
+    super(operands, SqlTypeName.TIMESTAMP);
+  }
+
+  /**
+   * Requires exactly 2 operands. One should be a timestamp, another an interval
+   */
+  @Override
+  public boolean accept() {
+    return operands.size() == 2
+        && SqlTypeName.TIMESTAMP.equals(operands.get(0).getOutputType())
+        && SUPPORTED_INTERVAL_TYPES.contains(operands.get(1).getOutputType());
+  }
+
+  /**
+   * Adds interval to the timestamp.
+   *
+   * <p>Interval has a value of 'multiplier * TimeUnit.multiplier'.
+   *
+   * <p>For example, '3 years' is going to have a type of INTERVAL_YEAR, and a value of 36.
+   * And '2 minutes' is going to be an INTERVAL_MINUTE with a value of 120000. This is the way
+   * Calcite handles interval expressions, and {@link BeamSqlIntervalMultiplyExpression} also works
+   * the same way.
+   */
+  @Override
+  public BeamSqlPrimitive evaluate(BeamRecord inputRow, BoundedWindow window) {
+    DateTime timestamp = getTimestampOperand(inputRow, window);
+    BeamSqlPrimitive intervalOperandPrimitive = getIntervalOperand(inputRow, window);
+    SqlTypeName intervalOperandType = intervalOperandPrimitive.getOutputType();
+    int intervalMultiplier = getIntervalMultiplier(intervalOperandPrimitive);
+
+    DateTime newDate = addInterval(timestamp, intervalOperandType, intervalMultiplier);
+    return BeamSqlPrimitive.of(SqlTypeName.TIMESTAMP, newDate.toDate());
+  }
+
+  private int getIntervalMultiplier(BeamSqlPrimitive intervalOperandPrimitive) {
+    BigDecimal intervalOperandValue = intervalOperandPrimitive.getDecimal();
+    BigDecimal multiplier = intervalOperandValue.divide(
+        timeUnitInternalMultiplier(intervalOperandPrimitive.getOutputType()),
+        BigDecimal.ROUND_CEILING);
+    return multiplier.intValueExact();
+  }
+
+  private BeamSqlPrimitive getIntervalOperand(BeamRecord inputRow, BoundedWindow window) {
+    return findExpressionOfType(operands, SUPPORTED_INTERVAL_TYPES).get()
+        .evaluate(inputRow, window);
+  }
+
+  private DateTime getTimestampOperand(BeamRecord inputRow, BoundedWindow window) {
+    BeamSqlPrimitive timestampOperandPrimitive =
+        findExpressionOfType(operands, SqlTypeName.TIMESTAMP).get().evaluate(inputRow, window);
+    return new DateTime(timestampOperandPrimitive.getDate());
+  }
+
+  private DateTime addInterval(
+      DateTime dateTime, SqlTypeName intervalType, int numberOfIntervals) {
+
+    switch (intervalType) {
+      case INTERVAL_SECOND:
+        return dateTime.plusSeconds(numberOfIntervals);
+      case INTERVAL_MINUTE:
+        return dateTime.plusMinutes(numberOfIntervals);
+      case INTERVAL_HOUR:
+        return dateTime.plusHours(numberOfIntervals);
+      case INTERVAL_DAY:
+        return dateTime.plusDays(numberOfIntervals);
+      case INTERVAL_MONTH:
+        return dateTime.plusMonths(numberOfIntervals);
+      case INTERVAL_YEAR:
+        return dateTime.plusYears(numberOfIntervals);
+      default:
+        throw new IllegalArgumentException("Adding "
+            + intervalType.getName() + " to date is not supported");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/820f8aff/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlIntervalMultiplyExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlIntervalMultiplyExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlIntervalMultiplyExpression.java
new file mode 100644
index 0000000..f4ddf71
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlIntervalMultiplyExpression.java
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.date;
+
+import static org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.date.TimeUnitUtils.timeUnitInternalMultiplier;
+import static org.apache.beam.sdk.extensions.sql.impl.utils.SqlTypeUtils.findExpressionOfType;
+
+import com.google.common.base.Optional;
+
+import java.math.BigDecimal;
+import java.util.List;
+
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.values.BeamRecord;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * Multiplication operator for intervals.
+ * For example, allows to express things like '3 years'.
+ *
+ * <p>One use case of this is implementation of TIMESTAMPADD().
+ * Calcite converts TIMESTAMPADD(date, multiplier, inteval) into
+ * DATETIME_PLUS(date, multiplier * interval).
+ * The 'multiplier * interval' part is what this class implements. It's not a regular
+ * numerical multiplication because the return type is expected to be an interval, and the value
+ * is expected to use corresponding TimeUnit's internal value (e.g. 12 for YEAR, 60000 for MINUTE).
+ */
+public class BeamSqlIntervalMultiplyExpression extends BeamSqlExpression {
+  public BeamSqlIntervalMultiplyExpression(List<BeamSqlExpression> operands) {
+    super(operands, deduceOutputType(operands));
+  }
+
+  /**
+   * Output type is null if no operands found with matching types.
+   * Execution will later fail when calling accept()
+   */
+  private static SqlTypeName deduceOutputType(List<BeamSqlExpression> operands) {
+    Optional<BeamSqlExpression> intervalOperand =
+        findExpressionOfType(operands, SqlTypeName.INTERVAL_TYPES);
+
+    return intervalOperand.isPresent()
+        ? intervalOperand.get().getOutputType()
+        : null;
+  }
+
+  /**
+   * Requires exactly 2 operands. One should be integer, another should be interval
+   */
+  @Override
+  public boolean accept() {
+    return operands.size() == 2
+        && findExpressionOfType(operands, SqlTypeName.INTEGER).isPresent()
+        && findExpressionOfType(operands, SqlTypeName.INTERVAL_TYPES).isPresent();
+  }
+  /**
+   * Evaluates the number of times the interval should be repeated, times the TimeUnit multiplier.
+   * For example for '3 * MONTH' this will return an object with type INTERVAL_MONTH and value 36.
+   *
+   * <p>This is due to the fact that TimeUnit has different internal multipliers for each interval,
+   * e.g. MONTH is 12, but MINUTE is 60000. When Calcite parses SQL interval literals, it returns
+   * those internal multipliers. This means we need to do similar thing, so that this multiplication
+   * expression behaves the same way as literal interval expression.
+   *
+   * <p>That is, we need to make sure that this:
+   *   "TIMESTAMP '1984-04-19 01:02:03' + INTERVAL '2' YEAR"
+   * is equivalent tot this:
+   *   "TIMESTAMPADD(YEAR, 2, TIMESTAMP '1984-04-19 01:02:03')"
+   */
+  @Override
+  public BeamSqlPrimitive evaluate(BeamRecord inputRow, BoundedWindow window) {
+    BeamSqlPrimitive intervalOperandPrimitive =
+        findExpressionOfType(operands, SqlTypeName.INTERVAL_TYPES).get().evaluate(inputRow, window);
+    SqlTypeName intervalOperandType = intervalOperandPrimitive.getOutputType();
+
+    BeamSqlPrimitive integerOperandPrimitive =
+        findExpressionOfType(operands, SqlTypeName.INTEGER).get().evaluate(inputRow, window);
+    BigDecimal integerOperandValue = new BigDecimal(integerOperandPrimitive.getInteger());
+
+    BigDecimal multiplicationResult =
+        integerOperandValue.multiply(
+            timeUnitInternalMultiplier(intervalOperandType));
+
+    return BeamSqlPrimitive.of(outputType, multiplicationResult);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/820f8aff/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/TimeUnitUtils.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/TimeUnitUtils.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/TimeUnitUtils.java
new file mode 100644
index 0000000..b432d20
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/TimeUnitUtils.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.date;
+
+import java.math.BigDecimal;
+
+import org.apache.calcite.avatica.util.TimeUnit;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * Utils to convert between Calcite's TimeUnit and Sql intervals.
+ */
+public abstract class TimeUnitUtils {
+
+  /**
+   * @return internal multiplier of a TimeUnit, e.g. YEAR is 12, MINUTE is 60000
+   * @throws IllegalArgumentException if interval type is not supported
+   */
+  public static BigDecimal timeUnitInternalMultiplier(final SqlTypeName sqlIntervalType) {
+    switch (sqlIntervalType) {
+      case INTERVAL_SECOND:
+        return TimeUnit.SECOND.multiplier;
+      case INTERVAL_MINUTE:
+        return TimeUnit.MINUTE.multiplier;
+      case INTERVAL_HOUR:
+        return TimeUnit.HOUR.multiplier;
+      case INTERVAL_DAY:
+        return TimeUnit.DAY.multiplier;
+      case INTERVAL_MONTH:
+        return TimeUnit.MONTH.multiplier;
+      case INTERVAL_YEAR:
+        return TimeUnit.YEAR.multiplier;
+      default:
+        throw new IllegalArgumentException("Interval " + sqlIntervalType
+            + " cannot be converted to TimeUnit");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/820f8aff/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/SqlTypeUtils.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/SqlTypeUtils.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/SqlTypeUtils.java
new file mode 100644
index 0000000..1ab703e
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/SqlTypeUtils.java
@@ -0,0 +1,46 @@
+package org.apache.beam.sdk.extensions.sql.impl.utils;
+
+import com.google.common.base.Optional;
+
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * Utils to help with SqlTypes.
+ */
+public class SqlTypeUtils {
+  /**
+   * Finds an operand with provided type.
+   * Returns Optional.absent() if no operand found with matching type
+   */
+  public static Optional<BeamSqlExpression> findExpressionOfType(
+      List<BeamSqlExpression> operands, SqlTypeName type) {
+
+    for (BeamSqlExpression operand : operands) {
+      if (type.equals(operand.getOutputType())) {
+        return Optional.of(operand);
+      }
+    }
+
+    return Optional.absent();
+  }
+
+  /**
+   * Finds an operand with the type in typesToFind.
+   * Returns Optional.absent() if no operand found with matching type
+   */
+  public static Optional<BeamSqlExpression> findExpressionOfType(
+      List<BeamSqlExpression> operands, Collection<SqlTypeName> typesToFind) {
+
+    for (BeamSqlExpression operand : operands) {
+      if (typesToFind.contains(operand.getOutputType())) {
+        return Optional.of(operand);
+      }
+    }
+
+    return Optional.absent();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/820f8aff/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/BeamSqlFnExecutorTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/BeamSqlFnExecutorTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/BeamSqlFnExecutorTest.java
index f350087..c4583ec 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/BeamSqlFnExecutorTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/BeamSqlFnExecutorTest.java
@@ -24,6 +24,7 @@ import java.util.Arrays;
 import java.util.Calendar;
 import java.util.Date;
 import java.util.TimeZone;
+
 import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlCaseExpression;
 import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
 import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlInputRefExpression;
@@ -40,7 +41,9 @@ import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.date.BeamSql
 import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.date.BeamSqlCurrentTimestampExpression;
 import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.date.BeamSqlDateCeilExpression;
 import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.date.BeamSqlDateFloorExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.date.BeamSqlDatetimePlusExpression;
 import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.date.BeamSqlExtractExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.date.BeamSqlIntervalMultiplyExpression;
 import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.logical.BeamSqlAndExpression;
 import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.logical.BeamSqlNotExpression;
 import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.logical.BeamSqlOrExpression;
@@ -57,12 +60,15 @@ import org.apache.beam.sdk.extensions.sql.impl.planner.BeamQueryPlanner;
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamFilterRel;
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamProjectRel;
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamRelNode;
+import org.apache.calcite.avatica.util.TimeUnit;
 import org.apache.calcite.avatica.util.TimeUnitRange;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.SqlIntervalQualifier;
 import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.fun.SqlTrimFunction;
+import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.junit.Assert;
 import org.junit.Test;
@@ -412,5 +418,29 @@ public class BeamSqlFnExecutorTest extends BeamSqlFnExecutorTestBase {
     );
     exp = BeamSqlFnExecutor.buildExpression(rexNode);
     assertTrue(exp instanceof BeamSqlCurrentTimestampExpression);
+
+    // DATETIME_PLUS
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.DATETIME_PLUS,
+        Arrays.<RexNode>asList(
+            rexBuilder.makeDateLiteral(calendar),
+            rexBuilder.makeIntervalLiteral(
+                new BigDecimal(10),
+                new SqlIntervalQualifier(TimeUnit.DAY, TimeUnit.DAY, SqlParserPos.ZERO))
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlDatetimePlusExpression);
+
+    // * for intervals
+    rexNode = rexBuilder.makeCall(SqlStdOperatorTable.MULTIPLY,
+        Arrays.<RexNode>asList(
+            rexBuilder.makeExactLiteral(new BigDecimal(1)),
+            rexBuilder.makeIntervalLiteral(
+                new BigDecimal(10),
+                new SqlIntervalQualifier(TimeUnit.DAY, TimeUnit.DAY, SqlParserPos.ZERO))
+        )
+    );
+    exp = BeamSqlFnExecutor.buildExpression(rexNode);
+    assertTrue(exp instanceof BeamSqlIntervalMultiplyExpression);
   }
 }

http://git-wip-us.apache.org/repos/asf/beam/blob/820f8aff/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlDateExpressionTestBase.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlDateExpressionTestBase.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlDateExpressionTestBase.java
index 0e57404..cb0b6ec 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlDateExpressionTestBase.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlDateExpressionTestBase.java
@@ -22,13 +22,14 @@ import java.text.ParseException;
 import java.text.SimpleDateFormat;
 import java.util.Date;
 import java.util.TimeZone;
+
 import org.apache.beam.sdk.extensions.sql.impl.interpreter.BeamSqlFnExecutorTestBase;
 
 /**
  * Base class for all date related expression test.
  */
 public class BeamSqlDateExpressionTestBase extends BeamSqlFnExecutorTestBase {
-  protected long str2LongTime(String dateStr) {
+  static long str2LongTime(String dateStr) {
     SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
     try {
       Date date = format.parse(dateStr);
@@ -38,7 +39,7 @@ public class BeamSqlDateExpressionTestBase extends BeamSqlFnExecutorTestBase {
     }
   }
 
-  protected Date str2DateTime(String dateStr) {
+  static Date str2DateTime(String dateStr) {
     SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
     try {
       format.setTimeZone(TimeZone.getTimeZone("GMT"));

http://git-wip-us.apache.org/repos/asf/beam/blob/820f8aff/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlDatetimePlusExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlDatetimePlusExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlDatetimePlusExpressionTest.java
new file mode 100644
index 0000000..57e709f
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlDatetimePlusExpressionTest.java
@@ -0,0 +1,155 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.date;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.math.BigDecimal;
+import java.util.Arrays;
+import java.util.Date;
+
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.values.BeamRecord;
+import org.apache.calcite.avatica.util.TimeUnit;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.joda.time.DateTime;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+/**
+ * Test for {@link BeamSqlDatetimePlusExpression}.
+ */
+public class BeamSqlDatetimePlusExpressionTest extends BeamSqlDateExpressionTestBase {
+  @Rule public ExpectedException thrown = ExpectedException.none();
+
+  private static final BeamRecord NULL_INPUT_ROW = null;
+  private static final BoundedWindow NULL_WINDOW = null;
+  private static final Date DATE = str2DateTime("1984-04-19 01:02:03");
+
+  private static final Date DATE_PLUS_15_SECONDS = new DateTime(DATE).plusSeconds(15).toDate();
+  private static final Date DATE_PLUS_10_MINUTES = new DateTime(DATE).plusMinutes(10).toDate();
+  private static final Date DATE_PLUS_7_HOURS = new DateTime(DATE).plusHours(7).toDate();
+  private static final Date DATE_PLUS_3_DAYS = new DateTime(DATE).plusDays(3).toDate();
+  private static final Date DATE_PLUS_2_MONTHS = new DateTime(DATE).plusMonths(2).toDate();
+  private static final Date DATE_PLUS_11_YEARS = new DateTime(DATE).plusYears(11).toDate();
+
+  private static final BeamSqlExpression SQL_INTERVAL_15_SECONDS =
+      interval(SqlTypeName.INTERVAL_SECOND, 15);
+  private static final BeamSqlExpression SQL_INTERVAL_10_MINUTES =
+      interval(SqlTypeName.INTERVAL_MINUTE, 10);
+  private static final BeamSqlExpression SQL_INTERVAL_7_HOURS =
+      interval(SqlTypeName.INTERVAL_HOUR, 7);
+  private static final BeamSqlExpression SQL_INTERVAL_3_DAYS =
+      interval(SqlTypeName.INTERVAL_DAY, 3);
+  private static final BeamSqlExpression SQL_INTERVAL_2_MONTHS =
+      interval(SqlTypeName.INTERVAL_MONTH, 2);
+  private static final BeamSqlExpression SQL_INTERVAL_4_MONTHS =
+      interval(SqlTypeName.INTERVAL_MONTH, 4);
+  private static final BeamSqlExpression SQL_INTERVAL_11_YEARS =
+      interval(SqlTypeName.INTERVAL_YEAR, 11);
+
+  private static final BeamSqlExpression SQL_TIMESTAMP =
+      BeamSqlPrimitive.of(SqlTypeName.TIMESTAMP, DATE);
+
+  @Test public void testHappyPath_outputTypeAndAccept() {
+    BeamSqlExpression plusExpression = dateTimePlus(SQL_TIMESTAMP, SQL_INTERVAL_3_DAYS);
+
+    assertEquals(SqlTypeName.TIMESTAMP, plusExpression.getOutputType());
+    assertTrue(plusExpression.accept());
+  }
+
+  @Test public void testDoesNotAcceptTreeOperands() {
+    BeamSqlDatetimePlusExpression plusExpression =
+        dateTimePlus(SQL_TIMESTAMP, SQL_INTERVAL_3_DAYS, SQL_INTERVAL_4_MONTHS);
+
+    assertEquals(SqlTypeName.TIMESTAMP, plusExpression.getOutputType());
+    assertFalse(plusExpression.accept());
+  }
+
+  @Test public void testDoesNotAcceptWithoutTimestampOperand() {
+    BeamSqlDatetimePlusExpression plusExpression =
+        dateTimePlus(SQL_INTERVAL_3_DAYS, SQL_INTERVAL_4_MONTHS);
+
+    assertEquals(SqlTypeName.TIMESTAMP, plusExpression.getOutputType());
+    assertFalse(plusExpression.accept());
+  }
+
+  @Test public void testDoesNotAcceptWithoutIntervalOperand() {
+    BeamSqlDatetimePlusExpression plusExpression =
+        dateTimePlus(SQL_TIMESTAMP, SQL_TIMESTAMP);
+
+    assertEquals(SqlTypeName.TIMESTAMP, plusExpression.getOutputType());
+    assertFalse(plusExpression.accept());
+  }
+
+  @Test public void testEvaluate() {
+    assertEquals(DATE_PLUS_15_SECONDS, evalDatetimePlus(SQL_TIMESTAMP, SQL_INTERVAL_15_SECONDS));
+    assertEquals(DATE_PLUS_10_MINUTES, evalDatetimePlus(SQL_TIMESTAMP, SQL_INTERVAL_10_MINUTES));
+    assertEquals(DATE_PLUS_7_HOURS, evalDatetimePlus(SQL_TIMESTAMP, SQL_INTERVAL_7_HOURS));
+    assertEquals(DATE_PLUS_3_DAYS, evalDatetimePlus(SQL_TIMESTAMP, SQL_INTERVAL_3_DAYS));
+    assertEquals(DATE_PLUS_2_MONTHS, evalDatetimePlus(SQL_TIMESTAMP, SQL_INTERVAL_2_MONTHS));
+    assertEquals(DATE_PLUS_11_YEARS, evalDatetimePlus(SQL_TIMESTAMP, SQL_INTERVAL_11_YEARS));
+  }
+
+  @Test public void testEvaluateThrowsForUnsupportedIntervalType() {
+    thrown.expect(UnsupportedOperationException.class);
+
+    BeamSqlPrimitive unsupportedInterval = BeamSqlPrimitive.of(SqlTypeName.INTERVAL_YEAR_MONTH, 3);
+    evalDatetimePlus(SQL_TIMESTAMP, unsupportedInterval);
+  }
+
+  private static Date evalDatetimePlus(BeamSqlExpression date, BeamSqlExpression interval) {
+    return dateTimePlus(date, interval).evaluate(NULL_INPUT_ROW, NULL_WINDOW).getDate();
+  }
+
+  private static BeamSqlDatetimePlusExpression dateTimePlus(BeamSqlExpression ... operands) {
+    return new BeamSqlDatetimePlusExpression(Arrays.asList(operands));
+  }
+
+  private static BeamSqlExpression interval(SqlTypeName type, int multiplier) {
+    return BeamSqlPrimitive.of(type,
+        timeUnitInternalMultiplier(type)
+            .multiply(new BigDecimal(multiplier)));
+  }
+
+  private static BigDecimal timeUnitInternalMultiplier(final SqlTypeName sqlIntervalType) {
+    switch (sqlIntervalType) {
+      case INTERVAL_SECOND:
+        return TimeUnit.SECOND.multiplier;
+      case INTERVAL_MINUTE:
+        return TimeUnit.MINUTE.multiplier;
+      case INTERVAL_HOUR:
+        return TimeUnit.HOUR.multiplier;
+      case INTERVAL_DAY:
+        return TimeUnit.DAY.multiplier;
+      case INTERVAL_MONTH:
+        return TimeUnit.MONTH.multiplier;
+      case INTERVAL_YEAR:
+        return TimeUnit.YEAR.multiplier;
+      default:
+        throw new IllegalArgumentException("Interval " + sqlIntervalType
+            + " cannot be converted to TimeUnit");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/820f8aff/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlIntervalMultiplyExpressionTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlIntervalMultiplyExpressionTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlIntervalMultiplyExpressionTest.java
new file mode 100644
index 0000000..0c91f40
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlIntervalMultiplyExpressionTest.java
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.date;
+
+import static org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.date.TimeUnitUtils.timeUnitInternalMultiplier;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.math.BigDecimal;
+import java.util.Arrays;
+
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.values.BeamRecord;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Test;
+
+/**
+ * Test for BeamSqlIntervalMultiplyExpression.
+ */
+public class BeamSqlIntervalMultiplyExpressionTest {
+  private static final BeamRecord NULL_INPUT_ROW = null;
+  private static final BoundedWindow NULL_WINDOW = null;
+  private static final BigDecimal DECIMAL_THREE = new BigDecimal(3);
+  private static final BigDecimal DECIMAL_FOUR = new BigDecimal(4);
+
+  private static final BeamSqlExpression SQL_INTERVAL_DAY =
+      BeamSqlPrimitive.of(SqlTypeName.INTERVAL_DAY, DECIMAL_THREE);
+
+  private static final BeamSqlExpression SQL_INTERVAL_MONTH =
+      BeamSqlPrimitive.of(SqlTypeName.INTERVAL_MONTH, DECIMAL_FOUR);
+
+  private static final BeamSqlExpression SQL_INTEGER_FOUR =
+      BeamSqlPrimitive.of(SqlTypeName.INTEGER, 4);
+
+  private static final BeamSqlExpression SQL_INTEGER_FIVE =
+      BeamSqlPrimitive.of(SqlTypeName.INTEGER, 5);
+
+  @Test public void testHappyPath_outputTypeAndAccept() {
+    BeamSqlExpression multiplyExpression =
+        newMultiplyExpression(SQL_INTERVAL_DAY, SQL_INTEGER_FOUR);
+
+    assertEquals(SqlTypeName.INTERVAL_DAY, multiplyExpression.getOutputType());
+    assertTrue(multiplyExpression.accept());
+  }
+
+  @Test public void testDoesNotAcceptTreeOperands() {
+    BeamSqlIntervalMultiplyExpression multiplyExpression =
+        newMultiplyExpression(SQL_INTERVAL_DAY, SQL_INTEGER_FIVE, SQL_INTEGER_FOUR);
+
+    assertEquals(SqlTypeName.INTERVAL_DAY, multiplyExpression.getOutputType());
+    assertFalse(multiplyExpression.accept());
+  }
+
+  @Test public void testDoesNotAcceptWithoutIntervalOperand() {
+    BeamSqlIntervalMultiplyExpression multiplyExpression =
+        newMultiplyExpression(SQL_INTEGER_FOUR, SQL_INTEGER_FIVE);
+
+    assertNull(multiplyExpression.getOutputType());
+    assertFalse(multiplyExpression.accept());
+  }
+
+  @Test public void testDoesNotAcceptWithoutIntegerOperand() {
+    BeamSqlIntervalMultiplyExpression multiplyExpression =
+        newMultiplyExpression(SQL_INTERVAL_DAY, SQL_INTERVAL_MONTH);
+
+    assertEquals(SqlTypeName.INTERVAL_DAY, multiplyExpression.getOutputType());
+    assertFalse(multiplyExpression.accept());
+  }
+
+  @Test public void testEvaluate_integerOperand() {
+    BeamSqlIntervalMultiplyExpression multiplyExpression =
+        newMultiplyExpression(SQL_INTERVAL_DAY, SQL_INTEGER_FOUR);
+
+    BeamSqlPrimitive multiplicationResult =
+        multiplyExpression.evaluate(NULL_INPUT_ROW, NULL_WINDOW);
+
+    BigDecimal expectedResult =
+        DECIMAL_FOUR.multiply(timeUnitInternalMultiplier(SqlTypeName.INTERVAL_DAY));
+
+    assertEquals(expectedResult, multiplicationResult.getDecimal());
+    assertEquals(SqlTypeName.INTERVAL_DAY, multiplicationResult.getOutputType());
+  }
+
+  private BeamSqlIntervalMultiplyExpression newMultiplyExpression(BeamSqlExpression ... operands) {
+    return new BeamSqlIntervalMultiplyExpression(Arrays.asList(operands));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/820f8aff/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/TimeUnitUtilsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/TimeUnitUtilsTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/TimeUnitUtilsTest.java
new file mode 100644
index 0000000..91552ae
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/TimeUnitUtilsTest.java
@@ -0,0 +1,54 @@
+package org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.date;
+
+import static org.junit.Assert.assertEquals;
+
+import org.apache.calcite.avatica.util.TimeUnit;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Unit tests for {@link TimeUnitUtils}.
+ */
+public class TimeUnitUtilsTest {
+  @Rule public ExpectedException thrown = ExpectedException.none();
+
+  @Test public void testReturnsInternalTimeUnitMultipliers() {
+    assertEquals(TimeUnit.SECOND.multiplier,
+        TimeUnitUtils.timeUnitInternalMultiplier(SqlTypeName.INTERVAL_SECOND));
+    assertEquals(TimeUnit.MINUTE.multiplier,
+        TimeUnitUtils.timeUnitInternalMultiplier(SqlTypeName.INTERVAL_MINUTE));
+    assertEquals(TimeUnit.HOUR.multiplier,
+        TimeUnitUtils.timeUnitInternalMultiplier(SqlTypeName.INTERVAL_HOUR));
+    assertEquals(TimeUnit.DAY.multiplier,
+        TimeUnitUtils.timeUnitInternalMultiplier(SqlTypeName.INTERVAL_DAY));
+    assertEquals(TimeUnit.MONTH.multiplier,
+        TimeUnitUtils.timeUnitInternalMultiplier(SqlTypeName.INTERVAL_MONTH));
+    assertEquals(TimeUnit.YEAR.multiplier,
+        TimeUnitUtils.timeUnitInternalMultiplier(SqlTypeName.INTERVAL_YEAR));
+  }
+
+  @Test public void testThrowsForUnsupportedIntervalType() {
+    thrown.expect(IllegalArgumentException.class);
+    TimeUnitUtils.timeUnitInternalMultiplier(SqlTypeName.INTERVAL_DAY_MINUTE);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/820f8aff/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/utils/SqlTypeUtilsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/utils/SqlTypeUtilsTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/utils/SqlTypeUtilsTest.java
new file mode 100644
index 0000000..1a14256
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/utils/SqlTypeUtilsTest.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.sdk.extensions.sql.impl.utils;
+
+import static org.apache.beam.sdk.extensions.sql.impl.utils.SqlTypeUtils.findExpressionOfType;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import com.google.common.base.Optional;
+
+import java.math.BigDecimal;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Test;
+
+/**
+ * Tests for {@link SqlTypeUtils}.
+ */
+public class SqlTypeUtilsTest {
+  private static final BigDecimal DECIMAL_THREE = new BigDecimal(3);
+  private static final BigDecimal DECIMAL_FOUR = new BigDecimal(4);
+
+  private static final List<BeamSqlExpression> EXPRESSIONS = Arrays.<BeamSqlExpression> asList(
+      BeamSqlPrimitive.of(SqlTypeName.INTERVAL_DAY, DECIMAL_THREE),
+      BeamSqlPrimitive.of(SqlTypeName.INTERVAL_MONTH, DECIMAL_FOUR),
+      BeamSqlPrimitive.of(SqlTypeName.INTEGER, 4),
+      BeamSqlPrimitive.of(SqlTypeName.INTEGER, 5));
+
+  @Test public void testFindExpressionOfType_success() {
+    Optional<BeamSqlExpression> typeName = findExpressionOfType(EXPRESSIONS, SqlTypeName.INTEGER);
+
+    assertTrue(typeName.isPresent());
+    assertEquals(SqlTypeName.INTEGER, typeName.get().getOutputType());
+  }
+
+  @Test public void testFindExpressionOfType_failure() {
+    Optional<BeamSqlExpression> typeName = findExpressionOfType(EXPRESSIONS, SqlTypeName.VARCHAR);
+
+    assertFalse(typeName.isPresent());
+  }
+
+  @Test public void testFindExpressionOfTypes_success() {
+    Optional<BeamSqlExpression> typeName = findExpressionOfType(EXPRESSIONS, SqlTypeName.INT_TYPES);
+
+    assertTrue(typeName.isPresent());
+    assertEquals(SqlTypeName.INTEGER, typeName.get().getOutputType());
+  }
+
+  @Test public void testFindExpressionOfTypes_failure() {
+    Optional<BeamSqlExpression> typeName =
+        findExpressionOfType(EXPRESSIONS, SqlTypeName.CHAR_TYPES);
+
+    assertFalse(typeName.isPresent());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/820f8aff/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlDateFunctionsIntegrationTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlDateFunctionsIntegrationTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlDateFunctionsIntegrationTest.java
index 1fdb35f..6937a18 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlDateFunctionsIntegrationTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlDateFunctionsIntegrationTest.java
@@ -23,6 +23,7 @@ import static org.junit.Assert.assertTrue;
 
 import java.util.Date;
 import java.util.Iterator;
+
 import org.apache.beam.sdk.extensions.sql.BeamSql;
 import org.apache.beam.sdk.testing.PAssert;
 import org.apache.beam.sdk.transforms.SerializableFunction;
@@ -35,7 +36,7 @@ import org.junit.Test;
  */
 public class BeamSqlDateFunctionsIntegrationTest
     extends BeamSqlBuiltinFunctionsIntegrationTestBase {
-  @Test public void testDateTimeFunctions() throws Exception {
+  @Test public void testBasicDateTimeFunctions() throws Exception {
     ExpressionChecker checker = new ExpressionChecker()
         .addExpr("EXTRACT(YEAR FROM ts)", 1986L)
         .addExpr("YEAR(ts)", 1986L)
@@ -54,6 +55,42 @@ public class BeamSqlDateFunctionsIntegrationTest
     checker.buildRunAndCheck();
   }
 
+  @Test public void testDatetimePlusFunction() throws Exception {
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("TIMESTAMPADD(SECOND, 3, TIMESTAMP '1984-04-19 01:02:03')",
+            parseDate("1984-04-19 01:02:06"))
+        .addExpr("TIMESTAMPADD(MINUTE, 3, TIMESTAMP '1984-04-19 01:02:03')",
+            parseDate("1984-04-19 01:05:03"))
+        .addExpr("TIMESTAMPADD(HOUR, 3, TIMESTAMP '1984-04-19 01:02:03')",
+            parseDate("1984-04-19 04:02:03"))
+        .addExpr("TIMESTAMPADD(DAY, 3, TIMESTAMP '1984-04-19 01:02:03')",
+            parseDate("1984-04-22 01:02:03"))
+        .addExpr("TIMESTAMPADD(MONTH, 2, TIMESTAMP '1984-01-19 01:02:03')",
+            parseDate("1984-03-19 01:02:03"))
+        .addExpr("TIMESTAMPADD(YEAR, 2, TIMESTAMP '1985-01-19 01:02:03')",
+            parseDate("1987-01-19 01:02:03"))
+        ;
+    checker.buildRunAndCheck();
+  }
+
+  @Test public void testDatetimeInfixPlus() throws Exception {
+    ExpressionChecker checker = new ExpressionChecker()
+        .addExpr("TIMESTAMP '1984-01-19 01:02:03' + INTERVAL '3' SECOND",
+            parseDate("1984-01-19 01:02:06"))
+        .addExpr("TIMESTAMP '1984-01-19 01:02:03' + INTERVAL '2' MINUTE",
+            parseDate("1984-01-19 01:04:03"))
+        .addExpr("TIMESTAMP '1984-01-19 01:02:03' + INTERVAL '2' HOUR",
+            parseDate("1984-01-19 03:02:03"))
+        .addExpr("TIMESTAMP '1984-01-19 01:02:03' + INTERVAL '2' DAY",
+            parseDate("1984-01-21 01:02:03"))
+        .addExpr("TIMESTAMP '1984-01-19 01:02:03' + INTERVAL '2' MONTH",
+            parseDate("1984-03-19 01:02:03"))
+        .addExpr("TIMESTAMP '1984-01-19 01:02:03' + INTERVAL '2' YEAR",
+            parseDate("1986-01-19 01:02:03"))
+        ;
+    checker.buildRunAndCheck();
+  }
+
   @Test public void testDateTimeFunctions_currentTime() throws Exception {
     String sql = "SELECT "
         + "LOCALTIME as l,"


[34/50] [abbrv] beam git commit: Add all portability protos to Go

Posted by ke...@apache.org.
Add all portability protos to Go


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

Branch: refs/heads/mr-runner
Commit: 4210045637d78e97c02f16386483e7c8ef45e92a
Parents: e25d217
Author: Henning Rohde <he...@google.com>
Authored: Fri Oct 27 11:30:57 2017 -0700
Committer: Thomas Groh <tg...@google.com>
Committed: Mon Nov 6 11:02:27 2017 -0800

----------------------------------------------------------------------
 .../beam/model/fnexecution_v1/beam_fn_api.pb.go | 2729 ++++++++++++++
 .../fnexecution_v1/beam_provision_api.pb.go     |   57 +-
 sdks/go/pkg/beam/model/gen.go                   |    5 +-
 .../jobmanagement_v1/beam_artifact_api.pb.go    |   68 +-
 .../model/jobmanagement_v1/beam_job_api.pb.go   |  903 +++++
 .../model/pipeline_v1/beam_runner_api.pb.go     | 3491 ++++++++++++++++++
 .../pkg/beam/model/pipeline_v1/endpoints.pb.go  |  160 +
 .../model/pipeline_v1/standard_window_fns.pb.go |  120 +
 8 files changed, 7432 insertions(+), 101 deletions(-)
----------------------------------------------------------------------



[23/50] [abbrv] beam git commit: Add sdks/java/fn-execution

Posted by ke...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/fdd5971d/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/test/TestExecutorsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/test/TestExecutorsTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/test/TestExecutorsTest.java
deleted file mode 100644
index 85c64d0..0000000
--- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/test/TestExecutorsTest.java
+++ /dev/null
@@ -1,160 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.beam.fn.harness.test;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertSame;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.atomic.AtomicBoolean;
-import org.apache.beam.fn.harness.test.TestExecutors.TestExecutorService;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-import org.junit.runners.model.Statement;
-
-/** Tests for {@link TestExecutors}. */
-@RunWith(JUnit4.class)
-public class TestExecutorsTest {
-  @Test
-  public void testSuccessfulTermination() throws Throwable {
-    ExecutorService service = Executors.newSingleThreadExecutor();
-    final TestExecutorService testService = TestExecutors.from(() -> service);
-    final AtomicBoolean taskRan = new AtomicBoolean();
-    testService
-        .apply(
-            new Statement() {
-              @Override
-              public void evaluate() throws Throwable {
-                testService.submit(() -> taskRan.set(true));
-              }
-            },
-            null)
-        .evaluate();
-    assertTrue(service.isTerminated());
-    assertTrue(taskRan.get());
-  }
-
-  @Test
-  public void testTaskBlocksForeverCausesFailure() throws Throwable {
-    ExecutorService service = Executors.newSingleThreadExecutor();
-    final TestExecutorService testService = TestExecutors.from(() -> service);
-    final AtomicBoolean taskStarted = new AtomicBoolean();
-    final AtomicBoolean taskWasInterrupted = new AtomicBoolean();
-    try {
-      testService
-          .apply(
-              new Statement() {
-                @Override
-                public void evaluate() throws Throwable {
-                  testService.submit(this::taskToRun);
-                }
-
-                private void taskToRun() {
-                  taskStarted.set(true);
-                  try {
-                    while (true) {
-                      Thread.sleep(10000);
-                    }
-                  } catch (InterruptedException e) {
-                    taskWasInterrupted.set(true);
-                    return;
-                  }
-                }
-              },
-              null)
-          .evaluate();
-      fail();
-    } catch (IllegalStateException e) {
-      assertEquals(IllegalStateException.class, e.getClass());
-      assertEquals("Test executor failed to shutdown cleanly.", e.getMessage());
-    }
-    assertTrue(service.isShutdown());
-  }
-
-  @Test
-  public void testStatementFailurePropagatedCleanly() throws Throwable {
-    ExecutorService service = Executors.newSingleThreadExecutor();
-    final TestExecutorService testService = TestExecutors.from(() -> service);
-    final RuntimeException exceptionToThrow = new RuntimeException();
-    try {
-      testService
-          .apply(
-              new Statement() {
-                @Override
-                public void evaluate() throws Throwable {
-                  throw exceptionToThrow;
-                }
-              },
-              null)
-          .evaluate();
-      fail();
-    } catch (RuntimeException thrownException) {
-      assertSame(exceptionToThrow, thrownException);
-    }
-    assertTrue(service.isShutdown());
-  }
-
-  @Test
-  public void testStatementFailurePropagatedWhenExecutorServiceFailingToTerminate()
-      throws Throwable {
-    ExecutorService service = Executors.newSingleThreadExecutor();
-    final TestExecutorService testService = TestExecutors.from(() -> service);
-    final AtomicBoolean taskStarted = new AtomicBoolean();
-    final AtomicBoolean taskWasInterrupted = new AtomicBoolean();
-    final RuntimeException exceptionToThrow = new RuntimeException();
-    try {
-      testService
-          .apply(
-              new Statement() {
-                @Override
-                public void evaluate() throws Throwable {
-                  testService.submit(this::taskToRun);
-                  throw exceptionToThrow;
-                }
-
-                private void taskToRun() {
-                  taskStarted.set(true);
-                  try {
-                    while (true) {
-                      Thread.sleep(10000);
-                    }
-                  } catch (InterruptedException e) {
-                    taskWasInterrupted.set(true);
-                    return;
-                  }
-                }
-              },
-              null)
-          .evaluate();
-      fail();
-    } catch (RuntimeException thrownException) {
-      assertSame(exceptionToThrow, thrownException);
-      assertEquals(1, exceptionToThrow.getSuppressed().length);
-      assertEquals(IllegalStateException.class, exceptionToThrow.getSuppressed()[0].getClass());
-      assertEquals(
-          "Test executor failed to shutdown cleanly.",
-          exceptionToThrow.getSuppressed()[0].getMessage());
-    }
-    assertTrue(service.isShutdown());
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/fdd5971d/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/test/TestStreams.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/test/TestStreams.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/test/TestStreams.java
deleted file mode 100644
index f398286..0000000
--- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/test/TestStreams.java
+++ /dev/null
@@ -1,162 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.beam.fn.harness.test;
-
-import io.grpc.stub.CallStreamObserver;
-import io.grpc.stub.StreamObserver;
-import java.util.function.Consumer;
-import java.util.function.Supplier;
-
-/** Utility methods which enable testing of {@link StreamObserver}s. */
-public class TestStreams {
-  /**
-   * Creates a test {@link CallStreamObserver}  {@link Builder} that forwards
-   * {@link StreamObserver#onNext} calls to the supplied {@link Consumer}.
-   */
-  public static <T> Builder<T> withOnNext(Consumer<T> onNext) {
-    return new Builder<>(new ForwardingCallStreamObserver<>(
-        onNext,
-        TestStreams::noop,
-        TestStreams::noop,
-        TestStreams::returnTrue));
-  }
-
-  /** A builder for a test {@link CallStreamObserver} that performs various callbacks. */
-  public static class Builder<T> {
-    private final ForwardingCallStreamObserver<T> observer;
-    private Builder(ForwardingCallStreamObserver<T> observer) {
-      this.observer = observer;
-    }
-
-    /**
-     * Returns a new {@link Builder} like this one with the specified
-     * {@link CallStreamObserver#isReady} callback.
-     */
-    public Builder<T> withIsReady(Supplier<Boolean> isReady) {
-      return new Builder<>(new ForwardingCallStreamObserver<>(
-          observer.onNext,
-          observer.onError,
-          observer.onCompleted,
-          isReady));
-    }
-
-    /**
-     * Returns a new {@link Builder} like this one with the specified
-     * {@link StreamObserver#onCompleted} callback.
-     */
-    public Builder<T> withOnCompleted(Runnable onCompleted) {
-      return new Builder<>(new ForwardingCallStreamObserver<>(
-          observer.onNext,
-          observer.onError,
-          onCompleted,
-          observer.isReady));
-    }
-
-    /**
-     * Returns a new {@link Builder} like this one with the specified
-     * {@link StreamObserver#onError} callback.
-     */
-    public Builder<T> withOnError(Runnable onError) {
-      return new Builder<>(new ForwardingCallStreamObserver<>(
-          observer.onNext,
-          new Consumer<Throwable>() {
-            @Override
-            public void accept(Throwable t) {
-              onError.run();
-            }
-          },
-          observer.onCompleted,
-          observer.isReady));
-    }
-
-    /**
-     * Returns a new {@link Builder} like this one with the specified
-     * {@link StreamObserver#onError} consumer.
-     */
-    public Builder<T> withOnError(Consumer<Throwable> onError) {
-      return new Builder<>(new ForwardingCallStreamObserver<>(
-          observer.onNext, onError, observer.onCompleted, observer.isReady));
-    }
-
-    public CallStreamObserver<T> build() {
-      return observer;
-    }
-  }
-
-  private static void noop() {
-  }
-
-  private static void noop(Throwable t) {
-  }
-
-  private static boolean returnTrue() {
-    return true;
-  }
-
-  /** A {@link CallStreamObserver} which executes the supplied callbacks. */
-  private static class ForwardingCallStreamObserver<T> extends CallStreamObserver<T> {
-    private final Consumer<T> onNext;
-    private final Supplier<Boolean> isReady;
-    private final Consumer<Throwable> onError;
-    private final Runnable onCompleted;
-
-    public ForwardingCallStreamObserver(
-        Consumer<T> onNext,
-        Consumer<Throwable> onError,
-        Runnable onCompleted,
-        Supplier<Boolean> isReady) {
-      this.onNext = onNext;
-      this.onError = onError;
-      this.onCompleted = onCompleted;
-      this.isReady = isReady;
-    }
-
-    @Override
-    public void onNext(T value) {
-      onNext.accept(value);
-    }
-
-    @Override
-    public void onError(Throwable t) {
-      onError.accept(t);
-    }
-
-    @Override
-    public void onCompleted() {
-      onCompleted.run();
-    }
-
-    @Override
-    public boolean isReady() {
-      return isReady.get();
-    }
-
-    @Override
-    public void setOnReadyHandler(Runnable onReadyHandler) {}
-
-    @Override
-    public void disableAutoInboundFlowControl() {}
-
-    @Override
-    public void request(int count) {}
-
-    @Override
-    public void setMessageCompression(boolean enable) {}
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/fdd5971d/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/test/TestStreamsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/test/TestStreamsTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/test/TestStreamsTest.java
deleted file mode 100644
index b684c90..0000000
--- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/test/TestStreamsTest.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.beam.fn.harness.test;
-
-import static org.hamcrest.Matchers.contains;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertThat;
-import static org.junit.Assert.assertTrue;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.concurrent.atomic.AtomicBoolean;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/** Tests for {@link TestStreams}. */
-@RunWith(JUnit4.class)
-public class TestStreamsTest {
-  @Test
-  public void testOnNextIsCalled() {
-    AtomicBoolean onNextWasCalled = new AtomicBoolean();
-    TestStreams.withOnNext(onNextWasCalled::set).build().onNext(true);
-    assertTrue(onNextWasCalled.get());
-  }
-
-  @Test
-  public void testIsReadyIsCalled() {
-    final AtomicBoolean isReadyWasCalled = new AtomicBoolean();
-    assertFalse(TestStreams.withOnNext(null)
-        .withIsReady(() -> isReadyWasCalled.getAndSet(true))
-        .build()
-        .isReady());
-    assertTrue(isReadyWasCalled.get());
-  }
-
-  @Test
-  public void testOnCompletedIsCalled() {
-    AtomicBoolean onCompletedWasCalled = new AtomicBoolean();
-    TestStreams.withOnNext(null)
-        .withOnCompleted(() -> onCompletedWasCalled.set(true))
-        .build()
-        .onCompleted();
-    assertTrue(onCompletedWasCalled.get());
-  }
-
-  @Test
-  public void testOnErrorRunnableIsCalled() {
-    RuntimeException throwable = new RuntimeException();
-    AtomicBoolean onErrorWasCalled = new AtomicBoolean();
-    TestStreams.withOnNext(null)
-        .withOnError(() -> onErrorWasCalled.set(true))
-        .build()
-        .onError(throwable);
-    assertTrue(onErrorWasCalled.get());
-  }
-
-  @Test
-  public void testOnErrorConsumerIsCalled() {
-    RuntimeException throwable = new RuntimeException();
-    Collection<Throwable> onErrorWasCalled = new ArrayList<>();
-    TestStreams.withOnNext(null)
-        .withOnError(onErrorWasCalled::add)
-        .build()
-        .onError(throwable);
-    assertThat(onErrorWasCalled, contains(throwable));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/fdd5971d/sdks/java/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/pom.xml b/sdks/java/pom.xml
index e5af784..62e4ec3 100644
--- a/sdks/java/pom.xml
+++ b/sdks/java/pom.xml
@@ -53,6 +53,7 @@
         <jdk>[1.8,)</jdk>
       </activation>
       <modules>
+        <module>fn-execution</module>
         <module>harness</module>
         <module>container</module>
         <module>java8tests</module>


[02/50] [abbrv] beam git commit: [BEAM-2482] - CodedValueMutationDetector should use the coders structural value

Posted by ke...@apache.org.
[BEAM-2482] - CodedValueMutationDetector should use the coders structural value


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

Branch: refs/heads/mr-runner
Commit: 84da6caca58a52731f9cf5d9b6bfb85977241925
Parents: e3f6d6f
Author: Luke Cwik <lc...@google.com>
Authored: Wed Nov 1 10:43:11 2017 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Wed Nov 1 10:43:11 2017 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/beam/sdk/util/MutationDetectors.java    | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/84da6cac/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MutationDetectors.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MutationDetectors.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MutationDetectors.java
index 3556667..79b960a 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MutationDetectors.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MutationDetectors.java
@@ -108,8 +108,8 @@ public class MutationDetectors {
     public CodedValueMutationDetector(T value, Coder<T> coder) throws CoderException {
       this.coder = coder;
       // We need to clone the original value before getting it's structural value.
-      // If the object is consistent with equals, the Structural value will be the exact
-      // same object reference making it impossible to detect changes.
+      // If the object is consistent with equals, the Structural value will be the
+      // exact same object reference making it impossible to detect changes.
       clonedOriginalValue = CoderUtils.clone(coder, value);
       this.originalStructuralValue = coder.structuralValue(clonedOriginalValue);
       this.possiblyModifiedObject = value;


[20/50] [abbrv] beam git commit: [BEAM-3135] Adding futures dependency to python SDK

Posted by ke...@apache.org.
[BEAM-3135] Adding futures dependency to python SDK


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

Branch: refs/heads/mr-runner
Commit: f0b2b3c6c45c03db9565e34683378b259281776f
Parents: 4925478
Author: Ankur Goenka <go...@goenka.svl.corp.google.com>
Authored: Wed Nov 1 16:27:52 2017 -0700
Committer: Ahmet Altay <al...@google.com>
Committed: Thu Nov 2 16:37:51 2017 -0700

----------------------------------------------------------------------
 sdks/python/setup.py | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/f0b2b3c6/sdks/python/setup.py
----------------------------------------------------------------------
diff --git a/sdks/python/setup.py b/sdks/python/setup.py
index 16672cf..5852d15 100644
--- a/sdks/python/setup.py
+++ b/sdks/python/setup.py
@@ -108,6 +108,7 @@ REQUIRED_PACKAGES = [
     # TODO(BEAM-2964): Remove the upper bound.
     'six>=1.9,<1.11',
     'typing>=3.6.0,<3.7.0',
+    'futures>=3.1.1,<4.0.0',
     ]
 
 REQUIRED_SETUP_PACKAGES = [


[05/50] [abbrv] beam git commit: Remove obsolete extra parameter

Posted by ke...@apache.org.
Remove obsolete extra parameter

This closes #4061


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

Branch: refs/heads/mr-runner
Commit: b013d7c5a35b81ae988dc0625fa15a0df21eba5a
Parents: b743ab1 6443754
Author: Luke Cwik <lc...@google.com>
Authored: Wed Nov 1 10:46:52 2017 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Wed Nov 1 10:46:52 2017 -0700

----------------------------------------------------------------------
 .../org/apache/beam/sdk/transforms/join/CoGbkResult.java     | 8 +++-----
 1 file changed, 3 insertions(+), 5 deletions(-)
----------------------------------------------------------------------



[37/50] [abbrv] beam git commit: update dataflow.version

Posted by ke...@apache.org.
update dataflow.version


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

Branch: refs/heads/mr-runner
Commit: 9e5f641de23a959046b617c79defd043724e1fae
Parents: e25d217
Author: Alex Amato <aj...@ajamato-linux0.sea.corp.google.com>
Authored: Mon Nov 6 15:21:13 2017 -0800
Committer: Alex Amato <aj...@ajamato-linux0.sea.corp.google.com>
Committed: Mon Nov 6 15:21:13 2017 -0800

----------------------------------------------------------------------
 pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/9e5f641d/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index baed9ba..0b7b323 100644
--- a/pom.xml
+++ b/pom.xml
@@ -113,7 +113,7 @@
     <cloudresourcemanager.version>v1-rev6-1.22.0</cloudresourcemanager.version>
     <pubsubgrpc.version>0.1.18</pubsubgrpc.version>
     <clouddebugger.version>v2-rev8-1.22.0</clouddebugger.version>
-    <dataflow.version>v1b3-rev214-1.22.0</dataflow.version>
+    <dataflow.version>v1b3-rev218-1.22.0</dataflow.version>
     <dataflow.proto.version>0.5.160222</dataflow.proto.version>
     <datastore.client.version>1.4.0</datastore.client.version>
     <datastore.proto.version>1.3.0</datastore.proto.version>


[08/50] [abbrv] beam git commit: This closes #4044

Posted by ke...@apache.org.
This closes #4044


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

Branch: refs/heads/mr-runner
Commit: 1f37c2932a7d3fa6180c8b0e45b3190edced8d4a
Parents: 8247b53 6ea8afd
Author: Thomas Groh <tg...@google.com>
Authored: Wed Nov 1 13:57:26 2017 -0700
Committer: Thomas Groh <tg...@google.com>
Committed: Wed Nov 1 13:57:26 2017 -0700

----------------------------------------------------------------------
 sdks/java/harness/pom.xml | 7 -------
 1 file changed, 7 deletions(-)
----------------------------------------------------------------------



[13/50] [abbrv] beam git commit: This closes #4052

Posted by ke...@apache.org.
This closes #4052


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

Branch: refs/heads/mr-runner
Commit: ee2292af544d2276390e9336f1c252b0a6c46c23
Parents: a33c717 820f8af
Author: mingmxu <mi...@ebay.com>
Authored: Wed Nov 1 16:31:38 2017 -0700
Committer: mingmxu <mi...@ebay.com>
Committed: Wed Nov 1 16:31:38 2017 -0700

----------------------------------------------------------------------
 .../sql/impl/interpreter/BeamSqlFnExecutor.java |  19 ++-
 .../interpreter/operator/BeamSqlPrimitive.java  |   8 +-
 .../date/BeamSqlDatetimePlusExpression.java     | 129 +++++++++++++++
 .../date/BeamSqlIntervalMultiplyExpression.java | 103 ++++++++++++
 .../operator/date/TimeUnitUtils.java            |  54 +++++++
 .../extensions/sql/impl/utils/SqlTypeUtils.java |  46 ++++++
 .../impl/interpreter/BeamSqlFnExecutorTest.java |  30 ++++
 .../date/BeamSqlDateExpressionTestBase.java     |   5 +-
 .../date/BeamSqlDatetimePlusExpressionTest.java | 155 +++++++++++++++++++
 .../BeamSqlIntervalMultiplyExpressionTest.java  | 107 +++++++++++++
 .../operator/date/TimeUnitUtilsTest.java        |  54 +++++++
 .../sql/impl/utils/SqlTypeUtilsTest.java        |  76 +++++++++
 .../BeamSqlDateFunctionsIntegrationTest.java    |  39 ++++-
 13 files changed, 818 insertions(+), 7 deletions(-)
----------------------------------------------------------------------



[35/50] [abbrv] beam git commit: Fix Go package comment for syscallx

Posted by ke...@apache.org.
Fix Go package comment for syscallx


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

Branch: refs/heads/mr-runner
Commit: 6c27b82bb80d8520b9e11ee32248a2f9020bcb66
Parents: 4210045
Author: Henning Rohde <he...@google.com>
Authored: Fri Oct 27 11:31:33 2017 -0700
Committer: Thomas Groh <tg...@google.com>
Committed: Mon Nov 6 11:02:27 2017 -0800

----------------------------------------------------------------------
 sdks/go/pkg/beam/util/syscallx/syscall.go | 7 ++++---
 1 file changed, 4 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/6c27b82b/sdks/go/pkg/beam/util/syscallx/syscall.go
----------------------------------------------------------------------
diff --git a/sdks/go/pkg/beam/util/syscallx/syscall.go b/sdks/go/pkg/beam/util/syscallx/syscall.go
index 151f860..ca352ec 100644
--- a/sdks/go/pkg/beam/util/syscallx/syscall.go
+++ b/sdks/go/pkg/beam/util/syscallx/syscall.go
@@ -13,9 +13,10 @@
 // See the License for the specific language governing permissions and
 // limitations under the License.
 
-// Syscallx provide system call utilities that attempt to hide platform differences.
-// Operations returns UnsupportedErr if not implemented on the given platform, so
-// consumers of this package should generally treat that error specially.
+// Package syscallx provides system call utilities that attempt to hide platform
+// differences. Operations return UnsupportedErr if not implemented on the
+// given platform. Consumers of this package should generally treat that
+// error specially.
 package syscallx
 
 import (


[39/50] [abbrv] beam git commit: This closes #4073

Posted by ke...@apache.org.
This closes #4073


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

Branch: refs/heads/mr-runner
Commit: 7550a589eec9abd81309326886349477ca2b59bc
Parents: 766b4f3 012c2e6
Author: Thomas Groh <tg...@google.com>
Authored: Mon Nov 6 16:06:57 2017 -0800
Committer: Thomas Groh <tg...@google.com>
Committed: Mon Nov 6 16:06:57 2017 -0800

----------------------------------------------------------------------
 runners/java-fn-execution/pom.xml               | 14 ------
 .../runners/fnexecution/ServerFactoryTest.java  | 45 ++++++++++++++-----
 runners/pom.xml                                 |  2 +-
 sdks/java/fn-execution/pom.xml                  | 20 ++-------
 .../org/apache/beam/harness/test/Consumer.java  | 26 +++++++++++
 .../org/apache/beam/harness/test/Supplier.java  | 26 +++++++++++
 .../apache/beam/harness/test/TestExecutors.java | 12 ++++-
 .../beam/harness/test/TestExecutorsTest.java    | 29 +++++++++---
 .../apache/beam/harness/test/TestStreams.java   | 35 ++++++++++++---
 .../beam/harness/test/TestStreamsTest.java      | 47 +++++++++++++++-----
 .../apache/beam/fn/harness/FnHarnessTest.java   |  4 +-
 .../harness/data/BeamFnDataGrpcClientTest.java  |  5 ++-
 .../stream/BufferingStreamObserverTest.java     |  2 +-
 .../stream/DirectStreamObserverTest.java        |  2 +-
 sdks/java/pom.xml                               |  2 +-
 15 files changed, 196 insertions(+), 75 deletions(-)
----------------------------------------------------------------------



[22/50] [abbrv] beam git commit: This closes #3986

Posted by ke...@apache.org.
This closes #3986


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

Branch: refs/heads/mr-runner
Commit: 4f4632cccef6e1f0890bfc333534c186a2065aeb
Parents: fbcb0ba 927a8db
Author: Thomas Groh <tg...@google.com>
Authored: Thu Nov 2 18:25:12 2017 -0700
Committer: Thomas Groh <tg...@google.com>
Committed: Thu Nov 2 18:25:12 2017 -0700

----------------------------------------------------------------------
 ...job_beam_PostCommit_Java_MavenInstall.groovy |   2 +-
 .../job_beam_PreCommit_Java_MavenInstall.groovy |   4 +-
 pom.xml                                         |  67 +++++++-
 runners/java-fn-execution/pom.xml               | 105 ++++++++++++
 .../beam/runners/fnexecution/ServerFactory.java | 104 ++++++++++++
 .../beam/runners/fnexecution/package-info.java  |  23 +++
 .../runners/fnexecution/ServerFactoryTest.java  | 128 +++++++++++++++
 runners/pom.xml                                 |   1 +
 sdks/java/fn-execution/pom.xml                  |  96 +++++++++++
 .../harness/channel/ManagedChannelFactory.java  |  82 ++++++++++
 .../harness/channel/SocketAddressFactory.java   |  64 ++++++++
 .../beam/harness/channel/package-info.java      |  22 +++
 .../channel/ManagedChannelFactoryTest.java      |  71 ++++++++
 .../channel/SocketAddressFactoryTest.java       |  56 +++++++
 .../apache/beam/harness/test/TestExecutors.java |  85 ++++++++++
 .../beam/harness/test/TestExecutorsTest.java    | 160 ++++++++++++++++++
 .../apache/beam/harness/test/TestStreams.java   | 162 +++++++++++++++++++
 .../beam/harness/test/TestStreamsTest.java      |  84 ++++++++++
 sdks/java/harness/pom.xml                       |  26 +++
 .../org/apache/beam/fn/harness/FnHarness.java   |  19 ++-
 .../harness/channel/ManagedChannelFactory.java  |  86 ----------
 .../harness/channel/SocketAddressFactory.java   |  64 --------
 .../beam/fn/harness/channel/package-info.java   |  22 ---
 .../fn/harness/BeamFnDataReadRunnerTest.java    |   4 +-
 .../apache/beam/fn/harness/FnHarnessTest.java   |   2 +-
 .../channel/ManagedChannelFactoryTest.java      |  73 ---------
 .../channel/SocketAddressFactoryTest.java       |  56 -------
 .../control/BeamFnControlClientTest.java        |   2 +-
 .../fn/harness/control/RegisterHandlerTest.java |   4 +-
 ...BeamFnDataBufferingOutboundObserverTest.java |   2 +-
 .../harness/data/BeamFnDataGrpcClientTest.java  |   2 +-
 .../data/BeamFnDataGrpcMultiplexerTest.java     |   2 +-
 .../logging/BeamFnLoggingClientTest.java        |   2 +-
 .../state/BeamFnStateGrpcClientCacheTest.java   |   2 +-
 .../stream/BufferingStreamObserverTest.java     |   6 +-
 .../stream/DirectStreamObserverTest.java        |   6 +-
 .../beam/fn/harness/test/TestExecutors.java     |  85 ----------
 .../beam/fn/harness/test/TestExecutorsTest.java | 160 ------------------
 .../beam/fn/harness/test/TestStreams.java       | 162 -------------------
 .../beam/fn/harness/test/TestStreamsTest.java   |  84 ----------
 sdks/java/pom.xml                               |   1 +
 41 files changed, 1370 insertions(+), 818 deletions(-)
----------------------------------------------------------------------



[26/50] [abbrv] beam git commit: [BEAM-2728] Extension for sketch-based statistics : HyperLogLog

Posted by ke...@apache.org.
[BEAM-2728] Extension for sketch-based statistics : HyperLogLog


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

Branch: refs/heads/mr-runner
Commit: fd58a423099b5aea5cd78c862e81c6a03bbf6521
Parents: 4f4632c
Author: Arnaud Fournier <ar...@gmail.com>
Authored: Thu Jul 20 16:57:38 2017 +0200
Committer: Eugene Kirpichov <ek...@gmail.com>
Committed: Fri Nov 3 15:14:57 2017 -0700

----------------------------------------------------------------------
 pom.xml                                         |   6 +
 sdks/java/extensions/pom.xml                    |   1 +
 sdks/java/extensions/sketching/pom.xml          | 104 ++++
 .../sketching/ApproximateDistinct.java          | 573 +++++++++++++++++++
 .../sdk/extensions/sketching/package-info.java  |  22 +
 .../sketching/ApproximateDistinctTest.java      | 209 +++++++
 sdks/java/javadoc/pom.xml                       |   5 +
 7 files changed, 920 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/fd58a423/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index b2ab5d7..baed9ba 100644
--- a/pom.xml
+++ b/pom.xml
@@ -502,6 +502,12 @@
 
       <dependency>
         <groupId>org.apache.beam</groupId>
+        <artifactId>beam-sdks-java-extensions-sketching</artifactId>
+        <version>${project.version}</version>
+      </dependency>
+
+      <dependency>
+        <groupId>org.apache.beam</groupId>
         <artifactId>beam-sdks-java-extensions-sorter</artifactId>
         <version>${project.version}</version>
       </dependency>

http://git-wip-us.apache.org/repos/asf/beam/blob/fd58a423/sdks/java/extensions/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/pom.xml b/sdks/java/extensions/pom.xml
index ec6efb6..5e8d495 100644
--- a/sdks/java/extensions/pom.xml
+++ b/sdks/java/extensions/pom.xml
@@ -36,6 +36,7 @@
     <module>jackson</module>
     <module>join-library</module>
     <module>protobuf</module>
+    <module>sketching</module>
     <module>sorter</module>
     <module>sql</module>
   </modules>

http://git-wip-us.apache.org/repos/asf/beam/blob/fd58a423/sdks/java/extensions/sketching/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sketching/pom.xml b/sdks/java/extensions/sketching/pom.xml
new file mode 100755
index 0000000..f0538ae
--- /dev/null
+++ b/sdks/java/extensions/sketching/pom.xml
@@ -0,0 +1,104 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+    Licensed to the Apache Software Foundation (ASF) under one or more
+    contributor license agreements.  See the NOTICE file distributed with
+    this work for additional information regarding copyright ownership.
+    The ASF licenses this file to You under the Apache License, Version 2.0
+    (the "License"); you may not use this file except in compliance with
+    the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing, software
+    distributed under the License is distributed on an "AS IS" BASIS,
+    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+    See the License for the specific language governing permissions and
+    limitations under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+
+  <modelVersion>4.0.0</modelVersion>
+
+  <parent>
+    <groupId>org.apache.beam</groupId>
+    <artifactId>beam-sdks-java-extensions-parent</artifactId>
+    <version>2.3.0-SNAPSHOT</version>
+    <relativePath>../pom.xml</relativePath>
+  </parent>
+
+  <artifactId>beam-sdks-java-extensions-sketching</artifactId>
+  <name>Apache Beam :: SDKs :: Java :: Extensions :: Sketching</name>
+
+  <properties>
+    <streamlib.version>2.9.5</streamlib.version>
+  </properties>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>beam-sdks-java-core</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.clearspring.analytics</groupId>
+      <artifactId>stream</artifactId>
+      <version>${streamlib.version}</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-api</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.auto.value</groupId>
+      <artifactId>auto-value</artifactId>
+      <scope>provided</scope>
+    </dependency>
+
+    <!-- test dependencies -->
+    <!-- https://mvnrepository.com/artifact/org.apache.avro/avro -->
+    <dependency>
+      <groupId>org.apache.avro</groupId>
+      <artifactId>avro</artifactId>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>beam-sdks-java-core</artifactId>
+      <classifier>tests</classifier>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.commons</groupId>
+      <artifactId>commons-lang3</artifactId>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>beam-runners-direct-java</artifactId>
+      <scope>test</scope>
+    </dependency>
+    
+    <dependency>
+      <groupId>org.hamcrest</groupId>
+      <artifactId>hamcrest-all</artifactId>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+
+  </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/beam/blob/fd58a423/sdks/java/extensions/sketching/src/main/java/org/apache/beam/sdk/extensions/sketching/ApproximateDistinct.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sketching/src/main/java/org/apache/beam/sdk/extensions/sketching/ApproximateDistinct.java b/sdks/java/extensions/sketching/src/main/java/org/apache/beam/sdk/extensions/sketching/ApproximateDistinct.java
new file mode 100644
index 0000000..1da0cc3
--- /dev/null
+++ b/sdks/java/extensions/sketching/src/main/java/org/apache/beam/sdk/extensions/sketching/ApproximateDistinct.java
@@ -0,0 +1,573 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.sketching;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import com.clearspring.analytics.stream.cardinality.CardinalityMergeException;
+import com.clearspring.analytics.stream.cardinality.HyperLogLogPlus;
+import com.google.auto.value.AutoValue;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.coders.ByteArrayCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CustomCoder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.transforms.Combine;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+import org.apache.beam.sdk.util.CoderUtils;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+
+/**
+ * {@link PTransform}s for computing the approximate number of distinct elements in a stream.
+ *
+ * <p>This class relies on the HyperLogLog algorithm, and more precisely HyperLogLog+, the improved
+ * version of Google.
+ *
+ * <h2>References</h2>
+ *
+ * <p>The implementation comes from <a href="https://github.com/addthis/stream-lib">Addthis'
+ * Stream-lib library</a>. <br>
+ * The original paper of the HyperLogLog is available <a
+ * href="http://algo.inria.fr/flajolet/Publications/FlFuGaMe07.pdf">here</a>. <br>
+ * A paper from the same authors to have a clearer view of the algorithm is available <a
+ * href="http://cscubs.cs.uni-bonn.de/2016/proceedings/paper-03.pdf">here</a>. <br>
+ * Google's HyperLogLog+ version is detailed in <a
+ * href="https://research.google.com/pubs/pub40671.html">this paper</a>.
+ *
+ * <h2>Parameters</h2>
+ *
+ * <p>Two parameters can be tuned in order to control the computation's accuracy:
+ *
+ * <ul>
+ *   <li><b>Precision: {@code p}</b> <br>
+ *       Controls the accuracy of the estimation. The precision value will have an impact on the
+ *       number of buckets used to store information about the distinct elements. <br>
+ *       In general one can expect a relative error of about {@code 1.1 / sqrt(2^p)}. The value
+ *       should be of at least 4 to guarantee a minimal accuracy. <br>
+ *       By default, the precision is set to {@code 12} for a relative error of around {@code 2%}.
+ *   <li><b>Sparse Precision: {@code sp}</b> <br>
+ *       Used to create a sparse representation in order to optimize memory and improve accuracy at
+ *       small cardinalities. <br>
+ *       The value of {@code sp} should be greater than {@code p}, but lower than 32. <br>
+ *       By default, the sparse representation is not used ({@code sp = 0}). One should use it if
+ *       the cardinality may be less than {@code 12000}.
+ * </ul>
+ *
+ * <h2>Examples</h2>
+ *
+ * <p>There are 2 ways of using this class:
+ *
+ * <ul>
+ *   <li>Use the {@link PTransform}s that return {@code PCollection<Long>} corresponding to the
+ *       estimate number of distinct elements in the input {@link PCollection} of objects or for
+ *       each key in a {@link PCollection} of {@link KV}s.
+ *   <li>Use the {@link ApproximateDistinctFn} {@code CombineFn} that is exposed in order to make
+ *       advanced processing involving the {@link HyperLogLogPlus} structure which resumes the
+ *       stream.
+ * </ul>
+ *
+ * <h3>Using the Transforms</h3>
+ *
+ * <h4>Example 1: globally default use</h4>
+ *
+ * <pre>{@code
+ * PCollection<Integer> input = ...;
+ * PCollection<Long> hllSketch = input.apply(ApproximateDistinct.<Integer>globally());
+ * }</pre>
+ *
+ * <h4>Example 2: per key default use</h4>
+ *
+ * <pre>{@code
+ * PCollection<Integer, String> input = ...;
+ * PCollection<Integer, Long> hllSketches = input.apply(ApproximateDistinct
+ *                .<Integer, String>perKey());
+ * }</pre>
+ *
+ * <h4>Example 3: tune precision and use sparse representation</h4>
+ *
+ * <p>One can tune the precision and sparse precision parameters in order to control the accuracy
+ * and the memory. The tuning works exactly the same for {@link #globally()} and {@link #perKey()}.
+ *
+ * <pre>{@code
+ * int precision = 15;
+ * int sparsePrecision = 25;
+ * PCollection<Double> input = ...;
+ * PCollection<Long> hllSketch = input.apply(ApproximateDistinct
+ *                .<Double>globally()
+ *                .withPrecision(precision)
+ *                .withSparsePrecision(sparsePrecision));
+ * }</pre>
+ *
+ * <h3>Using the {@link ApproximateDistinctFn} CombineFn</h3>
+ *
+ * <p>The CombineFn does the same thing as the transform but it can be used in cases where you want
+ * to manipulate the {@link HyperLogLogPlus} sketch, for example if you want to store it in a
+ * database to have a backup. It can also be used in stateful processing or in {@link
+ * org.apache.beam.sdk.transforms.CombineFns.ComposedCombineFn}.
+ *
+ * <h4>Example 1: basic use</h4>
+ *
+ * <p>This example is not really interesting but show how you can properly create an {@link
+ * ApproximateDistinctFn}. One must always specify a coder using the {@link
+ * ApproximateDistinctFn#create(Coder)} method.
+ *
+ * <pre>{@code
+ * PCollection<Integer> input = ...;
+ * PCollection<HyperLogLogPlus> output = input.apply(Combine.globally(ApproximateDistinctFn
+ *                 .<Integer>create(BigEndianIntegerCoder.of()));
+ * }</pre>
+ *
+ * <h4>Example 2: use the {@link CombineFn} in a stateful {@link ParDo}</h4>
+ *
+ * <p>One may want to use the {@link ApproximateDistinctFn} in a stateful ParDo in order to make
+ * some processing depending on the current cardinality of the stream. <br>
+ * For more information about stateful processing see the blog spot on this topic <a
+ * href="https://beam.apache.org/blog/2017/02/13/stateful-processing.html">here</a>.
+ *
+ * <p>Here is an example of {@link DoFn} using an {@link ApproximateDistinctFn} as a {@link
+ * org.apache.beam.sdk.state.CombiningState}:
+ *
+ * <pre><code>
+ * {@literal class StatefulCardinality<V> extends DoFn<V, OutputT>} {
+ *   {@literal @StateId}("hyperloglog")
+ *   {@literal private final StateSpec<CombiningState<V, HyperLogLogPlus, HyperLogLogPlus>>}
+ *      indexSpec;
+ *
+ *   {@literal public StatefulCardinality(ApproximateDistinctFn<V> fn)} {
+ *     indexSpec = StateSpecs.combining(fn);
+ *   }
+ *
+ *  {@literal @ProcessElement}
+ *   public void processElement(
+ *      ProcessContext context,
+ *      {@literal @StateId}("hllSketch")
+ *      {@literal CombiningState<V, HyperLogLogPlus, HyperLogLogPlus> hllSketch)} {
+ *     long current = MoreObjects.firstNonNull(hllSketch.getAccum().cardinality(), 0L);
+ *     hllSketch.add(context.element());
+ *     context.output(...);
+ *   }
+ * }
+ * </code></pre>
+ *
+ * <p>Then the {@link DoFn} can be called like this:
+ *
+ * <pre>{@code
+ * PCollection<V> input = ...;
+ * ApproximateDistinctFn<V> myFn = ApproximateDistinctFn.create(input.getCoder());
+ * PCollection<V> = input.apply(ParDo.of(new StatefulCardinality<>(myFn)));
+ * }</pre>
+ *
+ * <h4>Example 3: use the {@link RetrieveCardinality} utility class</h4>
+ *
+ * <p>One may want to retrieve the cardinality as a long after making some advanced processing using
+ * the {@link HyperLogLogPlus} structure. <br>
+ * The {@link RetrieveCardinality} utility class provides an easy way to do so:
+ *
+ * <pre>{@code
+ * PCollection<MyObject> input = ...;
+ * PCollection<HyperLogLogPlus> hll = input.apply(Combine.globally(ApproximateDistinctFn
+ *                  .<MyObject>create(new MyObjectCoder())
+ *                  .withSparseRepresentation(20)));
+ *
+ *  // Some advanced processing
+ *  PCollection<SomeObject> advancedResult = hll.apply(...);
+ *
+ *  PCollection<Long> cardinality = hll.apply(ApproximateDistinct.RetrieveCardinality.globally());
+ *
+ * }</pre>
+ *
+ * <p><b>Warning: this class is experimental.</b> Its API is subject to change in future versions of
+ * Beam. For example, it may be merged with the {@link
+ * org.apache.beam.sdk.transforms.ApproximateUnique} transform.
+ */
+@Experimental
+public final class ApproximateDistinct {
+
+  /**
+   * Computes the approximate number of distinct elements in the input {@code PCollection<InputT>}
+   * and returns a {@code PCollection<Long>}.
+   *
+   * @param <InputT> the type of the elements in the input {@link PCollection}
+   */
+  public static <InputT> GloballyDistinct<InputT> globally() {
+    return GloballyDistinct.<InputT>builder().build();
+  }
+
+  /**
+   * Like {@link #globally} but per key, i.e computes the approximate number of distinct values per
+   * key in a {@code PCollection<KV<K, V>>} and returns {@code PCollection<KV<K, Long>>}.
+   *
+   * @param <K> type of the keys mapping the elements
+   * @param <V> type of the values being combined per key
+   */
+  public static <K, V> PerKeyDistinct<K, V> perKey() {
+    return PerKeyDistinct.<K, V>builder().build();
+  }
+
+  /**
+   * Implementation of {@link #globally()}.
+   *
+   * @param <InputT>
+   */
+  @AutoValue
+  public abstract static class GloballyDistinct<InputT>
+      extends PTransform<PCollection<InputT>, PCollection<Long>> {
+
+    abstract int precision();
+
+    abstract int sparsePrecision();
+
+    abstract Builder<InputT> toBuilder();
+
+    static <InputT> Builder<InputT> builder() {
+      return new AutoValue_ApproximateDistinct_GloballyDistinct.Builder<InputT>()
+          .setPrecision(12)
+          .setSparsePrecision(0);
+    }
+
+    @AutoValue.Builder
+    abstract static class Builder<InputT> {
+      abstract Builder<InputT> setPrecision(int p);
+
+      abstract Builder<InputT> setSparsePrecision(int sp);
+
+      abstract GloballyDistinct<InputT> build();
+    }
+
+    public GloballyDistinct<InputT> withPrecision(int p) {
+      return toBuilder().setPrecision(p).build();
+    }
+
+    public GloballyDistinct<InputT> withSparsePrecision(int sp) {
+      return toBuilder().setSparsePrecision(sp).build();
+    }
+
+    @Override
+    public PCollection<Long> expand(PCollection<InputT> input) {
+      return input
+          .apply(
+              "Compute HyperLogLog Structure",
+              Combine.globally(
+                  ApproximateDistinctFn.<InputT>create(input.getCoder())
+                      .withPrecision(this.precision())
+                      .withSparseRepresentation(this.sparsePrecision())))
+          .apply("Retrieve Cardinality", ParDo.of(RetrieveCardinality.globally()));
+    }
+  }
+
+  /**
+   * Implementation of {@link #perKey()}.
+   *
+   * @param <K>
+   * @param <V>
+   */
+  @AutoValue
+  public abstract static class PerKeyDistinct<K, V>
+      extends PTransform<PCollection<KV<K, V>>, PCollection<KV<K, Long>>> {
+
+    abstract int precision();
+
+    abstract int sparsePrecision();
+
+    abstract Builder<K, V> toBuilder();
+
+    static <K, V> Builder<K, V> builder() {
+      return new AutoValue_ApproximateDistinct_PerKeyDistinct.Builder<K, V>()
+          .setPrecision(12)
+          .setSparsePrecision(0);
+    }
+
+    @AutoValue.Builder
+    abstract static class Builder<K, V> {
+      abstract Builder<K, V> setPrecision(int p);
+
+      abstract Builder<K, V> setSparsePrecision(int sp);
+
+      abstract PerKeyDistinct<K, V> build();
+    }
+
+    public PerKeyDistinct<K, V> withPrecision(int p) {
+      return toBuilder().setPrecision(p).build();
+    }
+
+    public PerKeyDistinct<K, V> withSparsePrecision(int sp) {
+      return toBuilder().setSparsePrecision(sp).build();
+    }
+
+    @Override
+    public PCollection<KV<K, Long>> expand(PCollection<KV<K, V>> input) {
+      KvCoder<K, V> inputCoder = (KvCoder<K, V>) input.getCoder();
+      return input
+          .apply(
+              Combine.<K, V, HyperLogLogPlus>perKey(
+                  ApproximateDistinctFn.<V>create(inputCoder.getValueCoder())
+                      .withPrecision(this.precision())
+                      .withSparseRepresentation(this.sparsePrecision())))
+          .apply("Retrieve Cardinality", ParDo.of(RetrieveCardinality.<K>perKey()));
+    }
+  }
+
+  /**
+   * Implements the {@link CombineFn} of {@link ApproximateDistinct} transforms.
+   *
+   * @param <InputT> the type of the elements in the input {@link PCollection}
+   */
+  public static class ApproximateDistinctFn<InputT>
+      extends CombineFn<InputT, HyperLogLogPlus, HyperLogLogPlus> {
+
+    private final int p;
+
+    private final int sp;
+
+    private final Coder<InputT> inputCoder;
+
+    private ApproximateDistinctFn(int p, int sp, Coder<InputT> coder) {
+      this.p = p;
+      this.sp = sp;
+      inputCoder = coder;
+    }
+
+    /**
+     * Returns an {@link ApproximateDistinctFn} combiner with the given input coder.
+     *
+     * @param coder the coder that encodes the elements' type
+     */
+    public static <InputT> ApproximateDistinctFn<InputT> create(Coder<InputT> coder) {
+      try {
+        coder.verifyDeterministic();
+      } catch (Coder.NonDeterministicException e) {
+        throw new IllegalArgumentException("Coder is not deterministic ! " + e.getMessage(), e);
+      }
+      return new ApproximateDistinctFn<>(12, 0, coder);
+    }
+
+    /**
+     * Returns a new {@link ApproximateDistinctFn} combiner with a new precision {@code p}.
+     *
+     * <p>Keep in mind that {@code p} cannot be lower than 4, because the estimation would be too
+     * inaccurate.
+     *
+     * <p>See {@link ApproximateDistinct#precisionForRelativeError(double)} and {@link
+     * ApproximateDistinct#relativeErrorForPrecision(int)} to have more information about the
+     * relationship between precision and relative error.
+     *
+     * @param p the precision value for the normal representation
+     */
+    public ApproximateDistinctFn<InputT> withPrecision(int p) {
+      checkArgument(p >= 4, "Expected: p >= 4. Actual: p = %s", p);
+      return new ApproximateDistinctFn<>(p, this.sp, this.inputCoder);
+    }
+
+    /**
+     * Returns a new {@link ApproximateDistinctFn} combiner with a sparse representation of
+     * precision {@code sp}.
+     *
+     * <p>Values above 32 are not yet supported by the AddThis version of HyperLogLog+.
+     *
+     * <p>Fore more information about the sparse representation, read Google's paper available <a
+     * href="https://research.google.com/pubs/pub40671.html">here</a>.
+     *
+     * @param sp the precision of HyperLogLog+' sparse representation
+     */
+    public ApproximateDistinctFn<InputT> withSparseRepresentation(int sp) {
+      checkArgument(
+          (sp > this.p && sp < 32) || (sp == 0),
+          "Expected: p <= sp <= 32." + "Actual: p = %s, sp = %s",
+          this.p,
+          sp);
+      return new ApproximateDistinctFn<>(this.p, sp, this.inputCoder);
+    }
+
+    @Override
+    public HyperLogLogPlus createAccumulator() {
+      return new HyperLogLogPlus(p, sp);
+    }
+
+    @Override
+    public HyperLogLogPlus addInput(HyperLogLogPlus acc, InputT record) {
+      try {
+        acc.offer(CoderUtils.encodeToByteArray(inputCoder, record));
+      } catch (CoderException e) {
+        throw new IllegalStateException("The input value cannot be encoded: " + e.getMessage(), e);
+      }
+      return acc;
+    }
+
+    /** Output the whole structure so it can be queried, reused or stored easily. */
+    @Override
+    public HyperLogLogPlus extractOutput(HyperLogLogPlus accumulator) {
+      return accumulator;
+    }
+
+    @Override
+    public HyperLogLogPlus mergeAccumulators(Iterable<HyperLogLogPlus> accumulators) {
+      HyperLogLogPlus mergedAccum = createAccumulator();
+      for (HyperLogLogPlus accum : accumulators) {
+        try {
+          mergedAccum.addAll(accum);
+        } catch (CardinalityMergeException e) {
+          // Should never happen because only HyperLogLogPlus accumulators are instantiated.
+          throw new IllegalStateException(
+              "The accumulators cannot be merged: " + e.getMessage(), e);
+        }
+      }
+      return mergedAccum;
+    }
+
+    @Override
+    public void populateDisplayData(DisplayData.Builder builder) {
+      super.populateDisplayData(builder);
+      builder
+          .add(DisplayData.item("p", p).withLabel("precision"))
+          .add(DisplayData.item("sp", sp).withLabel("sparse representation precision"));
+    }
+  }
+
+  /** Coder for {@link HyperLogLogPlus} class. */
+  public static class HyperLogLogPlusCoder extends CustomCoder<HyperLogLogPlus> {
+
+    private static final HyperLogLogPlusCoder INSTANCE = new HyperLogLogPlusCoder();
+
+    private static final ByteArrayCoder BYTE_ARRAY_CODER = ByteArrayCoder.of();
+
+    public static HyperLogLogPlusCoder of() {
+      return INSTANCE;
+    }
+
+    @Override
+    public void encode(HyperLogLogPlus value, OutputStream outStream) throws IOException {
+      if (value == null) {
+        throw new CoderException("cannot encode a null HyperLogLogPlus sketch");
+      }
+      BYTE_ARRAY_CODER.encode(value.getBytes(), outStream);
+    }
+
+    @Override
+    public HyperLogLogPlus decode(InputStream inStream) throws IOException {
+      return HyperLogLogPlus.Builder.build(BYTE_ARRAY_CODER.decode(inStream));
+    }
+
+    @Override
+    public boolean isRegisterByteSizeObserverCheap(HyperLogLogPlus value) {
+      return true;
+    }
+
+    @Override
+    protected long getEncodedElementByteSize(HyperLogLogPlus value) throws IOException {
+      if (value == null) {
+        throw new CoderException("cannot encode a null HyperLogLogPlus sketch");
+      }
+      return value.sizeof();
+    }
+  }
+
+  /**
+   * Utility class that provides {@link DoFn}s to retrieve the cardinality from a {@link
+   * HyperLogLogPlus} structure in a global or perKey context.
+   */
+  public static class RetrieveCardinality {
+
+    public static <K> DoFn<KV<K, HyperLogLogPlus>, KV<K, Long>> perKey() {
+      return new DoFn<KV<K, HyperLogLogPlus>, KV<K, Long>>() {
+        @ProcessElement
+        public void processElement(ProcessContext c) {
+          KV<K, HyperLogLogPlus> kv = c.element();
+          c.output(KV.of(kv.getKey(), kv.getValue().cardinality()));
+        }
+      };
+    }
+
+    public static DoFn<HyperLogLogPlus, Long> globally() {
+      return new DoFn<HyperLogLogPlus, Long>() {
+        @ProcessElement
+        public void apply(ProcessContext c) {
+          c.output(c.element().cardinality());
+        }
+      };
+    }
+  }
+
+  /**
+   * Computes the precision based on the desired relative error.
+   *
+   * <p>According to the paper, the mean squared error is bounded by the following formula:
+   *
+   * <pre>b(m) / sqrt(m)
+   * Where m is the number of buckets used ({@code p = log2(m)})
+   * and {@code b(m) < 1.106} for {@code m > 16 (and p > 4)}.
+   * </pre>
+   *
+   * <br>
+   * <b>WARNING:</b> <br>
+   * This does not mean relative error in the estimation <b>can't</b> be higher. <br>
+   * This only means that on average the relative error will be lower than the desired relative
+   * error. <br>
+   * Nevertheless, the more elements arrive in the {@link PCollection}, the lower the variation will
+   * be. <br>
+   * Indeed, this is like when you throw a dice millions of time: the relative frequency of each
+   * different result <code>{1,2,3,4,5,6}</code> will get closer to {@code 1/6}.
+   *
+   * @param relativeError the mean squared error should be in the interval ]0,1]
+   * @return the minimum precision p in order to have the desired relative error on average.
+   */
+  public static long precisionForRelativeError(double relativeError) {
+    return Math.round(
+        Math.ceil(Math.log(Math.pow(1.106, 2.0) / Math.pow(relativeError, 2.0)) / Math.log(2)));
+  }
+
+  /**
+   * @param p the precision i.e. the number of bits used for indexing the buckets
+   * @return the Mean squared error of the Estimation of cardinality to expect for the given value
+   *     of p.
+   */
+  public static double relativeErrorForPrecision(int p) {
+    if (p < 4) {
+      return 1.0;
+    }
+    double betaM;
+    switch (p) {
+      case 4:
+        betaM = 1.156;
+        break;
+      case 5:
+        betaM = 1.2;
+        break;
+      case 6:
+        betaM = 1.104;
+        break;
+      case 7:
+        betaM = 1.096;
+        break;
+      default:
+        betaM = 1.05;
+        break;
+    }
+    return betaM / Math.sqrt(Math.exp(p * Math.log(2)));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/fd58a423/sdks/java/extensions/sketching/src/main/java/org/apache/beam/sdk/extensions/sketching/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sketching/src/main/java/org/apache/beam/sdk/extensions/sketching/package-info.java b/sdks/java/extensions/sketching/src/main/java/org/apache/beam/sdk/extensions/sketching/package-info.java
new file mode 100755
index 0000000..2e8d60e
--- /dev/null
+++ b/sdks/java/extensions/sketching/src/main/java/org/apache/beam/sdk/extensions/sketching/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Utilities for computing statistical indicators using probabilistic sketches.
+ */
+package org.apache.beam.sdk.extensions.sketching;

http://git-wip-us.apache.org/repos/asf/beam/blob/fd58a423/sdks/java/extensions/sketching/src/test/java/org/apache/beam/sdk/extensions/sketching/ApproximateDistinctTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sketching/src/test/java/org/apache/beam/sdk/extensions/sketching/ApproximateDistinctTest.java b/sdks/java/extensions/sketching/src/test/java/org/apache/beam/sdk/extensions/sketching/ApproximateDistinctTest.java
new file mode 100644
index 0000000..cdbcc45
--- /dev/null
+++ b/sdks/java/extensions/sketching/src/test/java/org/apache/beam/sdk/extensions/sketching/ApproximateDistinctTest.java
@@ -0,0 +1,209 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.sketching;
+
+import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem;
+import static org.hamcrest.MatcherAssert.assertThat;
+
+import com.clearspring.analytics.stream.cardinality.HyperLogLogPlus;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.avro.Schema;
+import org.apache.avro.SchemaBuilder;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.beam.sdk.coders.AvroCoder;
+import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
+import org.apache.beam.sdk.extensions.sketching.ApproximateDistinct.ApproximateDistinctFn;
+import org.apache.beam.sdk.testing.CoderProperties;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.transforms.Values;
+import org.apache.beam.sdk.transforms.WithKeys;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+import org.apache.beam.sdk.values.PCollection;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Tests for {@link ApproximateDistinct}. */
+@RunWith(JUnit4.class)
+public class ApproximateDistinctTest implements Serializable {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ApproximateDistinctTest.class);
+
+  @Rule public final transient TestPipeline tp = TestPipeline.create();
+
+  @Test
+  public void smallCardinality() {
+    final int smallCard = 1000;
+    final int p = 6;
+    final double expectedErr = 1.104 / Math.sqrt(p);
+
+    List<Integer> small = new ArrayList<>();
+    for (int i = 0; i < smallCard; i++) {
+      small.add(i);
+    }
+
+    PCollection<Long> cardinality =
+        tp.apply("small stream", Create.<Integer>of(small))
+            .apply("small cardinality", ApproximateDistinct.<Integer>globally().withPrecision(p));
+
+    PAssert.that("Not Accurate Enough", cardinality)
+        .satisfies(new VerifyAccuracy(smallCard, expectedErr));
+
+    tp.run();
+  }
+
+  @Test
+  public void bigCardinality() {
+    final int cardinality = 15000;
+    final int p = 15;
+    final int sp = 20;
+    final double expectedErr = 1.04 / Math.sqrt(p);
+
+    List<Integer> stream = new ArrayList<>();
+    for (int i = 1; i <= cardinality; i++) {
+      stream.addAll(Collections.nCopies(2, i));
+    }
+    Collections.shuffle(stream);
+
+    PCollection<Long> res =
+        tp.apply("big stream", Create.<Integer>of(stream))
+            .apply(
+                "big cardinality",
+                ApproximateDistinct.<Integer>globally().withPrecision(p).withSparsePrecision(sp));
+
+    PAssert.that("Verify Accuracy for big cardinality", res)
+        .satisfies(new VerifyAccuracy(cardinality, expectedErr));
+
+    tp.run();
+  }
+
+  @Test
+  public void perKey() {
+    final int cardinality = 1000;
+    final int p = 15;
+    final double expectedErr = 1.04 / Math.sqrt(p);
+
+    List<Integer> stream = new ArrayList<>();
+    for (int i = 1; i <= cardinality; i++) {
+      stream.addAll(Collections.nCopies(2, i));
+    }
+    Collections.shuffle(stream);
+
+    PCollection<Long> results =
+        tp.apply("per key stream", Create.of(stream))
+            .apply("create keys", WithKeys.<Integer, Integer>of(1))
+            .apply(
+                "per key cardinality",
+                ApproximateDistinct.<Integer, Integer>perKey().withPrecision(p))
+            .apply("extract values", Values.<Long>create());
+
+    PAssert.that("Verify Accuracy for cardinality per key", results)
+        .satisfies(new VerifyAccuracy(cardinality, expectedErr));
+
+    tp.run();
+  }
+
+  @Test
+  public void customObject() {
+    final int cardinality = 500;
+    final int p = 15;
+    final double expectedErr = 1.04 / Math.sqrt(p);
+
+    Schema schema =
+        SchemaBuilder.record("User")
+            .fields()
+            .requiredString("Pseudo")
+            .requiredInt("Age")
+            .endRecord();
+    List<GenericRecord> users = new ArrayList<>();
+    for (int i = 1; i <= cardinality; i++) {
+      GenericData.Record newRecord = new GenericData.Record(schema);
+      newRecord.put("Pseudo", "User" + i);
+      newRecord.put("Age", i);
+      users.add(newRecord);
+    }
+    PCollection<Long> results =
+        tp.apply("Create stream", Create.of(users).withCoder(AvroCoder.of(schema)))
+            .apply(
+                "Test custom object",
+                ApproximateDistinct.<GenericRecord>globally().withPrecision(p));
+
+    PAssert.that("Verify Accuracy for custom object", results)
+        .satisfies(new VerifyAccuracy(cardinality, expectedErr));
+
+    tp.run();
+  }
+
+  @Test
+  public void testCoder() throws Exception {
+    HyperLogLogPlus hllp = new HyperLogLogPlus(12, 18);
+    for (int i = 0; i < 10; i++) {
+      hllp.offer(i);
+    }
+    CoderProperties.<HyperLogLogPlus>coderDecodeEncodeEqual(
+        ApproximateDistinct.HyperLogLogPlusCoder.of(), hllp);
+  }
+
+  @Test
+  public void testDisplayData() {
+    final ApproximateDistinctFn<Integer> fnWithPrecision =
+        ApproximateDistinctFn.create(BigEndianIntegerCoder.of()).withPrecision(23);
+
+    assertThat(DisplayData.from(fnWithPrecision), hasDisplayItem("p", 23));
+    assertThat(DisplayData.from(fnWithPrecision), hasDisplayItem("sp", 0));
+  }
+
+  class VerifyAccuracy implements SerializableFunction<Iterable<Long>, Void> {
+
+    private final int expectedCard;
+
+    private final double expectedError;
+
+    VerifyAccuracy(int expectedCard, double expectedError) {
+      this.expectedCard = expectedCard;
+      this.expectedError = expectedError;
+    }
+
+    @Override
+    public Void apply(Iterable<Long> input) {
+      for (Long estimate : input) {
+        boolean isAccurate = Math.abs(estimate - expectedCard) / expectedCard < expectedError;
+        Assert.assertTrue(
+            "not accurate enough : \nExpected Cardinality : "
+                + expectedCard
+                + "\nComputed Cardinality : "
+                + estimate,
+            isAccurate);
+      }
+      return null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/fd58a423/sdks/java/javadoc/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/javadoc/pom.xml b/sdks/java/javadoc/pom.xml
index 79ac933..85440ff 100644
--- a/sdks/java/javadoc/pom.xml
+++ b/sdks/java/javadoc/pom.xml
@@ -94,6 +94,11 @@
 
     <dependency>
       <groupId>org.apache.beam</groupId>
+      <artifactId>beam-sdks-java-extensions-sketching</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.beam</groupId>
       <artifactId>beam-sdks-java-extensions-sorter</artifactId>
     </dependency>
 


[21/50] [abbrv] beam git commit: This closes #4070

Posted by ke...@apache.org.
This closes #4070


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

Branch: refs/heads/mr-runner
Commit: fbcb0ba6826b1e2016acfce138884ab9dd398f86
Parents: 4925478 f0b2b3c
Author: Ahmet Altay <al...@google.com>
Authored: Thu Nov 2 16:38:07 2017 -0700
Committer: Ahmet Altay <al...@google.com>
Committed: Thu Nov 2 16:38:07 2017 -0700

----------------------------------------------------------------------
 sdks/python/setup.py | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------



[07/50] [abbrv] beam git commit: This closes #4067

Posted by ke...@apache.org.
This closes #4067


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

Branch: refs/heads/mr-runner
Commit: 8247b534613eabed7fff4b62c6afc6603b84031c
Parents: b013d7c ac43634
Author: chamikara@google.com <ch...@google.com>
Authored: Wed Nov 1 13:04:58 2017 -0700
Committer: chamikara@google.com <ch...@google.com>
Committed: Wed Nov 1 13:04:58 2017 -0700

----------------------------------------------------------------------
 .../examples/cookbook/datastore_wordcount.py    | 24 +++++++++++---------
 .../io/gcp/datastore/v1/datastoreio.py          | 16 +++++++++++--
 2 files changed, 27 insertions(+), 13 deletions(-)
----------------------------------------------------------------------



[38/50] [abbrv] beam git commit: Migrate shared Fn Execution code to Java7

Posted by ke...@apache.org.
Migrate shared Fn Execution code to Java7


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

Branch: refs/heads/mr-runner
Commit: 012c2e64ceb691fd69649ba8cc02d9a2f16e519f
Parents: 766b4f3
Author: Thomas Groh <tg...@google.com>
Authored: Wed Oct 18 11:26:48 2017 -0700
Committer: Thomas Groh <tg...@google.com>
Committed: Mon Nov 6 16:06:57 2017 -0800

----------------------------------------------------------------------
 runners/java-fn-execution/pom.xml               | 14 ------
 .../runners/fnexecution/ServerFactoryTest.java  | 45 ++++++++++++++-----
 runners/pom.xml                                 |  2 +-
 sdks/java/fn-execution/pom.xml                  | 20 ++-------
 .../org/apache/beam/harness/test/Consumer.java  | 26 +++++++++++
 .../org/apache/beam/harness/test/Supplier.java  | 26 +++++++++++
 .../apache/beam/harness/test/TestExecutors.java | 12 ++++-
 .../beam/harness/test/TestExecutorsTest.java    | 29 +++++++++---
 .../apache/beam/harness/test/TestStreams.java   | 35 ++++++++++++---
 .../beam/harness/test/TestStreamsTest.java      | 47 +++++++++++++++-----
 .../apache/beam/fn/harness/FnHarnessTest.java   |  4 +-
 .../harness/data/BeamFnDataGrpcClientTest.java  |  5 ++-
 .../stream/BufferingStreamObserverTest.java     |  2 +-
 .../stream/DirectStreamObserverTest.java        |  2 +-
 sdks/java/pom.xml                               |  2 +-
 15 files changed, 196 insertions(+), 75 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/012c2e64/runners/java-fn-execution/pom.xml
----------------------------------------------------------------------
diff --git a/runners/java-fn-execution/pom.xml b/runners/java-fn-execution/pom.xml
index bd4fcf0..f57c58b 100644
--- a/runners/java-fn-execution/pom.xml
+++ b/runners/java-fn-execution/pom.xml
@@ -32,20 +32,6 @@
 
   <packaging>jar</packaging>
 
-  <build>
-    <plugins>
-      <plugin>
-        <!--  Override Beam parent to allow Java8 -->
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-compiler-plugin</artifactId>
-        <configuration>
-          <source>1.8</source>
-          <target>1.8</target>
-        </configuration>
-      </plugin>
-    </plugins>
-  </build>
-
   <dependencies>
     <dependency>
       <groupId>org.apache.beam</groupId>

http://git-wip-us.apache.org/repos/asf/beam/blob/012c2e64/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/ServerFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/ServerFactoryTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/ServerFactoryTest.java
index aa8d246..b78e88a 100644
--- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/ServerFactoryTest.java
+++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/ServerFactoryTest.java
@@ -39,6 +39,7 @@ import java.util.Collection;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.LinkedBlockingQueue;
 import org.apache.beam.harness.channel.ManagedChannelFactory;
+import org.apache.beam.harness.test.Consumer;
 import org.apache.beam.harness.test.TestStreams;
 import org.apache.beam.model.fnexecution.v1.BeamFnApi;
 import org.apache.beam.model.fnexecution.v1.BeamFnApi.Elements;
@@ -74,24 +75,48 @@ public class ServerFactoryTest {
     Endpoints.ApiServiceDescriptor.Builder apiServiceDescriptorBuilder =
         Endpoints.ApiServiceDescriptor.newBuilder();
 
-    Collection<Elements> serverElements = new ArrayList<>();
-    CountDownLatch clientHangedUp = new CountDownLatch(1);
+    final Collection<Elements> serverElements = new ArrayList<>();
+    final CountDownLatch clientHangedUp = new CountDownLatch(1);
     CallStreamObserver<Elements> serverInboundObserver =
-        TestStreams.withOnNext(serverElements::add)
-        .withOnCompleted(clientHangedUp::countDown)
-        .build();
+        TestStreams.withOnNext(
+                new Consumer<Elements>() {
+                  @Override
+                  public void accept(Elements item) {
+                    serverElements.add(item);
+                  }
+                })
+            .withOnCompleted(
+                new Runnable() {
+                  @Override
+                  public void run() {
+                    clientHangedUp.countDown();
+                  }
+                })
+            .build();
     TestDataService service = new TestDataService(serverInboundObserver);
     Server server = serverFactory.allocatePortAndCreate(service, apiServiceDescriptorBuilder);
     assertFalse(server.isShutdown());
 
     ManagedChannel channel = channelFactory.forDescriptor(apiServiceDescriptorBuilder.build());
     BeamFnDataGrpc.BeamFnDataStub stub = BeamFnDataGrpc.newStub(channel);
-    Collection<BeamFnApi.Elements> clientElements = new ArrayList<>();
-    CountDownLatch serverHangedUp = new CountDownLatch(1);
+    final Collection<BeamFnApi.Elements> clientElements = new ArrayList<>();
+    final CountDownLatch serverHangedUp = new CountDownLatch(1);
     CallStreamObserver<BeamFnApi.Elements> clientInboundObserver =
-        TestStreams.withOnNext(clientElements::add)
-        .withOnCompleted(serverHangedUp::countDown)
-        .build();
+        TestStreams.withOnNext(
+                new Consumer<Elements>() {
+                  @Override
+                  public void accept(Elements item) {
+                    clientElements.add(item);
+                  }
+                })
+            .withOnCompleted(
+                new Runnable() {
+                  @Override
+                  public void run() {
+                    serverHangedUp.countDown();
+                  }
+                })
+            .build();
 
     StreamObserver<Elements> clientOutboundObserver = stub.data(clientInboundObserver);
     StreamObserver<BeamFnApi.Elements> serverOutboundObserver = service.outboundObservers.take();

http://git-wip-us.apache.org/repos/asf/beam/blob/012c2e64/runners/pom.xml
----------------------------------------------------------------------
diff --git a/runners/pom.xml b/runners/pom.xml
index df3faa9..47f3c0e 100644
--- a/runners/pom.xml
+++ b/runners/pom.xml
@@ -35,6 +35,7 @@
   <modules>
     <module>core-construction-java</module>
     <module>core-java</module>
+    <module>java-fn-execution</module>
     <module>local-artifact-service-java</module>
     <module>reference</module>
     <module>direct-java</module>
@@ -63,7 +64,6 @@
         <jdk>[1.8,)</jdk>
       </activation>
       <modules>
-        <module>java-fn-execution</module>
         <module>gearpump</module>
       </modules>
     </profile>

http://git-wip-us.apache.org/repos/asf/beam/blob/012c2e64/sdks/java/fn-execution/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/fn-execution/pom.xml b/sdks/java/fn-execution/pom.xml
index 9929c29..7c203eb 100644
--- a/sdks/java/fn-execution/pom.xml
+++ b/sdks/java/fn-execution/pom.xml
@@ -27,27 +27,13 @@
   </parent>
 
   <artifactId>beam-sdks-java-fn-execution</artifactId>
-  <name>Apache Beam :: SDKs :: Java :: Harness Core</name>
-  <description>Contains code shared across the Beam Java SDK Harness and the Java Runner Harness
-    libraries.
+  <name>Apache Beam :: SDKs :: Java :: Fn Execution</name>
+  <description>Contains code shared across the Beam Java SDK Harness Java Runners to execute using
+    the Beam Portability Framework
   </description>
 
   <packaging>jar</packaging>
 
-  <build>
-    <plugins>
-      <plugin>
-        <!--  Override Beam parent to allow Java8 -->
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-compiler-plugin</artifactId>
-        <configuration>
-          <source>1.8</source>
-          <target>1.8</target>
-        </configuration>
-      </plugin>
-    </plugins>
-  </build>
-
   <dependencies>
     <dependency>
       <groupId>org.apache.beam</groupId>

http://git-wip-us.apache.org/repos/asf/beam/blob/012c2e64/sdks/java/fn-execution/src/test/java/org/apache/beam/harness/test/Consumer.java
----------------------------------------------------------------------
diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/harness/test/Consumer.java b/sdks/java/fn-execution/src/test/java/org/apache/beam/harness/test/Consumer.java
new file mode 100644
index 0000000..279fc29
--- /dev/null
+++ b/sdks/java/fn-execution/src/test/java/org/apache/beam/harness/test/Consumer.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.harness.test;
+
+/**
+ * A fork of the Java 8 consumer interface. This exists to enable migration for existing consumers.
+ */
+public interface Consumer<T> {
+  void accept(T item);
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/012c2e64/sdks/java/fn-execution/src/test/java/org/apache/beam/harness/test/Supplier.java
----------------------------------------------------------------------
diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/harness/test/Supplier.java b/sdks/java/fn-execution/src/test/java/org/apache/beam/harness/test/Supplier.java
new file mode 100644
index 0000000..629afc2
--- /dev/null
+++ b/sdks/java/fn-execution/src/test/java/org/apache/beam/harness/test/Supplier.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.harness.test;
+
+/**
+ * A fork of the Java 8 Supplier interface, to enable migrations.
+ */
+public interface Supplier<T> {
+  T get();
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/012c2e64/sdks/java/fn-execution/src/test/java/org/apache/beam/harness/test/TestExecutors.java
----------------------------------------------------------------------
diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/harness/test/TestExecutors.java b/sdks/java/fn-execution/src/test/java/org/apache/beam/harness/test/TestExecutors.java
index d818a61..ca12d5a 100644
--- a/sdks/java/fn-execution/src/test/java/org/apache/beam/harness/test/TestExecutors.java
+++ b/sdks/java/fn-execution/src/test/java/org/apache/beam/harness/test/TestExecutors.java
@@ -21,7 +21,6 @@ package org.apache.beam.harness.test;
 import com.google.common.util.concurrent.ForwardingExecutorService;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.TimeUnit;
-import java.util.function.Supplier;
 import org.junit.rules.TestRule;
 import org.junit.runner.Description;
 import org.junit.runners.model.Statement;
@@ -31,6 +30,15 @@ import org.junit.runners.model.Statement;
  * allows for testing that tasks have exercised the appropriate shutdown logic.
  */
 public class TestExecutors {
+  public static TestExecutorService from(final ExecutorService staticExecutorService) {
+    return from(new Supplier<ExecutorService>() {
+      @Override
+      public ExecutorService get() {
+        return staticExecutorService;
+      }
+    });
+  }
+
   public static TestExecutorService from(Supplier<ExecutorService> executorServiceSuppler) {
     return new FromSupplier(executorServiceSuppler);
   }
@@ -48,7 +56,7 @@ public class TestExecutors {
     }
 
     @Override
-    public Statement apply(Statement statement, Description arg1) {
+    public Statement apply(final Statement statement, Description arg1) {
       return new Statement() {
         @Override
         public void evaluate() throws Throwable {

http://git-wip-us.apache.org/repos/asf/beam/blob/012c2e64/sdks/java/fn-execution/src/test/java/org/apache/beam/harness/test/TestExecutorsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/harness/test/TestExecutorsTest.java b/sdks/java/fn-execution/src/test/java/org/apache/beam/harness/test/TestExecutorsTest.java
index 1381b55..f0c98e0 100644
--- a/sdks/java/fn-execution/src/test/java/org/apache/beam/harness/test/TestExecutorsTest.java
+++ b/sdks/java/fn-execution/src/test/java/org/apache/beam/harness/test/TestExecutorsTest.java
@@ -38,14 +38,19 @@ public class TestExecutorsTest {
   @Test
   public void testSuccessfulTermination() throws Throwable {
     ExecutorService service = Executors.newSingleThreadExecutor();
-    final TestExecutorService testService = TestExecutors.from(() -> service);
+    final TestExecutorService testService = TestExecutors.from(service);
     final AtomicBoolean taskRan = new AtomicBoolean();
     testService
         .apply(
             new Statement() {
               @Override
               public void evaluate() throws Throwable {
-                testService.submit(() -> taskRan.set(true));
+                testService.submit(new Runnable() {
+                  @Override
+                  public void run() {
+                    taskRan.set(true);
+                  }
+                });
               }
             },
             null)
@@ -57,7 +62,7 @@ public class TestExecutorsTest {
   @Test
   public void testTaskBlocksForeverCausesFailure() throws Throwable {
     ExecutorService service = Executors.newSingleThreadExecutor();
-    final TestExecutorService testService = TestExecutors.from(() -> service);
+    final TestExecutorService testService = TestExecutors.from(service);
     final AtomicBoolean taskStarted = new AtomicBoolean();
     final AtomicBoolean taskWasInterrupted = new AtomicBoolean();
     try {
@@ -66,7 +71,12 @@ public class TestExecutorsTest {
               new Statement() {
                 @Override
                 public void evaluate() throws Throwable {
-                  testService.submit(this::taskToRun);
+                  testService.submit(new Runnable() {
+                    @Override
+                    public void run() {
+                      taskToRun();
+                    }
+                  });
                 }
 
                 private void taskToRun() {
@@ -94,7 +104,7 @@ public class TestExecutorsTest {
   @Test
   public void testStatementFailurePropagatedCleanly() throws Throwable {
     ExecutorService service = Executors.newSingleThreadExecutor();
-    final TestExecutorService testService = TestExecutors.from(() -> service);
+    final TestExecutorService testService = TestExecutors.from(service);
     final RuntimeException exceptionToThrow = new RuntimeException();
     try {
       testService
@@ -118,7 +128,7 @@ public class TestExecutorsTest {
   public void testStatementFailurePropagatedWhenExecutorServiceFailingToTerminate()
       throws Throwable {
     ExecutorService service = Executors.newSingleThreadExecutor();
-    final TestExecutorService testService = TestExecutors.from(() -> service);
+    final TestExecutorService testService = TestExecutors.from(service);
     final AtomicBoolean taskStarted = new AtomicBoolean();
     final AtomicBoolean taskWasInterrupted = new AtomicBoolean();
     final RuntimeException exceptionToThrow = new RuntimeException();
@@ -128,7 +138,12 @@ public class TestExecutorsTest {
               new Statement() {
                 @Override
                 public void evaluate() throws Throwable {
-                  testService.submit(this::taskToRun);
+                  testService.submit(new Runnable() {
+                    @Override
+                    public void run() {
+                      taskToRun();
+                    }
+                  });
                   throw exceptionToThrow;
                 }
 

http://git-wip-us.apache.org/repos/asf/beam/blob/012c2e64/sdks/java/fn-execution/src/test/java/org/apache/beam/harness/test/TestStreams.java
----------------------------------------------------------------------
diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/harness/test/TestStreams.java b/sdks/java/fn-execution/src/test/java/org/apache/beam/harness/test/TestStreams.java
index a7b362d..3df743a 100644
--- a/sdks/java/fn-execution/src/test/java/org/apache/beam/harness/test/TestStreams.java
+++ b/sdks/java/fn-execution/src/test/java/org/apache/beam/harness/test/TestStreams.java
@@ -20,8 +20,6 @@ package org.apache.beam.harness.test;
 
 import io.grpc.stub.CallStreamObserver;
 import io.grpc.stub.StreamObserver;
-import java.util.function.Consumer;
-import java.util.function.Supplier;
 
 /** Utility methods which enable testing of {@link StreamObserver}s. */
 public class TestStreams {
@@ -32,9 +30,9 @@ public class TestStreams {
   public static <T> Builder<T> withOnNext(Consumer<T> onNext) {
     return new Builder<>(new ForwardingCallStreamObserver<>(
         onNext,
-        TestStreams::noop,
-        TestStreams::noop,
-        TestStreams::returnTrue));
+        TestStreams.<Throwable>noopConsumer(),
+        TestStreams.noopRunnable(),
+        TestStreams.alwaysTrueSupplier()));
   }
 
   /** A builder for a test {@link CallStreamObserver} that performs various callbacks. */
@@ -72,7 +70,7 @@ public class TestStreams {
      * Returns a new {@link Builder} like this one with the specified
      * {@link StreamObserver#onError} callback.
      */
-    public Builder<T> withOnError(Runnable onError) {
+    public Builder<T> withOnError(final Runnable onError) {
       return new Builder<>(new ForwardingCallStreamObserver<>(
           observer.onNext,
           new Consumer<Throwable>() {
@@ -102,13 +100,38 @@ public class TestStreams {
   private static void noop() {
   }
 
+  private static Runnable noopRunnable() {
+    return new Runnable() {
+      @Override
+      public void run() {
+      }
+    };
+  }
+
   private static void noop(Throwable t) {
   }
 
+  private static <T> Consumer<T> noopConsumer() {
+    return new Consumer<T>() {
+      @Override
+      public void accept(T item) {
+      }
+    };
+  }
+
   private static boolean returnTrue() {
     return true;
   }
 
+  private static Supplier<Boolean> alwaysTrueSupplier() {
+    return new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        return true;
+      }
+    };
+  }
+
   /** A {@link CallStreamObserver} which executes the supplied callbacks. */
   private static class ForwardingCallStreamObserver<T> extends CallStreamObserver<T> {
     private final Consumer<T> onNext;

http://git-wip-us.apache.org/repos/asf/beam/blob/012c2e64/sdks/java/fn-execution/src/test/java/org/apache/beam/harness/test/TestStreamsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/harness/test/TestStreamsTest.java b/sdks/java/fn-execution/src/test/java/org/apache/beam/harness/test/TestStreamsTest.java
index f5741ae..c578397 100644
--- a/sdks/java/fn-execution/src/test/java/org/apache/beam/harness/test/TestStreamsTest.java
+++ b/sdks/java/fn-execution/src/test/java/org/apache/beam/harness/test/TestStreamsTest.java
@@ -18,7 +18,6 @@
 
 package org.apache.beam.harness.test;
 
-import static org.hamcrest.Matchers.contains;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
@@ -26,6 +25,7 @@ import static org.junit.Assert.assertTrue;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.concurrent.atomic.AtomicBoolean;
+import org.hamcrest.Matchers;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
@@ -35,8 +35,13 @@ import org.junit.runners.JUnit4;
 public class TestStreamsTest {
   @Test
   public void testOnNextIsCalled() {
-    AtomicBoolean onNextWasCalled = new AtomicBoolean();
-    TestStreams.withOnNext(onNextWasCalled::set).build().onNext(true);
+    final AtomicBoolean onNextWasCalled = new AtomicBoolean();
+    TestStreams.withOnNext(new Consumer<Boolean>() {
+      @Override
+      public void accept(Boolean item) {
+        onNextWasCalled.set(item);
+      }
+    }).build().onNext(true);
     assertTrue(onNextWasCalled.get());
   }
 
@@ -44,7 +49,12 @@ public class TestStreamsTest {
   public void testIsReadyIsCalled() {
     final AtomicBoolean isReadyWasCalled = new AtomicBoolean();
     assertFalse(TestStreams.withOnNext(null)
-        .withIsReady(() -> isReadyWasCalled.getAndSet(true))
+        .withIsReady(new Supplier<Boolean>() {
+          @Override
+          public Boolean get() {
+            return isReadyWasCalled.getAndSet(true);
+          }
+        })
         .build()
         .isReady());
     assertTrue(isReadyWasCalled.get());
@@ -52,9 +62,14 @@ public class TestStreamsTest {
 
   @Test
   public void testOnCompletedIsCalled() {
-    AtomicBoolean onCompletedWasCalled = new AtomicBoolean();
+    final AtomicBoolean onCompletedWasCalled = new AtomicBoolean();
     TestStreams.withOnNext(null)
-        .withOnCompleted(() -> onCompletedWasCalled.set(true))
+        .withOnCompleted(new Runnable() {
+          @Override
+          public void run() {
+            onCompletedWasCalled.set(true);
+          }
+        })
         .build()
         .onCompleted();
     assertTrue(onCompletedWasCalled.get());
@@ -63,9 +78,14 @@ public class TestStreamsTest {
   @Test
   public void testOnErrorRunnableIsCalled() {
     RuntimeException throwable = new RuntimeException();
-    AtomicBoolean onErrorWasCalled = new AtomicBoolean();
+    final AtomicBoolean onErrorWasCalled = new AtomicBoolean();
     TestStreams.withOnNext(null)
-        .withOnError(() -> onErrorWasCalled.set(true))
+        .withOnError(new Runnable() {
+          @Override
+          public void run() {
+            onErrorWasCalled.set(true);
+          }
+        })
         .build()
         .onError(throwable);
     assertTrue(onErrorWasCalled.get());
@@ -74,11 +94,16 @@ public class TestStreamsTest {
   @Test
   public void testOnErrorConsumerIsCalled() {
     RuntimeException throwable = new RuntimeException();
-    Collection<Throwable> onErrorWasCalled = new ArrayList<>();
+    final Collection<Throwable> onErrorWasCalled = new ArrayList<>();
     TestStreams.withOnNext(null)
-        .withOnError(onErrorWasCalled::add)
+        .withOnError(new Consumer<Throwable>() {
+          @Override
+          public void accept(Throwable item) {
+            onErrorWasCalled.add(item);
+          }
+        })
         .build()
         .onError(throwable);
-    assertThat(onErrorWasCalled, contains(throwable));
+    assertThat(onErrorWasCalled, Matchers.<Throwable>contains(throwable));
   }
 }

http://git-wip-us.apache.org/repos/asf/beam/blob/012c2e64/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnHarnessTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnHarnessTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnHarnessTest.java
index 66c31a8..c926414 100644
--- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnHarnessTest.java
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnHarnessTest.java
@@ -28,7 +28,7 @@ import io.grpc.stub.StreamObserver;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.CountDownLatch;
-import java.util.function.Consumer;
+import org.apache.beam.harness.test.Consumer;
 import org.apache.beam.harness.test.TestStreams;
 import org.apache.beam.model.fnexecution.v1.BeamFnApi;
 import org.apache.beam.model.fnexecution.v1.BeamFnApi.InstructionRequest;
@@ -91,7 +91,7 @@ public class FnHarnessTest {
             responseObserver.onCompleted();
           }
         });
-        return TestStreams.withOnNext(new Consumer<BeamFnApi.InstructionResponse>() {
+        return TestStreams.withOnNext(new Consumer<InstructionResponse>() {
           @Override
           public void accept(InstructionResponse t) {
             instructionResponses.add(t);

http://git-wip-us.apache.org/repos/asf/beam/blob/012c2e64/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataGrpcClientTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataGrpcClientTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataGrpcClientTest.java
index 7df8925..9e21398 100644
--- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataGrpcClientTest.java
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataGrpcClientTest.java
@@ -41,12 +41,13 @@ import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
-import java.util.function.Consumer;
 import java.util.function.Function;
 import org.apache.beam.fn.harness.fn.CloseableThrowingConsumer;
 import org.apache.beam.fn.harness.fn.ThrowingConsumer;
+import org.apache.beam.harness.test.Consumer;
 import org.apache.beam.harness.test.TestStreams;
 import org.apache.beam.model.fnexecution.v1.BeamFnApi;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi.Elements;
 import org.apache.beam.model.fnexecution.v1.BeamFnDataGrpc;
 import org.apache.beam.model.pipeline.v1.Endpoints;
 import org.apache.beam.sdk.coders.Coder;
@@ -263,7 +264,7 @@ public class BeamFnDataGrpcClientTest {
     Collection<BeamFnApi.Elements> inboundServerValues = new ConcurrentLinkedQueue<>();
     CallStreamObserver<BeamFnApi.Elements> inboundServerObserver =
         TestStreams.withOnNext(
-            new Consumer<BeamFnApi.Elements>() {
+            new Consumer<Elements>() {
               @Override
               public void accept(BeamFnApi.Elements t) {
                 inboundServerValues.add(t);

http://git-wip-us.apache.org/repos/asf/beam/blob/012c2e64/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/stream/BufferingStreamObserverTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/stream/BufferingStreamObserverTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/stream/BufferingStreamObserverTest.java
index 3f66c4c..96648e9 100644
--- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/stream/BufferingStreamObserverTest.java
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/stream/BufferingStreamObserverTest.java
@@ -31,7 +31,7 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.function.Consumer;
+import org.apache.beam.harness.test.Consumer;
 import org.apache.beam.harness.test.TestExecutors;
 import org.apache.beam.harness.test.TestExecutors.TestExecutorService;
 import org.apache.beam.harness.test.TestStreams;

http://git-wip-us.apache.org/repos/asf/beam/blob/012c2e64/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/stream/DirectStreamObserverTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/stream/DirectStreamObserverTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/stream/DirectStreamObserverTest.java
index 120a73d..05d8d5a 100644
--- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/stream/DirectStreamObserverTest.java
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/stream/DirectStreamObserverTest.java
@@ -31,7 +31,7 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.function.Consumer;
+import org.apache.beam.harness.test.Consumer;
 import org.apache.beam.harness.test.TestExecutors;
 import org.apache.beam.harness.test.TestExecutors.TestExecutorService;
 import org.apache.beam.harness.test.TestStreams;

http://git-wip-us.apache.org/repos/asf/beam/blob/012c2e64/sdks/java/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/pom.xml b/sdks/java/pom.xml
index 62e4ec3..c6ab234 100644
--- a/sdks/java/pom.xml
+++ b/sdks/java/pom.xml
@@ -40,6 +40,7 @@
     <module>io</module>
     <module>maven-archetypes</module>
     <module>extensions</module>
+    <module>fn-execution</module>
     <!-- javadoc runs directly from the root parent as the last module
          in the build to be able to capture runner-specific javadoc.
     <module>javadoc</module> -->
@@ -53,7 +54,6 @@
         <jdk>[1.8,)</jdk>
       </activation>
       <modules>
-        <module>fn-execution</module>
         <module>harness</module>
         <module>container</module>
         <module>java8tests</module>


[44/50] [abbrv] beam git commit: This closes #3729

Posted by ke...@apache.org.
This closes #3729


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

Branch: refs/heads/mr-runner
Commit: 3dfcb44475c92cea85cdceebe1d0cce0ca0e31cf
Parents: 35952f6 227801b
Author: chamikara@google.com <ch...@google.com>
Authored: Mon Nov 6 17:34:54 2017 -0800
Committer: chamikara@google.com <ch...@google.com>
Committed: Mon Nov 6 17:34:54 2017 -0800

----------------------------------------------------------------------
 .../sdk/io/gcp/spanner/NaiveSpannerReadFn.java  |   1 +
 .../sdk/io/gcp/spanner/SerializedMutation.java  |  35 ++
 .../io/gcp/spanner/SerializedMutationCoder.java |  60 +++
 .../beam/sdk/io/gcp/spanner/SpannerIO.java      | 309 ++++++++++++-
 .../sdk/io/gcp/spanner/SpannerWriteGroupFn.java | 133 ------
 .../sdk/io/gcp/spanner/SpannerIOWriteTest.java  | 447 ++++++++++++++-----
 .../beam/sdk/io/gcp/spanner/SpannerWriteIT.java |   5 +-
 7 files changed, 746 insertions(+), 244 deletions(-)
----------------------------------------------------------------------



[17/50] [abbrv] beam git commit: [BEAM-2530] Updating to AutoValue 1.5.1 to support Java 9.

Posted by ke...@apache.org.
[BEAM-2530] Updating to AutoValue 1.5.1 to support Java 9.

This closes #4035


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

Branch: refs/heads/mr-runner
Commit: 482d17889f379512c8a7ac97e6d27aaeca00aca2
Parents: 5aaff7b a63345b
Author: Luke Cwik <lc...@google.com>
Authored: Thu Nov 2 14:06:18 2017 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Thu Nov 2 14:06:18 2017 -0700

----------------------------------------------------------------------
 examples/java/pom.xml     | 6 ++++++
 pom.xml                   | 8 +++++++-
 sdks/java/nexmark/pom.xml | 6 ++++++
 3 files changed, 19 insertions(+), 1 deletion(-)
----------------------------------------------------------------------



[41/50] [abbrv] beam git commit: Fix Repackaging Configuration in the the DirectRunner

Posted by ke...@apache.org.
Fix Repackaging Configuration in the the DirectRunner

Repackage the actually generated model classes rather than the previous
namespace.


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

Branch: refs/heads/mr-runner
Commit: bd1586fa83608590502b251814e523b054bdd378
Parents: 7550a58
Author: Thomas Groh <tg...@google.com>
Authored: Tue Oct 31 14:23:58 2017 -0700
Committer: Thomas Groh <tg...@google.com>
Committed: Mon Nov 6 16:48:08 2017 -0800

----------------------------------------------------------------------
 runners/direct-java/pom.xml | 18 ++----------------
 1 file changed, 2 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/bd1586fa/runners/direct-java/pom.xml
----------------------------------------------------------------------
diff --git a/runners/direct-java/pom.xml b/runners/direct-java/pom.xml
index 6e356fc..752af44 100644
--- a/runners/direct-java/pom.xml
+++ b/runners/direct-java/pom.xml
@@ -82,23 +82,9 @@
                     </shadedPattern>
                   </relocation>
                   <relocation>
-                    <pattern>org.apache.beam.sdk.common</pattern>
+                    <pattern>org.apache.beam.model</pattern>
                     <shadedPattern>
-                      org.apache.beam.runners.direct.repackaged.sdk.common
-                    </shadedPattern>
-                  </relocation>
-                  <relocation>
-                    <!-- portability protos currently contain a typo, to be fixed later -->
-                    <pattern>org.apache.beam.sdks.common</pattern>
-                    <shadedPattern>
-                      org.apache.beam.runners.direct.repackaged.sdks.common
-                    </shadedPattern>
-                  </relocation>
-                  <relocation>
-                    <!-- portability protos also invent this namespace -->
-                    <pattern>org.apache.beam.portability</pattern>
-                    <shadedPattern>
-                      org.apache.beam.runners.direct.repackaged.portability
+                      org.apache.beam.runners.direct.repackaged.model
                     </shadedPattern>
                   </relocation>
                   <relocation>


[15/50] [abbrv] beam git commit: This closes #4072

Posted by ke...@apache.org.
This closes #4072


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

Branch: refs/heads/mr-runner
Commit: 5aaff7b1b2679e9298e805a0c614b74d2f5425d9
Parents: ee2292a d7329a0
Author: Thomas Groh <tg...@google.com>
Authored: Thu Nov 2 09:54:40 2017 -0700
Committer: Thomas Groh <tg...@google.com>
Committed: Thu Nov 2 09:54:40 2017 -0700

----------------------------------------------------------------------
 .../extensions/sql/impl/utils/SqlTypeUtils.java    | 17 +++++++++++++++++
 1 file changed, 17 insertions(+)
----------------------------------------------------------------------