You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by lc...@apache.org on 2017/01/30 20:48:52 UTC

[1/6] beam git commit: A proposal for a portability framework to execute user definable functions.

Repository: beam
Updated Branches:
  refs/heads/master 582c4a8a4 -> 343176c00


http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/sdks/java/harness/src/test/java/org/apache/beam/runners/core/BoundedSourceRunnerTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/test/java/org/apache/beam/runners/core/BoundedSourceRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/runners/core/BoundedSourceRunnerTest.java
new file mode 100644
index 0000000..73860ef
--- /dev/null
+++ b/sdks/java/harness/src/test/java/org/apache/beam/runners/core/BoundedSourceRunnerTest.java
@@ -0,0 +1,113 @@
+/*
+ * 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.core;
+
+import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow;
+import static org.hamcrest.Matchers.contains;
+import static org.hamcrest.collection.IsEmptyCollection.empty;
+import static org.junit.Assert.assertThat;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.protobuf.Any;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.BytesValue;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import org.apache.beam.fn.harness.fn.ThrowingConsumer;
+import org.apache.beam.fn.v1.BeamFnApi;
+import org.apache.beam.sdk.io.BoundedSource;
+import org.apache.beam.sdk.io.CountingSource;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.util.SerializableUtils;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Tests for {@link BoundedSourceRunner}. */
+@RunWith(JUnit4.class)
+public class BoundedSourceRunnerTest {
+  @Test
+  public void testRunReadLoopWithMultipleSources() throws Exception {
+    List<WindowedValue<Long>> out1ValuesA = new ArrayList<>();
+    List<WindowedValue<Long>> out1ValuesB = new ArrayList<>();
+    List<WindowedValue<Long>> out2Values = new ArrayList<>();
+    Map<String, Collection<ThrowingConsumer<WindowedValue<Long>>>> outputMap = ImmutableMap.of(
+        "out1", ImmutableList.of(out1ValuesA::add, out1ValuesB::add),
+        "out2", ImmutableList.of(out2Values::add));
+
+    BoundedSourceRunner<BoundedSource<Long>, Long> runner =
+        new BoundedSourceRunner<>(
+        PipelineOptionsFactory.create(),
+        BeamFnApi.FunctionSpec.getDefaultInstance(),
+        outputMap);
+
+    runner.runReadLoop(valueInGlobalWindow(CountingSource.upTo(2)));
+    runner.runReadLoop(valueInGlobalWindow(CountingSource.upTo(1)));
+
+    assertThat(out1ValuesA,
+        contains(valueInGlobalWindow(0L), valueInGlobalWindow(1L), valueInGlobalWindow(0L)));
+    assertThat(out1ValuesB,
+        contains(valueInGlobalWindow(0L), valueInGlobalWindow(1L), valueInGlobalWindow(0L)));
+    assertThat(out2Values,
+        contains(valueInGlobalWindow(0L), valueInGlobalWindow(1L), valueInGlobalWindow(0L)));
+  }
+
+  @Test
+  public void testRunReadLoopWithEmptySource() throws Exception {
+    List<WindowedValue<Long>> out1Values = new ArrayList<>();
+    Map<String, Collection<ThrowingConsumer<WindowedValue<Long>>>> outputMap = ImmutableMap.of(
+        "out1", ImmutableList.of(out1Values::add));
+
+    BoundedSourceRunner<BoundedSource<Long>, Long> runner =
+        new BoundedSourceRunner<>(
+        PipelineOptionsFactory.create(),
+        BeamFnApi.FunctionSpec.getDefaultInstance(),
+        outputMap);
+
+    runner.runReadLoop(valueInGlobalWindow(CountingSource.upTo(0)));
+
+    assertThat(out1Values, empty());
+  }
+
+  @Test
+  public void testStart() throws Exception {
+    List<WindowedValue<Long>> outValues = new ArrayList<>();
+    Map<String, Collection<ThrowingConsumer<WindowedValue<Long>>>> outputMap = ImmutableMap.of(
+        "out", ImmutableList.of(outValues::add));
+
+    ByteString encodedSource =
+        ByteString.copyFrom(SerializableUtils.serializeToByteArray(CountingSource.upTo(3)));
+
+    BoundedSourceRunner<BoundedSource<Long>, Long> runner =
+        new BoundedSourceRunner<>(
+        PipelineOptionsFactory.create(),
+        BeamFnApi.FunctionSpec.newBuilder().setData(
+            Any.pack(BytesValue.newBuilder().setValue(encodedSource).build())).build(),
+        outputMap);
+
+    runner.start();
+
+    assertThat(outValues,
+        contains(valueInGlobalWindow(0L), valueInGlobalWindow(1L), valueInGlobalWindow(2L)));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/sdks/java/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/pom.xml b/sdks/java/pom.xml
index 555fdd4..a1a914b 100644
--- a/sdks/java/pom.xml
+++ b/sdks/java/pom.xml
@@ -49,6 +49,7 @@
         <jdk>[1.8,)</jdk>
       </activation>
       <modules>
+        <module>harness</module>
         <module>java8tests</module>
       </modules>
     </profile>

http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/sdks/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/pom.xml b/sdks/pom.xml
index 06dbb9b..150e1ae 100644
--- a/sdks/pom.xml
+++ b/sdks/pom.xml
@@ -33,6 +33,7 @@
   <name>Apache Beam :: SDKs</name>
 
   <modules>
+    <module>common</module>
     <module>java</module>
   </modules>
 


[4/6] beam git commit: A proposal for a portability framework to execute user definable functions.

Posted by lc...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/RegisterHandler.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/RegisterHandler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/RegisterHandler.java
new file mode 100644
index 0000000..14e26f0
--- /dev/null
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/RegisterHandler.java
@@ -0,0 +1,92 @@
+/*
+ * 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.control;
+
+import com.google.protobuf.Message;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import org.apache.beam.fn.v1.BeamFnApi;
+import org.apache.beam.fn.v1.BeamFnApi.RegisterResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A handler and datastore for types that be can be registered via the Fn API.
+ *
+ * <p>Allows for {@link org.apache.beam.fn.v1.BeamFnApi.RegisterRequest}s to occur in parallel with
+ * subsequent requests that may lookup registered values by blocking lookups until registration
+ * occurs.
+ */
+public class RegisterHandler {
+  private static final Logger LOGGER = LoggerFactory.getLogger(RegisterHandler.class);
+  private final ConcurrentMap<Long, CompletableFuture<Message>> idToObject;
+
+  public RegisterHandler() {
+    idToObject = new ConcurrentHashMap<>();
+  }
+
+  public <T extends Message> T getById(long id) {
+    try {
+      @SuppressWarnings("unchecked")
+      CompletableFuture<T> returnValue = (CompletableFuture<T>) computeIfAbsent(id);
+      /*
+       * TODO: Even though the register request instruction occurs before the process bundle
+       * instruction in the control stream, the instructions are being processed in parallel
+       * in the Java harness causing a data race which is why we use a future. This will block
+       * forever in the case of a runner having a bug sending the wrong ids. Instead of blocking
+       * forever, we could do a timed wait or come up with another way of ordering the instruction
+       * processing to remove the data race.
+       */
+      return returnValue.get();
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format("Failed to load %s", id), e);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException(String.format("Failed to load %s", id), e);
+    }
+  }
+
+  public BeamFnApi.InstructionResponse.Builder register(BeamFnApi.InstructionRequest request) {
+    BeamFnApi.InstructionResponse.Builder response = BeamFnApi.InstructionResponse.newBuilder()
+        .setRegister(RegisterResponse.getDefaultInstance());
+
+    BeamFnApi.RegisterRequest registerRequest = request.getRegister();
+    for (BeamFnApi.ProcessBundleDescriptor processBundleDescriptor
+        : registerRequest.getProcessBundleDescriptorList()) {
+      LOGGER.debug("Registering {} with type {}",
+          processBundleDescriptor.getId(),
+          processBundleDescriptor.getClass());
+      computeIfAbsent(processBundleDescriptor.getId()).complete(processBundleDescriptor);
+      for (BeamFnApi.Coder coder : processBundleDescriptor.getCodersList()) {
+        LOGGER.debug("Registering {} with type {}",
+            coder.getFunctionSpec().getId(),
+            coder.getClass());
+        computeIfAbsent(coder.getFunctionSpec().getId()).complete(coder);
+      }
+    }
+
+    return response;
+  }
+
+  private CompletableFuture<Message> computeIfAbsent(long id) {
+    return idToObject.computeIfAbsent(id, (Long ignored) -> new CompletableFuture<>());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/package-info.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/package-info.java
new file mode 100644
index 0000000..6535555
--- /dev/null
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/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.
+ */
+
+/**
+ * Control service client and individual request handlers.
+ */
+package org.apache.beam.fn.harness.control;

http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/BeamFnDataBufferingOutboundObserver.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/BeamFnDataBufferingOutboundObserver.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/BeamFnDataBufferingOutboundObserver.java
new file mode 100644
index 0000000..3bf44ab
--- /dev/null
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/BeamFnDataBufferingOutboundObserver.java
@@ -0,0 +1,135 @@
+/*
+ * 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.data;
+
+import com.google.protobuf.ByteString;
+import io.grpc.stub.StreamObserver;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.function.Consumer;
+import org.apache.beam.fn.harness.fn.CloseableThrowingConsumer;
+import org.apache.beam.fn.v1.BeamFnApi;
+import org.apache.beam.runners.dataflow.options.DataflowPipelineDebugOptions;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.Coder.Context;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.KV;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A buffering outbound {@link Consumer} for the Beam Fn Data API.
+ *
+ * <p>Encodes individually consumed elements with the provided {@link Coder} producing
+ * a single {@link org.apache.beam.fn.v1.BeamFnApi.Elements} message when the buffer threshold
+ * is surpassed.
+ *
+ * <p>The default buffer threshold can be overridden by specifying the experiment
+ * {@code beam_fn_api_data_buffer_limit=<bytes>}
+ *
+ * <p>TODO: Handle outputting large elements (&gt; 2GiBs). Note that this also applies to the
+ * input side as well.
+ *
+ * <p>TODO: Handle outputting elements that are zero bytes by outputting a single byte as
+ * a marker, detect on the input side that no bytes were read and force reading a single byte.
+ */
+public class BeamFnDataBufferingOutboundObserver<T>
+    implements CloseableThrowingConsumer<WindowedValue<T>> {
+  private static final String BEAM_FN_API_DATA_BUFFER_LIMIT = "beam_fn_api_data_buffer_limit=";
+  private static final int DEFAULT_BUFFER_LIMIT_BYTES = 1_000_000;
+  private static final Logger LOGGER =
+      LoggerFactory.getLogger(BeamFnDataBufferingOutboundObserver.class);
+
+  private long byteCounter;
+  private long counter;
+  private final int bufferLimit;
+  private final Coder<WindowedValue<T>> coder;
+  private final KV<Long, BeamFnApi.Target> outputLocation;
+  private final StreamObserver<BeamFnApi.Elements> outboundObserver;
+  private final ByteString.Output bufferedElements;
+
+  public BeamFnDataBufferingOutboundObserver(
+      PipelineOptions options,
+      KV<Long, BeamFnApi.Target> outputLocation,
+      Coder<WindowedValue<T>> coder,
+      StreamObserver<BeamFnApi.Elements> outboundObserver) {
+    this.bufferLimit = getBufferLimit(options);
+    this.outputLocation = outputLocation;
+    this.coder = coder;
+    this.outboundObserver = outboundObserver;
+    this.bufferedElements = ByteString.newOutput();
+  }
+
+  /**
+   * Returns the {@code beam_fn_api_data_buffer_limit=<int>} experiment value if set. Otherwise
+   * returns the default buffer limit.
+   */
+  private static int getBufferLimit(PipelineOptions options) {
+    List<String> experiments = options.as(DataflowPipelineDebugOptions.class).getExperiments();
+    for (String experiment : experiments == null ? Collections.<String>emptyList() : experiments) {
+      if (experiment.startsWith(BEAM_FN_API_DATA_BUFFER_LIMIT)) {
+        return Integer.parseInt(experiment.substring(BEAM_FN_API_DATA_BUFFER_LIMIT.length()));
+      }
+    }
+    return DEFAULT_BUFFER_LIMIT_BYTES;
+  }
+
+  @Override
+  public void close() throws Exception {
+    BeamFnApi.Elements.Builder elements = convertBufferForTransmission();
+    // This will add an empty data block representing the end of stream.
+    elements.addDataBuilder()
+        .setInstructionReference(outputLocation.getKey())
+        .setTarget(outputLocation.getValue());
+
+    LOGGER.debug("Closing stream for instruction {} and "
+        + "target {} having transmitted {} values {} bytes",
+        outputLocation.getKey(),
+        outputLocation.getValue(),
+        counter,
+        byteCounter);
+    outboundObserver.onNext(elements.build());
+  }
+
+  @Override
+  public void accept(WindowedValue<T> t) throws IOException {
+    coder.encode(t, bufferedElements, Context.NESTED);
+    counter += 1;
+    if (bufferedElements.size() >= bufferLimit) {
+      outboundObserver.onNext(convertBufferForTransmission().build());
+    }
+  }
+
+  private BeamFnApi.Elements.Builder convertBufferForTransmission() {
+    BeamFnApi.Elements.Builder elements = BeamFnApi.Elements.newBuilder();
+    if (bufferedElements.size() == 0) {
+      return elements;
+    }
+
+    elements.addDataBuilder()
+        .setInstructionReference(outputLocation.getKey())
+        .setTarget(outputLocation.getValue())
+        .setData(bufferedElements.toByteString());
+
+    byteCounter += bufferedElements.size();
+    bufferedElements.reset();
+    return elements;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/BeamFnDataClient.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/BeamFnDataClient.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/BeamFnDataClient.java
new file mode 100644
index 0000000..27b1acb
--- /dev/null
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/BeamFnDataClient.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.fn.harness.data;
+
+import java.util.concurrent.CompletableFuture;
+import org.apache.beam.fn.harness.fn.CloseableThrowingConsumer;
+import org.apache.beam.fn.harness.fn.ThrowingConsumer;
+import org.apache.beam.fn.v1.BeamFnApi;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.KV;
+
+/**
+ * The {@link BeamFnDataClient} is able to forward inbound elements to a consumer and is also a
+ * consumer of outbound elements. Callers can register themselves as consumers for inbound elements
+ * or can get a handle for a consumer for outbound elements.
+ */
+public interface BeamFnDataClient {
+  /**
+   * Registers the following inbound consumer for the provided instruction id and target.
+   *
+   * <p>The provided coder is used to decode inbound elements. The decoded elements
+   * are passed to the provided consumer. Any failure during decoding or processing of the element
+   * will complete the returned future exceptionally. On successful termination of the stream,
+   * the returned future is completed successfully.
+   *
+   * <p>The consumer is not required to be thread safe.
+   */
+  <T> CompletableFuture<Void> forInboundConsumer(
+      BeamFnApi.ApiServiceDescriptor apiServiceDescriptor,
+      KV<Long, BeamFnApi.Target> inputLocation,
+      Coder<WindowedValue<T>> coder,
+      ThrowingConsumer<WindowedValue<T>> consumer);
+
+  /**
+   * Creates a closeable consumer using the provided instruction id and target.
+   *
+   * <p>The provided coder is used to encode elements on the outbound stream.
+   *
+   * <p>Closing the returned consumer signals the end of the stream.
+   *
+   * <p>The returned closeable consumer is not thread safe.
+   */
+  <T> CloseableThrowingConsumer<WindowedValue<T>> forOutboundConsumer(
+      BeamFnApi.ApiServiceDescriptor apiServiceDescriptor,
+      KV<Long, BeamFnApi.Target> outputLocation,
+      Coder<WindowedValue<T>> coder);
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/BeamFnDataGrpcClient.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/BeamFnDataGrpcClient.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/BeamFnDataGrpcClient.java
new file mode 100644
index 0000000..9bbdc78
--- /dev/null
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/BeamFnDataGrpcClient.java
@@ -0,0 +1,122 @@
+/*
+ * 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.data;
+
+import io.grpc.ManagedChannel;
+import io.grpc.stub.StreamObserver;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.function.BiFunction;
+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.v1.BeamFnApi;
+import org.apache.beam.fn.v1.BeamFnDataGrpc;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.KV;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A {@link BeamFnDataClient} that uses gRPC for sending and receiving data.
+ *
+ * <p>TODO: Handle closing clients that are currently not a consumer nor are being consumed.
+ */
+public class BeamFnDataGrpcClient implements BeamFnDataClient {
+  private static final Logger LOGGER = LoggerFactory.getLogger(BeamFnDataGrpcClient.class);
+
+  private final ConcurrentMap<BeamFnApi.ApiServiceDescriptor, BeamFnDataGrpcMultiplexer> cache;
+  private final Function<BeamFnApi.ApiServiceDescriptor, ManagedChannel> channelFactory;
+  private final BiFunction<Function<StreamObserver<BeamFnApi.Elements>,
+                                    StreamObserver<BeamFnApi.Elements>>,
+                           StreamObserver<BeamFnApi.Elements>,
+                           StreamObserver<BeamFnApi.Elements>> streamObserverFactory;
+  private final PipelineOptions options;
+
+  public BeamFnDataGrpcClient(
+      PipelineOptions options,
+      Function<BeamFnApi.ApiServiceDescriptor, ManagedChannel> channelFactory,
+      BiFunction<Function<StreamObserver<BeamFnApi.Elements>, StreamObserver<BeamFnApi.Elements>>,
+                 StreamObserver<BeamFnApi.Elements>,
+                 StreamObserver<BeamFnApi.Elements>> streamObserverFactory) {
+    this.options = options;
+    this.channelFactory = channelFactory;
+    this.streamObserverFactory = streamObserverFactory;
+    this.cache = new ConcurrentHashMap<>();
+  }
+
+  /**
+   * Registers the following inbound stream consumer for the provided instruction id and target.
+   *
+   * <p>The provided coder is used to decode elements on the inbound stream. The decoded elements
+   * are passed to the provided consumer. Any failure during decoding or processing of the element
+   * will complete the returned future exceptionally. On successful termination of the stream
+   * (signaled by an empty data block), the returned future is completed successfully.
+   */
+  @Override
+  public <T> CompletableFuture<Void> forInboundConsumer(
+      BeamFnApi.ApiServiceDescriptor apiServiceDescriptor,
+      KV<Long, BeamFnApi.Target> inputLocation,
+      Coder<WindowedValue<T>> coder,
+      ThrowingConsumer<WindowedValue<T>> consumer) {
+    LOGGER.debug("Registering consumer instruction {} for target {}",
+        inputLocation.getKey(),
+        inputLocation.getValue());
+
+    CompletableFuture<Void> readFuture = new CompletableFuture<>();
+    BeamFnDataGrpcMultiplexer client = getClientFor(apiServiceDescriptor);
+    client.futureForKey(inputLocation).complete(
+        new BeamFnDataInboundObserver<>(coder, consumer, readFuture));
+    return readFuture;
+  }
+
+  /**
+   * Creates a closeable consumer using the provided instruction id and target.
+   *
+   * <p>The provided coder is used to encode elements on the outbound stream.
+   *
+   * <p>On closing the returned consumer, an empty data block is sent as a signal of the
+   * logical data stream finishing.
+   *
+   * <p>The returned closeable consumer is not thread safe.
+   */
+  @Override
+  public <T> CloseableThrowingConsumer<WindowedValue<T>> forOutboundConsumer(
+      BeamFnApi.ApiServiceDescriptor apiServiceDescriptor,
+      KV<Long, BeamFnApi.Target> outputLocation,
+      Coder<WindowedValue<T>> coder) {
+    BeamFnDataGrpcMultiplexer client = getClientFor(apiServiceDescriptor);
+
+    return new BeamFnDataBufferingOutboundObserver<>(
+        options, outputLocation, coder, client.getOutboundObserver());
+  }
+
+  private BeamFnDataGrpcMultiplexer getClientFor(
+      BeamFnApi.ApiServiceDescriptor apiServiceDescriptor) {
+    return cache.computeIfAbsent(apiServiceDescriptor,
+        (BeamFnApi.ApiServiceDescriptor descriptor) -> new BeamFnDataGrpcMultiplexer(
+            descriptor,
+            (StreamObserver<BeamFnApi.Elements> inboundObserver) -> streamObserverFactory.apply(
+                BeamFnDataGrpc.newStub(channelFactory.apply(apiServiceDescriptor))::data,
+                inboundObserver)));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/BeamFnDataGrpcMultiplexer.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/BeamFnDataGrpcMultiplexer.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/BeamFnDataGrpcMultiplexer.java
new file mode 100644
index 0000000..ea059df
--- /dev/null
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/BeamFnDataGrpcMultiplexer.java
@@ -0,0 +1,140 @@
+/*
+ * 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.data;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.MoreObjects;
+import io.grpc.stub.StreamObserver;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import org.apache.beam.fn.v1.BeamFnApi;
+import org.apache.beam.sdk.values.KV;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A gRPC multiplexer for a specific {@link org.apache.beam.fn.v1.BeamFnApi.ApiServiceDescriptor}.
+ *
+ * <p>Multiplexes data for inbound consumers based upon their individual
+ * {@link org.apache.beam.fn.v1.BeamFnApi.Target}s.
+ *
+ * <p>Multiplexing inbound and outbound streams is as thread safe as the consumers of those
+ * streams. For inbound streams, this is as thread safe as the inbound observers. For outbound
+ * streams, this is as thread safe as the underlying stream observer.
+ *
+ * <p>TODO: Add support for multiplexing over multiple outbound observers by stickying
+ * the output location with a specific outbound observer.
+ */
+public class BeamFnDataGrpcMultiplexer {
+  private static final Logger LOGGER = LoggerFactory.getLogger(BeamFnDataGrpcMultiplexer.class);
+  private final BeamFnApi.ApiServiceDescriptor apiServiceDescriptor;
+  private final StreamObserver<BeamFnApi.Elements> inboundObserver;
+  private final StreamObserver<BeamFnApi.Elements> outboundObserver;
+  @VisibleForTesting
+  final ConcurrentMap<KV<Long, BeamFnApi.Target>,
+                              CompletableFuture<Consumer<BeamFnApi.Elements.Data>>> consumers;
+
+  public BeamFnDataGrpcMultiplexer(
+      BeamFnApi.ApiServiceDescriptor apiServiceDescriptor,
+      Function<StreamObserver<BeamFnApi.Elements>,
+               StreamObserver<BeamFnApi.Elements>> outboundObserverFactory) {
+    this.apiServiceDescriptor = apiServiceDescriptor;
+    this.consumers = new ConcurrentHashMap<>();
+    this.inboundObserver = new InboundObserver();
+    this.outboundObserver = outboundObserverFactory.apply(inboundObserver);
+  }
+
+  @Override
+  public String toString() {
+    return MoreObjects.toStringHelper(this)
+        .add("apiServiceDescriptor", apiServiceDescriptor)
+        .add("consumers", consumers)
+        .toString();
+  }
+
+  public StreamObserver<BeamFnApi.Elements> getInboundObserver() {
+    return inboundObserver;
+  }
+
+  public StreamObserver<BeamFnApi.Elements> getOutboundObserver() {
+    return outboundObserver;
+  }
+
+  public CompletableFuture<Consumer<BeamFnApi.Elements.Data>> futureForKey(
+      KV<Long, BeamFnApi.Target> key) {
+    return consumers.computeIfAbsent(
+        key,
+        (KV<Long, BeamFnApi.Target> providedKey) -> new CompletableFuture<>());
+  }
+
+  /**
+   * A multiplexing {@link StreamObserver} that selects the inbound {@link Consumer} to pass
+   * the elements to.
+   *
+   * <p>The inbound observer blocks until the {@link Consumer} is bound allowing for the
+   * sending harness to initiate transmitting data without needing for the receiving harness to
+   * signal that it is ready to consume that data.
+   */
+  private final class InboundObserver implements StreamObserver<BeamFnApi.Elements> {
+    @Override
+    public void onNext(BeamFnApi.Elements value) {
+      for (BeamFnApi.Elements.Data data : value.getDataList()) {
+        try {
+          KV<Long, BeamFnApi.Target> key =
+              KV.of(data.getInstructionReference(), data.getTarget());
+          futureForKey(key).get().accept(data);
+          if (data.getData().isEmpty()) {
+            consumers.remove(key);
+          }
+        /*
+         * TODO: On failure we should fail any bundles that were impacted eagerly
+         * instead of relying on the Runner harness to do all the failure handling.
+         */
+        } catch (ExecutionException | InterruptedException e) {
+          LOGGER.error(
+              "Client interrupted during handling of data for instruction {} and target {}",
+              data.getInstructionReference(),
+              data.getTarget(),
+              e);
+          outboundObserver.onError(e);
+        } catch (RuntimeException e) {
+          LOGGER.error(
+              "Client failed to handle data for instruction {} and target {}",
+              data.getInstructionReference(),
+              data.getTarget(),
+              e);
+          outboundObserver.onError(e);
+        }
+      }
+    }
+
+    @Override
+    public void onError(Throwable t) {
+      LOGGER.error("Failed to handle for {}", apiServiceDescriptor, t);
+    }
+
+    @Override
+    public void onCompleted() {
+      LOGGER.warn("Hanged up for {}.", apiServiceDescriptor);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/BeamFnDataInboundObserver.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/BeamFnDataInboundObserver.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/BeamFnDataInboundObserver.java
new file mode 100644
index 0000000..f8b5ab8
--- /dev/null
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/BeamFnDataInboundObserver.java
@@ -0,0 +1,81 @@
+/*
+ * 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.data;
+
+import java.io.InputStream;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Consumer;
+import org.apache.beam.fn.harness.fn.ThrowingConsumer;
+import org.apache.beam.fn.v1.BeamFnApi;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.Coder.Context;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Decodes individually consumed {@link org.apache.beam.fn.v1.BeamFnApi.Elements.Data} with the
+ * provided {@link Coder} passing the individual decoded elements to the provided consumer.
+ */
+public class BeamFnDataInboundObserver<T> implements Consumer<BeamFnApi.Elements.Data> {
+  private static final Logger LOGGER = LoggerFactory.getLogger(BeamFnDataInboundObserver.class);
+  private final ThrowingConsumer<WindowedValue<T>> consumer;
+  private final Coder<WindowedValue<T>> coder;
+  private final CompletableFuture<Void> readFuture;
+  private long byteCounter;
+  private long counter;
+
+  public BeamFnDataInboundObserver(
+      Coder<WindowedValue<T>> coder,
+      ThrowingConsumer<WindowedValue<T>> consumer,
+      CompletableFuture<Void> readFuture) {
+    this.coder = coder;
+    this.consumer = consumer;
+    this.readFuture = readFuture;
+  }
+
+  @Override
+  public void accept(BeamFnApi.Elements.Data t) {
+    if (readFuture.isDone()) {
+      // Drop any incoming data if the stream processing has finished.
+      return;
+    }
+    try {
+      if (t.getData().isEmpty()) {
+        LOGGER.debug("Closing stream for instruction {} and "
+            + "target {} having consumed {} values {} bytes",
+            t.getInstructionReference(),
+            t.getTarget(),
+            counter,
+            byteCounter);
+        readFuture.complete(null);
+        return;
+      }
+
+      byteCounter += t.getData().size();
+      InputStream inputStream = t.getData().newInput();
+      while (inputStream.available() > 0) {
+        counter += 1;
+        WindowedValue<T> value = coder.decode(inputStream, Context.NESTED);
+        consumer.accept(value);
+      }
+    } catch (Exception e) {
+      readFuture.completeExceptionally(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/package-info.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/package-info.java
new file mode 100644
index 0000000..edaaa65
--- /dev/null
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/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.
+ */
+
+/**
+ * Data service client and logical stream multiplexing.
+ */
+package org.apache.beam.fn.harness.data;

http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fake/FakeAggregatorFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fake/FakeAggregatorFactory.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fake/FakeAggregatorFactory.java
new file mode 100644
index 0000000..b3b7f48
--- /dev/null
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fake/FakeAggregatorFactory.java
@@ -0,0 +1,52 @@
+/*
+ * 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.fake;
+
+import org.apache.beam.runners.core.AggregatorFactory;
+import org.apache.beam.runners.core.ExecutionContext.StepContext;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+
+/**
+ * A fake implementation of an {@link AggregatorFactory} that is to be filled in at a later time.
+ * The factory returns {@link Aggregator}s that do nothing when a value is added.
+ */
+public class FakeAggregatorFactory implements AggregatorFactory {
+  @Override
+  public <InputT, AccumT, OutputT> Aggregator<InputT, OutputT> createAggregatorForDoFn(
+      Class<?> fnClass,
+      StepContext stepContext,
+      String aggregatorName,
+      CombineFn<InputT, AccumT, OutputT> combine) {
+    return new Aggregator<InputT, OutputT>() {
+      @Override
+      public void addValue(InputT value) {}
+
+      @Override
+      public String getName() {
+        return aggregatorName;
+      }
+
+      @Override
+      public CombineFn<InputT, ?, OutputT> getCombineFn() {
+        return combine;
+      }
+    };
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fake/FakeStepContext.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fake/FakeStepContext.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fake/FakeStepContext.java
new file mode 100644
index 0000000..84da059
--- /dev/null
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fake/FakeStepContext.java
@@ -0,0 +1,70 @@
+/*
+ * 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.fake;
+
+import java.io.IOException;
+import org.apache.beam.runners.core.ExecutionContext.StepContext;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.TimerInternals;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.state.StateInternals;
+import org.apache.beam.sdk.values.TupleTag;
+
+/**
+ * A fake {@link StepContext} factory that performs no-ops.
+ */
+public class FakeStepContext implements StepContext {
+  @Override
+  public String getStepName() {
+    return "TODO";
+  }
+
+  @Override
+  public String getTransformName() {
+    return "TODO";
+  }
+
+  @Override
+  public void noteOutput(WindowedValue<?> output) {
+  }
+
+  @Override
+  public void noteSideOutput(TupleTag<?> tag, WindowedValue<?> output) {
+  }
+
+  @Override
+  public <T, W extends BoundedWindow> void writePCollectionViewData(
+      TupleTag<?> tag,
+      Iterable<WindowedValue<T>> data,
+      Coder<Iterable<WindowedValue<T>>> dataCoder,
+      W window,
+      Coder<W> windowCoder) throws IOException {
+  }
+
+  @Override
+  public StateInternals<?> stateInternals() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public TimerInternals timerInternals() {
+    throw new UnsupportedOperationException();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fake/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fake/package-info.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fake/package-info.java
new file mode 100644
index 0000000..cd6eb02
--- /dev/null
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fake/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.
+ */
+
+/**
+ * Fake implementations of bindings used with runners-core.
+ */
+package org.apache.beam.fn.harness.fake;

http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fn/CloseableThrowingConsumer.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fn/CloseableThrowingConsumer.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fn/CloseableThrowingConsumer.java
new file mode 100644
index 0000000..59ab149
--- /dev/null
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fn/CloseableThrowingConsumer.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.
+ */
+
+package org.apache.beam.fn.harness.fn;
+
+/** A {@link ThrowingConsumer} that can be closed. */
+public interface CloseableThrowingConsumer<T> extends AutoCloseable, ThrowingConsumer<T> {
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fn/ThrowingBiFunction.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fn/ThrowingBiFunction.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fn/ThrowingBiFunction.java
new file mode 100644
index 0000000..9d505da
--- /dev/null
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fn/ThrowingBiFunction.java
@@ -0,0 +1,32 @@
+/*
+ * 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.fn;
+
+import java.util.function.BiFunction;
+
+/**
+ * A {@link BiFunction} which can throw {@link Exception}s.
+ *
+ * <p>Used to expand the allowed set of method references to be used by Java 8
+ * functional interfaces.
+ */
+@FunctionalInterface
+public interface ThrowingBiFunction<T1, T2, T3> {
+  T3 apply(T1 t1, T2 t2) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fn/ThrowingConsumer.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fn/ThrowingConsumer.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fn/ThrowingConsumer.java
new file mode 100644
index 0000000..b34e857
--- /dev/null
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fn/ThrowingConsumer.java
@@ -0,0 +1,32 @@
+/*
+ * 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.fn;
+
+import java.util.function.Consumer;
+
+/**
+ * A {@link Consumer} which can throw {@link Exception}s.
+ *
+ * <p>Used to expand the allowed set of method references to be used by Java 8
+ * functional interfaces.
+ */
+@FunctionalInterface
+public interface ThrowingConsumer<T> {
+  void accept(T t) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fn/ThrowingFunction.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fn/ThrowingFunction.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fn/ThrowingFunction.java
new file mode 100644
index 0000000..446ff60
--- /dev/null
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fn/ThrowingFunction.java
@@ -0,0 +1,32 @@
+/*
+ * 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.fn;
+
+import java.util.function.Function;
+
+/**
+ * A {@link Function} which can throw {@link Exception}s.
+ *
+ * <p>Used to expand the allowed set of method references to be used by Java 8
+ * functional interfaces.
+ */
+@FunctionalInterface
+public interface ThrowingFunction<T1, T2> {
+  T2 apply(T1 value) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fn/ThrowingRunnable.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fn/ThrowingRunnable.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fn/ThrowingRunnable.java
new file mode 100644
index 0000000..c7fc29e
--- /dev/null
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fn/ThrowingRunnable.java
@@ -0,0 +1,30 @@
+/*
+ * 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.fn;
+
+/**
+ * A {@link Runnable} which can throw {@link Exception}s.
+ *
+ * <p>Used to expand the allowed set of method references to be used by Java 8
+ * functional interfaces.
+ */
+@FunctionalInterface
+public interface ThrowingRunnable {
+  void run() throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fn/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fn/package-info.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fn/package-info.java
new file mode 100644
index 0000000..bbf3396
--- /dev/null
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fn/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.
+ */
+
+/**
+ * Java 8 functional interface extensions.
+ */
+package org.apache.beam.fn.harness.fn;

http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/logging/BeamFnLoggingClient.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/logging/BeamFnLoggingClient.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/logging/BeamFnLoggingClient.java
new file mode 100644
index 0000000..d74d9fa
--- /dev/null
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/logging/BeamFnLoggingClient.java
@@ -0,0 +1,308 @@
+/*
+ * 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.logging;
+
+import static com.google.common.base.Throwables.getStackTraceAsString;
+
+import com.google.common.base.MoreObjects;
+import com.google.common.collect.ImmutableMap;
+import com.google.protobuf.Timestamp;
+import io.grpc.ManagedChannel;
+import io.grpc.stub.StreamObserver;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.BlockingDeque;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingDeque;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BiFunction;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.logging.Formatter;
+import java.util.logging.Handler;
+import java.util.logging.Level;
+import java.util.logging.LogManager;
+import java.util.logging.LogRecord;
+import java.util.logging.Logger;
+import java.util.logging.SimpleFormatter;
+import org.apache.beam.fn.v1.BeamFnApi;
+import org.apache.beam.fn.v1.BeamFnLoggingGrpc;
+import org.apache.beam.runners.dataflow.options.DataflowWorkerLoggingOptions;
+import org.apache.beam.sdk.options.GcsOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+
+/**
+ * Configures {@link java.util.logging} to send all {@link LogRecord}s via the Beam Fn Logging API.
+ */
+public class BeamFnLoggingClient implements AutoCloseable {
+  private static final String ROOT_LOGGER_NAME = "";
+  private static final ImmutableMap<Level, BeamFnApi.LogEntry.Severity> LOG_LEVEL_MAP =
+      ImmutableMap.<Level, BeamFnApi.LogEntry.Severity>builder()
+      .put(Level.SEVERE, BeamFnApi.LogEntry.Severity.ERROR)
+      .put(Level.WARNING, BeamFnApi.LogEntry.Severity.WARN)
+      .put(Level.INFO, BeamFnApi.LogEntry.Severity.INFO)
+      .put(Level.FINE, BeamFnApi.LogEntry.Severity.DEBUG)
+      .put(Level.FINEST, BeamFnApi.LogEntry.Severity.TRACE)
+      .build();
+
+  private static final ImmutableMap<DataflowWorkerLoggingOptions.Level, Level> LEVEL_CONFIGURATION =
+      ImmutableMap.<DataflowWorkerLoggingOptions.Level, Level>builder()
+          .put(DataflowWorkerLoggingOptions.Level.OFF, Level.OFF)
+          .put(DataflowWorkerLoggingOptions.Level.ERROR, Level.SEVERE)
+          .put(DataflowWorkerLoggingOptions.Level.WARN, Level.WARNING)
+          .put(DataflowWorkerLoggingOptions.Level.INFO, Level.INFO)
+          .put(DataflowWorkerLoggingOptions.Level.DEBUG, Level.FINE)
+          .put(DataflowWorkerLoggingOptions.Level.TRACE, Level.FINEST)
+          .build();
+
+  private static final Formatter FORMATTER = new SimpleFormatter();
+
+  /* Used to signal to a thread processing a queue to finish its work gracefully. */
+  private static final BeamFnApi.LogEntry POISON_PILL =
+      BeamFnApi.LogEntry.newBuilder().setInstructionReference(Long.MIN_VALUE).build();
+
+  /**
+   * The number of log messages that will be buffered. Assuming log messages are at most 1 KiB,
+   * this represents a buffer of about 10 MiBs.
+   */
+  private static final int MAX_BUFFERED_LOG_ENTRY_COUNT = 10_000;
+
+  /* We need to store a reference to the configured loggers so that they are not
+   * garbage collected. java.util.logging only has weak references to the loggers
+   * so if they are garbage collected, our hierarchical configuration will be lost. */
+  private final Collection<Logger> configuredLoggers;
+  private final BeamFnApi.ApiServiceDescriptor apiServiceDescriptor;
+  private final ManagedChannel channel;
+  private final StreamObserver<BeamFnApi.LogEntry.List> outboundObserver;
+  private final LogControlObserver inboundObserver;
+  private final LogRecordHandler logRecordHandler;
+  private final CompletableFuture<Object> inboundObserverCompletion;
+
+  public BeamFnLoggingClient(
+      PipelineOptions options,
+      BeamFnApi.ApiServiceDescriptor apiServiceDescriptor,
+      Function<BeamFnApi.ApiServiceDescriptor, ManagedChannel> channelFactory,
+      BiFunction<Function<StreamObserver<BeamFnApi.LogControl>,
+                          StreamObserver<BeamFnApi.LogEntry.List>>,
+                 StreamObserver<BeamFnApi.LogControl>,
+                 StreamObserver<BeamFnApi.LogEntry.List>> streamObserverFactory) {
+    this.apiServiceDescriptor = apiServiceDescriptor;
+    this.inboundObserverCompletion = new CompletableFuture<>();
+    this.configuredLoggers = new ArrayList<>();
+    this.channel = channelFactory.apply(apiServiceDescriptor);
+
+    // Reset the global log manager, get the root logger and remove the default log handlers.
+    LogManager logManager = LogManager.getLogManager();
+    logManager.reset();
+    Logger rootLogger = logManager.getLogger(ROOT_LOGGER_NAME);
+    for (Handler handler : rootLogger.getHandlers()) {
+      rootLogger.removeHandler(handler);
+    }
+
+    // Use the passed in logging options to configure the various logger levels.
+    DataflowWorkerLoggingOptions loggingOptions = options.as(DataflowWorkerLoggingOptions.class);
+    if (loggingOptions.getDefaultWorkerLogLevel() != null) {
+      rootLogger.setLevel(LEVEL_CONFIGURATION.get(loggingOptions.getDefaultWorkerLogLevel()));
+    }
+
+    if (loggingOptions.getWorkerLogLevelOverrides() != null) {
+      for (Map.Entry<String, DataflowWorkerLoggingOptions.Level> loggerOverride :
+        loggingOptions.getWorkerLogLevelOverrides().entrySet()) {
+        Logger logger = Logger.getLogger(loggerOverride.getKey());
+        logger.setLevel(LEVEL_CONFIGURATION.get(loggerOverride.getValue()));
+        configuredLoggers.add(logger);
+      }
+    }
+
+    BeamFnLoggingGrpc.BeamFnLoggingStub stub = BeamFnLoggingGrpc.newStub(channel);
+    inboundObserver = new LogControlObserver();
+    logRecordHandler = new LogRecordHandler(options.as(GcsOptions.class).getExecutorService());
+    logRecordHandler.setLevel(Level.ALL);
+    outboundObserver = streamObserverFactory.apply(stub::logging, inboundObserver);
+    rootLogger.addHandler(logRecordHandler);
+  }
+
+  @Override
+  public void close() throws Exception {
+    // Hang up with the server
+    logRecordHandler.close();
+
+    // Wait for the server to hang up
+    inboundObserverCompletion.get();
+
+    // Reset the logging configuration to what it is at startup
+    for (Logger logger : configuredLoggers) {
+      logger.setLevel(null);
+    }
+    configuredLoggers.clear();
+    LogManager.getLogManager().readConfiguration();
+
+    // Shut the channel down
+    channel.shutdown();
+    if (!channel.awaitTermination(10, TimeUnit.SECONDS)) {
+      channel.shutdownNow();
+    }
+  }
+
+  @Override
+  public String toString() {
+    return MoreObjects.toStringHelper(BeamFnLoggingClient.class)
+        .add("apiServiceDescriptor", apiServiceDescriptor)
+        .toString();
+  }
+
+  private class LogRecordHandler extends Handler implements Runnable {
+    private final BlockingDeque<BeamFnApi.LogEntry> bufferedLogEntries =
+        new LinkedBlockingDeque<>(MAX_BUFFERED_LOG_ENTRY_COUNT);
+    private final Future<?> bufferedLogWriter;
+    private final ThreadLocal<Consumer<BeamFnApi.LogEntry>> logEntryHandler;
+
+    private LogRecordHandler(ExecutorService executorService) {
+      bufferedLogWriter = executorService.submit(this);
+      logEntryHandler = new ThreadLocal<>();
+    }
+
+    @Override
+    public void publish(LogRecord record) {
+      BeamFnApi.LogEntry.Severity severity = LOG_LEVEL_MAP.get(record.getLevel());
+      if (severity == null) {
+        return;
+      }
+      BeamFnApi.LogEntry.Builder builder = BeamFnApi.LogEntry.newBuilder()
+          .setSeverity(severity)
+          .setLogLocation(record.getLoggerName())
+          .setMessage(FORMATTER.formatMessage(record))
+          .setThread(Integer.toString(record.getThreadID()))
+          .setTimestamp(Timestamp.newBuilder()
+              .setSeconds(record.getMillis() / 1000)
+              .setNanos((int) (record.getMillis() % 1000) * 1_000_000));
+      if (record.getThrown() != null) {
+        builder.setTrace(getStackTraceAsString(record.getThrown()));
+      }
+      // The thread that sends log records should never perform a blocking publish and
+      // only insert log records best effort. We detect which thread is logging
+      // by using the thread local, defaulting to the blocking publish.
+      MoreObjects.firstNonNull(
+          logEntryHandler.get(), this::blockingPublish).accept(builder.build());
+    }
+
+    /** Blocks caller till enough space exists to publish this log entry. */
+    private void blockingPublish(BeamFnApi.LogEntry logEntry) {
+      try {
+        bufferedLogEntries.put(logEntry);
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        throw new RuntimeException(e);
+      }
+    }
+
+    @Override
+    public void run() {
+      // Logging which occurs in this thread will attempt to publish log entries into the
+      // above handler which should never block if the queue is full otherwise
+      // this thread will get stuck.
+      logEntryHandler.set(bufferedLogEntries::offer);
+      List<BeamFnApi.LogEntry> additionalLogEntries =
+          new ArrayList<>(MAX_BUFFERED_LOG_ENTRY_COUNT);
+      try {
+        BeamFnApi.LogEntry logEntry;
+        while ((logEntry = bufferedLogEntries.take()) != POISON_PILL) {
+          BeamFnApi.LogEntry.List.Builder builder =
+              BeamFnApi.LogEntry.List.newBuilder().addLogEntries(logEntry);
+          bufferedLogEntries.drainTo(additionalLogEntries);
+          for (int i = 0; i < additionalLogEntries.size(); ++i) {
+            if (additionalLogEntries.get(i) == POISON_PILL) {
+              additionalLogEntries = additionalLogEntries.subList(0, i);
+              break;
+            }
+          }
+          builder.addAllLogEntries(additionalLogEntries);
+          outboundObserver.onNext(builder.build());
+        }
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        throw new IllegalStateException(e);
+      }
+    }
+
+    @Override
+    public void flush() {
+    }
+
+    @Override
+    public void close() {
+      synchronized (outboundObserver) {
+        // If we are done, then a previous caller has already shutdown the queue processing thread
+        // hence we don't need to do it again.
+        if (!bufferedLogWriter.isDone()) {
+          // We check to see if we were able to successfully insert the poison pill at the end of
+          // the queue forcing the remainder of the elements to be processed or if the processing
+          // thread is done.
+          try {
+            // The order of these checks is important because short circuiting will cause us to
+            // insert into the queue first and only if it fails do we check that the thread is done.
+            while (!bufferedLogEntries.offer(POISON_PILL, 60, TimeUnit.SECONDS)
+                || !bufferedLogWriter.isDone()) {
+            }
+          } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+            throw new RuntimeException(e);
+          }
+          waitTillFinish();
+        }
+        outboundObserver.onCompleted();
+      }
+    }
+
+    private void waitTillFinish() {
+      try {
+        bufferedLogWriter.get();
+      } catch (CancellationException e) {
+        // Ignore cancellations
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        throw new RuntimeException(e);
+      } catch (ExecutionException e) {
+        throw new RuntimeException(e);
+      }
+    }
+  }
+
+  private class LogControlObserver implements StreamObserver<BeamFnApi.LogControl> {
+    @Override
+    public void onNext(BeamFnApi.LogControl value) {
+    }
+
+    @Override
+    public void onError(Throwable t) {
+      inboundObserverCompletion.completeExceptionally(t);
+    }
+
+    @Override
+    public void onCompleted() {
+      inboundObserverCompletion.complete(null);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/logging/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/logging/package-info.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/logging/package-info.java
new file mode 100644
index 0000000..7a4d0a8
--- /dev/null
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/logging/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.
+ */
+
+/**
+ * Logging service client and JUL logging handler adapter.
+ */
+package org.apache.beam.fn.harness.logging;

http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/package-info.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/package-info.java
new file mode 100644
index 0000000..58080e4
--- /dev/null
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/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.
+ */
+
+/**
+ * Top level package for Beam Java Fn Harness.
+ */
+package org.apache.beam.fn.harness;

http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/stream/AdvancingPhaser.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/stream/AdvancingPhaser.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/stream/AdvancingPhaser.java
new file mode 100644
index 0000000..2007139
--- /dev/null
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/stream/AdvancingPhaser.java
@@ -0,0 +1,36 @@
+/*
+ * 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.stream;
+
+import java.util.concurrent.Phaser;
+
+/**
+ * A {@link Phaser} which never terminates. The default {@link Phaser} implementation terminates
+ * after the first advancement.
+ */
+public final class AdvancingPhaser extends Phaser {
+  public AdvancingPhaser(int numParties) {
+    super(numParties);
+  }
+
+  @Override
+  protected boolean onAdvance(int phase, int registeredParties) {
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/stream/BufferingStreamObserver.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/stream/BufferingStreamObserver.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/stream/BufferingStreamObserver.java
new file mode 100644
index 0000000..cda3a4b
--- /dev/null
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/stream/BufferingStreamObserver.java
@@ -0,0 +1,166 @@
+/*
+ * 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.stream;
+
+import static com.google.common.base.Preconditions.checkState;
+
+import com.google.common.annotations.VisibleForTesting;
+import io.grpc.stub.CallStreamObserver;
+import io.grpc.stub.StreamObserver;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingDeque;
+import java.util.concurrent.Phaser;
+import java.util.concurrent.TimeUnit;
+import javax.annotation.concurrent.ThreadSafe;
+
+/**
+ * A thread safe {@link StreamObserver} which uses a bounded queue to pass elements to a processing
+ * thread responsible for interacting with the underlying {@link CallStreamObserver}.
+ *
+ * <p>Flow control with the underlying {@link CallStreamObserver} is handled with a {@link Phaser}
+ * which waits for advancement of the phase if the {@link CallStreamObserver} is not ready. Callers
+ * are expected to advance the {@link Phaser} whenever the underlying {@link CallStreamObserver}
+ * becomes ready.
+ */
+@ThreadSafe
+public final class BufferingStreamObserver<T> implements StreamObserver<T> {
+  private static final Object POISON_PILL = new Object();
+  private final LinkedBlockingDeque<T> queue;
+  private final Phaser phaser;
+  private final CallStreamObserver<T> outboundObserver;
+  private final Future<?> queueDrainer;
+  private final int bufferSize;
+
+  public BufferingStreamObserver(
+      Phaser phaser,
+      CallStreamObserver<T> outboundObserver,
+      ExecutorService executor,
+      int bufferSize) {
+    this.phaser = phaser;
+    this.bufferSize = bufferSize;
+    this.queue = new LinkedBlockingDeque<>(bufferSize);
+    this.outboundObserver = outboundObserver;
+    this.queueDrainer = executor.submit(this::drainQueue);
+  }
+
+  private void drainQueue() {
+    try {
+      while (true) {
+        int currentPhase = phaser.getPhase();
+        while (outboundObserver.isReady()) {
+          T value = queue.take();
+          if (value != POISON_PILL) {
+            outboundObserver.onNext(value);
+          } else {
+            return;
+          }
+        }
+        phaser.awaitAdvance(currentPhase);
+      }
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new IllegalStateException(e);
+    }
+  }
+
+  @Override
+  public void onNext(T value) {
+    try {
+      // Attempt to add an element to the bounded queue occasionally checking to see
+      // if the queue drainer is still alive.
+      while (!queue.offer(value, 60, TimeUnit.SECONDS)) {
+        checkState(!queueDrainer.isDone(), "Stream observer has finished.");
+      }
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Override
+  public void onError(Throwable t) {
+    synchronized (outboundObserver) {
+      // If we are done, then a previous caller has already shutdown the queue processing thread
+      // hence we don't need to do it again.
+      if (!queueDrainer.isDone()) {
+        // We check to see if we were able to successfully insert the poison pill at the front of
+        // the queue to cancel the processing thread eagerly or if the processing thread is done.
+        try {
+          // The order of these checks is important because short circuiting will cause us to
+          // insert into the queue first and only if it fails do we check that the thread is done.
+          while (!queue.offerFirst((T) POISON_PILL, 60, TimeUnit.SECONDS)
+              || !queueDrainer.isDone()) {
+          }
+        } catch (InterruptedException e) {
+          Thread.currentThread().interrupt();
+          throw new RuntimeException(e);
+        }
+        waitTillFinish();
+      }
+      outboundObserver.onError(t);
+    }
+  }
+
+  @Override
+  public void onCompleted() {
+    synchronized (outboundObserver) {
+      // If we are done, then a previous caller has already shutdown the queue processing thread
+      // hence we don't need to do it again.
+      if (!queueDrainer.isDone()) {
+        // We check to see if we were able to successfully insert the poison pill at the end of
+        // the queue forcing the remainder of the elements to be processed or if the processing
+        // thread is done.
+        try {
+          // The order of these checks is important because short circuiting will cause us to
+          // insert into the queue first and only if it fails do we check that the thread is done.
+          while (!queue.offer((T) POISON_PILL, 60, TimeUnit.SECONDS)
+              || !queueDrainer.isDone()) {
+          }
+        } catch (InterruptedException e) {
+          Thread.currentThread().interrupt();
+          throw new RuntimeException(e);
+        }
+        waitTillFinish();
+      }
+      outboundObserver.onCompleted();
+    }
+  }
+
+  @VisibleForTesting
+  public int getBufferSize() {
+    return bufferSize;
+  }
+
+  private void waitTillFinish() {
+    try {
+      queueDrainer.get();
+    } catch (CancellationException e) {
+      // Cancellation is expected
+      return;
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException(e);
+    } catch (ExecutionException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/stream/DirectStreamObserver.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/stream/DirectStreamObserver.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/stream/DirectStreamObserver.java
new file mode 100644
index 0000000..82a1aa4
--- /dev/null
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/stream/DirectStreamObserver.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.fn.harness.stream;
+
+import io.grpc.stub.CallStreamObserver;
+import io.grpc.stub.StreamObserver;
+import java.util.concurrent.Phaser;
+import javax.annotation.concurrent.ThreadSafe;
+
+/**
+ * A {@link StreamObserver} which uses synchronization on the underlying
+ * {@link CallStreamObserver} to provide thread safety.
+ *
+ * <p>Flow control with the underlying {@link CallStreamObserver} is handled with a {@link Phaser}
+ * which waits for advancement of the phase if the {@link CallStreamObserver} is not ready.
+ * Creator is expected to advance the {@link Phaser} whenever the underlying
+ * {@link CallStreamObserver} becomes ready.
+ */
+@ThreadSafe
+public final class DirectStreamObserver<T> implements StreamObserver<T> {
+  private final Phaser phaser;
+  private final CallStreamObserver<T> outboundObserver;
+
+  public DirectStreamObserver(
+      Phaser phaser,
+      CallStreamObserver<T> outboundObserver) {
+    this.phaser = phaser;
+    this.outboundObserver = outboundObserver;
+  }
+
+  @Override
+  public void onNext(T value) {
+    int phase = phaser.getPhase();
+    if (!outboundObserver.isReady()) {
+      phaser.awaitAdvance(phase);
+    }
+    synchronized (outboundObserver) {
+      outboundObserver.onNext(value);
+    }
+  }
+
+  @Override
+  public void onError(Throwable t) {
+    synchronized (outboundObserver) {
+      outboundObserver.onError(t);
+    }
+  }
+
+  @Override
+  public void onCompleted() {
+    synchronized (outboundObserver) {
+      outboundObserver.onCompleted();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/stream/ForwardingClientResponseObserver.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/stream/ForwardingClientResponseObserver.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/stream/ForwardingClientResponseObserver.java
new file mode 100644
index 0000000..ef641b0
--- /dev/null
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/stream/ForwardingClientResponseObserver.java
@@ -0,0 +1,63 @@
+/*
+ * 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.stream;
+
+import io.grpc.stub.ClientCallStreamObserver;
+import io.grpc.stub.ClientResponseObserver;
+import io.grpc.stub.StreamObserver;
+
+/**
+ * A {@link ClientResponseObserver} which delegates all {@link StreamObserver} calls.
+ *
+ * <p>Used to wrap existing {@link StreamObserver}s to be able to install an
+ * {@link ClientCallStreamObserver#setOnReadyHandler(Runnable) onReadyHandler}.
+ *
+ * <p>This is as thread-safe as the undering stream observer that is being wrapped.
+ */
+final class ForwardingClientResponseObserver<ReqT, RespT>
+    implements ClientResponseObserver<RespT, ReqT> {
+  private final Runnable onReadyHandler;
+  private final StreamObserver<ReqT> inboundObserver;
+
+  ForwardingClientResponseObserver(
+      StreamObserver<ReqT> inboundObserver, Runnable onReadyHandler) {
+    this.inboundObserver = inboundObserver;
+    this.onReadyHandler = onReadyHandler;
+  }
+
+  @Override
+  public void onNext(ReqT value) {
+    inboundObserver.onNext(value);
+  }
+
+  @Override
+  public void onError(Throwable t) {
+    inboundObserver.onError(t);
+  }
+
+  @Override
+  public void onCompleted() {
+    inboundObserver.onCompleted();
+  }
+
+  @Override
+  public void beforeStart(ClientCallStreamObserver<RespT> stream) {
+    stream.setOnReadyHandler(onReadyHandler);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/stream/StreamObserverFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/stream/StreamObserverFactory.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/stream/StreamObserverFactory.java
new file mode 100644
index 0000000..9326e11
--- /dev/null
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/stream/StreamObserverFactory.java
@@ -0,0 +1,91 @@
+/*
+ * 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.stream;
+
+import io.grpc.stub.CallStreamObserver;
+import io.grpc.stub.StreamObserver;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.function.Function;
+import org.apache.beam.runners.dataflow.options.DataflowPipelineDebugOptions;
+import org.apache.beam.sdk.options.GcsOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+
+/**
+ * Uses {@link PipelineOptions} to configure which underlying {@link StreamObserver} implementation
+ * to use.
+ */
+public abstract class StreamObserverFactory {
+  public static StreamObserverFactory fromOptions(PipelineOptions options) {
+    List<String> experiments = options.as(DataflowPipelineDebugOptions.class).getExperiments();
+    if (experiments != null && experiments.contains("beam_fn_api_buffered_stream")) {
+      int bufferSize = Buffered.DEFAULT_BUFFER_SIZE;
+      for (String experiment : experiments) {
+        if (experiment.startsWith("beam_fn_api_buffered_stream_buffer_size=")) {
+          bufferSize = Integer.parseInt(
+              experiment.substring("beam_fn_api_buffered_stream_buffer_size=".length()));
+        }
+      }
+      return new Buffered(options.as(GcsOptions.class).getExecutorService(), bufferSize);
+    }
+    return new Direct();
+  }
+
+  public abstract <ReqT, RespT> StreamObserver<RespT> from(
+      Function<StreamObserver<ReqT>, StreamObserver<RespT>> clientFactory,
+      StreamObserver<ReqT> responseObserver);
+
+  private static class Direct extends StreamObserverFactory {
+
+    @Override
+    public <ReqT, RespT> StreamObserver<RespT> from(
+        Function<StreamObserver<ReqT>, StreamObserver<RespT>> clientFactory,
+        StreamObserver<ReqT> inboundObserver) {
+      AdvancingPhaser phaser = new AdvancingPhaser(1);
+      CallStreamObserver<RespT> outboundObserver = (CallStreamObserver<RespT>) clientFactory.apply(
+          new ForwardingClientResponseObserver<ReqT, RespT>(
+              inboundObserver, phaser::arrive));
+      return new DirectStreamObserver<>(phaser, outboundObserver);
+    }
+  }
+
+  private static class Buffered extends StreamObserverFactory {
+    private static final int DEFAULT_BUFFER_SIZE = 64;
+    private final ExecutorService executorService;
+    private final int bufferSize;
+
+    private Buffered(ExecutorService executorService, int bufferSize) {
+      this.executorService = executorService;
+      this.bufferSize = bufferSize;
+    }
+
+    @Override
+    public <ReqT, RespT> StreamObserver<RespT> from(
+        Function<StreamObserver<ReqT>, StreamObserver<RespT>> clientFactory,
+        StreamObserver<ReqT> inboundObserver) {
+      AdvancingPhaser phaser = new AdvancingPhaser(1);
+      CallStreamObserver<RespT> outboundObserver = (CallStreamObserver<RespT>) clientFactory.apply(
+          new ForwardingClientResponseObserver<ReqT, RespT>(
+              inboundObserver, phaser::arrive));
+      return new BufferingStreamObserver<>(
+          phaser, outboundObserver, executorService, bufferSize);
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/stream/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/stream/package-info.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/stream/package-info.java
new file mode 100644
index 0000000..df4042c
--- /dev/null
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/stream/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 stream management.
+ */
+package org.apache.beam.fn.harness.stream;


[5/6] beam git commit: A proposal for a portability framework to execute user definable functions.

Posted by lc...@apache.org.
A proposal for a portability framework to execute user definable functions.


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

Branch: refs/heads/master
Commit: 0b4b2becb45b9f637ba31f599ebe8be0331bd633
Parents: 582c4a8
Author: Luke Cwik <lc...@google.com>
Authored: Thu Jan 19 15:16:55 2017 -0800
Committer: Luke Cwik <lc...@google.com>
Committed: Mon Jan 30 12:47:55 2017 -0800

----------------------------------------------------------------------
 pom.xml                                         |  36 +-
 runners/apex/pom.xml                            |   2 +-
 sdks/common/fn-api/pom.xml                      | 111 +++
 .../fn-api/src/main/proto/beam_fn_api.proto     | 771 +++++++++++++++++++
 sdks/common/pom.xml                             |  38 +
 .../src/main/resources/beam/findbugs-filter.xml |  32 +-
 sdks/java/harness/pom.xml                       | 167 ++++
 .../org/apache/beam/fn/harness/FnHarness.java   | 131 ++++
 .../harness/channel/ManagedChannelFactory.java  |  80 ++
 .../harness/channel/SocketAddressFactory.java   |  64 ++
 .../beam/fn/harness/channel/package-info.java   |  22 +
 .../fn/harness/control/BeamFnControlClient.java | 165 ++++
 .../harness/control/ProcessBundleHandler.java   | 334 ++++++++
 .../fn/harness/control/RegisterHandler.java     |  92 +++
 .../beam/fn/harness/control/package-info.java   |  22 +
 .../BeamFnDataBufferingOutboundObserver.java    | 135 ++++
 .../beam/fn/harness/data/BeamFnDataClient.java  |  64 ++
 .../fn/harness/data/BeamFnDataGrpcClient.java   | 122 +++
 .../harness/data/BeamFnDataGrpcMultiplexer.java | 140 ++++
 .../harness/data/BeamFnDataInboundObserver.java |  81 ++
 .../beam/fn/harness/data/package-info.java      |  22 +
 .../fn/harness/fake/FakeAggregatorFactory.java  |  52 ++
 .../beam/fn/harness/fake/FakeStepContext.java   |  70 ++
 .../beam/fn/harness/fake/package-info.java      |  22 +
 .../harness/fn/CloseableThrowingConsumer.java   |  23 +
 .../beam/fn/harness/fn/ThrowingBiFunction.java  |  32 +
 .../beam/fn/harness/fn/ThrowingConsumer.java    |  32 +
 .../beam/fn/harness/fn/ThrowingFunction.java    |  32 +
 .../beam/fn/harness/fn/ThrowingRunnable.java    |  30 +
 .../apache/beam/fn/harness/fn/package-info.java |  22 +
 .../fn/harness/logging/BeamFnLoggingClient.java | 308 ++++++++
 .../beam/fn/harness/logging/package-info.java   |  22 +
 .../apache/beam/fn/harness/package-info.java    |  22 +
 .../beam/fn/harness/stream/AdvancingPhaser.java |  36 +
 .../harness/stream/BufferingStreamObserver.java | 166 ++++
 .../fn/harness/stream/DirectStreamObserver.java |  71 ++
 .../ForwardingClientResponseObserver.java       |  63 ++
 .../harness/stream/StreamObserverFactory.java   |  91 +++
 .../beam/fn/harness/stream/package-info.java    |  22 +
 .../beam/runners/core/BeamFnDataReadRunner.java | 104 +++
 .../runners/core/BeamFnDataWriteRunner.java     |  87 +++
 .../beam/runners/core/BoundedSourceRunner.java  | 105 +++
 .../apache/beam/runners/core/package-info.java  |  22 +
 .../apache/beam/fn/harness/FnHarnessTest.java   | 130 ++++
 .../channel/ManagedChannelFactoryTest.java      |  74 ++
 .../channel/SocketAddressFactoryTest.java       |  56 ++
 .../control/BeamFnControlClientTest.java        | 182 +++++
 .../control/ProcessBundleHandlerTest.java       | 674 ++++++++++++++++
 .../fn/harness/control/RegisterHandlerTest.java |  80 ++
 ...BeamFnDataBufferingOutboundObserverTest.java | 142 ++++
 .../harness/data/BeamFnDataGrpcClientTest.java  | 309 ++++++++
 .../data/BeamFnDataGrpcMultiplexerTest.java     |  96 +++
 .../data/BeamFnDataInboundObserverTest.java     | 116 +++
 .../logging/BeamFnLoggingClientTest.java        | 169 ++++
 .../fn/harness/stream/AdvancingPhaserTest.java  |  48 ++
 .../stream/BufferingStreamObserverTest.java     | 146 ++++
 .../stream/DirectStreamObserverTest.java        | 139 ++++
 .../ForwardingClientResponseObserverTest.java   |  60 ++
 .../stream/StreamObserverFactoryTest.java       |  84 ++
 .../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 ++
 .../runners/core/BeamFnDataReadRunnerTest.java  | 187 +++++
 .../runners/core/BeamFnDataWriteRunnerTest.java | 155 ++++
 .../runners/core/BoundedSourceRunnerTest.java   | 113 +++
 sdks/java/pom.xml                               |   1 +
 sdks/pom.xml                                    |   1 +
 68 files changed, 7514 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index d09bf59..a53453b 100644
--- a/pom.xml
+++ b/pom.xml
@@ -117,7 +117,7 @@
     <google-clients.version>1.22.0</google-clients.version>
     <google-cloud-bigdataoss.version>1.4.5</google-cloud-bigdataoss.version>
     <google-cloud-dataflow-java-proto-library-all.version>0.5.160304</google-cloud-dataflow-java-proto-library-all.version>
-    <guava.version>19.0</guava.version>
+    <guava.version>20.0</guava.version>
     <grpc.version>1.0.1</grpc.version>
     <hamcrest.version>1.3</hamcrest.version>
     <jackson.version>2.7.2</jackson.version>
@@ -127,7 +127,7 @@
     <mockito.version>1.9.5</mockito.version>
     <netty.version>4.1.3.Final</netty.version>
     <os-maven-plugin.version>1.4.0.Final</os-maven-plugin.version>
-    <protobuf.version>3.0.0</protobuf.version>
+    <protobuf.version>3.1.0</protobuf.version>
     <pubsub.version>v1-rev10-1.22.0</pubsub.version>
     <slf4j.version>1.7.14</slf4j.version>
     <stax2.version>3.1.4</stax2.version>
@@ -314,6 +314,11 @@
 
   <dependencyManagement>
     <dependencies>
+      <dependency>
+        <groupId>org.apache.beam</groupId>
+        <artifactId>beam-sdks-common-fn-api</artifactId>
+        <version>${project.version}</version>
+      </dependency>
 
       <dependency>
         <groupId>org.apache.beam</groupId>
@@ -729,6 +734,13 @@
       </dependency>
 
       <dependency>
+        <groupId>io.netty</groupId>
+        <artifactId>netty-transport-native-epoll</artifactId>
+        <version>${netty.version}</version>
+        <classifier>linux-x86_64</classifier>
+      </dependency>
+
+      <dependency>
         <groupId>org.apache.avro</groupId>
         <artifactId>avro</artifactId>
         <version>${avro.version}</version>
@@ -741,6 +753,12 @@
       </dependency>
 
       <dependency>
+        <groupId>com.google.errorprone</groupId>
+        <artifactId>error_prone_annotations</artifactId>
+        <version>2.0.13</version>
+      </dependency>
+
+      <dependency>
         <groupId>joda-time</groupId>
         <artifactId>joda-time</artifactId>
         <version>${joda.version}</version>
@@ -824,9 +842,23 @@
   </dependencyManagement>
 
   <build>
+    <extensions>
+      <extension>
+        <groupId>kr.motd.maven</groupId>
+        <artifactId>os-maven-plugin</artifactId>
+        <version>${os-maven-plugin.version}</version>
+      </extension>
+    </extensions>
+
     <pluginManagement>
       <plugins>
         <plugin>
+          <groupId>org.xolstice.maven.plugins</groupId>
+          <artifactId>protobuf-maven-plugin</artifactId>
+          <version>0.5.0</version>
+        </plugin>
+
+        <plugin>
           <groupId>org.apache.maven.plugins</groupId>
           <artifactId>maven-checkstyle-plugin</artifactId>
           <version>2.17</version>

http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/runners/apex/pom.xml
----------------------------------------------------------------------
diff --git a/runners/apex/pom.xml b/runners/apex/pom.xml
index 7ae07e2..5e16083 100644
--- a/runners/apex/pom.xml
+++ b/runners/apex/pom.xml
@@ -226,7 +226,7 @@
                 <ignoredUsedUndeclaredDependency>org.slf4j:slf4j-api:jar:1.7.14</ignoredUsedUndeclaredDependency>
                 <ignoredUsedUndeclaredDependency>org.apache.hadoop:hadoop-common:jar:2.6.0</ignoredUsedUndeclaredDependency>
                 <ignoredUsedUndeclaredDependency>joda-time:joda-time:jar:2.4</ignoredUsedUndeclaredDependency>
-                <ignoredUsedUndeclaredDependency>com.google.guava:guava:jar:19.0</ignoredUsedUndeclaredDependency>
+                <ignoredUsedUndeclaredDependency>com.google.guava:guava:jar:20.0</ignoredUsedUndeclaredDependency>
               </ignoredUsedUndeclaredDependencies>
             </configuration>
           </execution>

http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/sdks/common/fn-api/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/common/fn-api/pom.xml b/sdks/common/fn-api/pom.xml
new file mode 100644
index 0000000..72788d0
--- /dev/null
+++ b/sdks/common/fn-api/pom.xml
@@ -0,0 +1,111 @@
+<?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>
+
+  <packaging>jar</packaging>
+  <parent>
+    <groupId>org.apache.beam</groupId>
+    <artifactId>beam-sdks-common-parent</artifactId>
+    <version>0.6.0-SNAPSHOT</version>
+    <relativePath>../pom.xml</relativePath>
+  </parent>
+
+  <artifactId>beam-sdks-common-fn-api</artifactId>
+  <name>Apache Beam :: SDKs :: Common :: Fn API</name>
+  <description>This artifact generates the stub bindings.</description>
+
+  <build>
+    <resources>
+      <resource>
+        <directory>src/main/resources</directory>
+        <filtering>true</filtering>
+      </resource>
+      <resource>
+        <directory>${project.build.directory}/original_sources_to_package</directory>
+      </resource>
+    </resources>
+
+    <plugins>
+      <!-- Skip the checkstyle plugin on generated code -->
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-checkstyle-plugin</artifactId>
+        <configuration>
+          <skip>true</skip>
+        </configuration>
+      </plugin>
+
+      <!-- Skip the findbugs plugin on generated code -->
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>findbugs-maven-plugin</artifactId>
+        <configuration>
+          <skip>true</skip>
+        </configuration>
+      </plugin>
+
+      <plugin>
+        <groupId>org.xolstice.maven.plugins</groupId>
+        <artifactId>protobuf-maven-plugin</artifactId>
+        <configuration>
+          <protocArtifact>com.google.protobuf:protoc:${protobuf.version}:exe:${os.detected.classifier}</protocArtifact>
+          <pluginId>grpc-java</pluginId>
+          <pluginArtifact>io.grpc:protoc-gen-grpc-java:${grpc.version}:exe:${os.detected.classifier}</pluginArtifact>
+        </configuration>
+        <executions>
+          <execution>
+            <goals>
+              <goal>compile</goal>
+              <goal>compile-custom</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+
+  <dependencies>
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.protobuf</groupId>
+      <artifactId>protobuf-java</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-core</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-protobuf</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-stub</artifactId>
+    </dependency>
+  </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/sdks/common/fn-api/src/main/proto/beam_fn_api.proto
----------------------------------------------------------------------
diff --git a/sdks/common/fn-api/src/main/proto/beam_fn_api.proto b/sdks/common/fn-api/src/main/proto/beam_fn_api.proto
new file mode 100644
index 0000000..3ac0fbf
--- /dev/null
+++ b/sdks/common/fn-api/src/main/proto/beam_fn_api.proto
@@ -0,0 +1,771 @@
+/*
+ * 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.
+ */
+
+/*
+ * Protocol Buffers describing the Fn API and boostrapping.
+ *
+ * TODO: Usage of plural names in lists looks awkward in Java
+ * e.g. getOutputsMap, addCodersBuilder
+ *
+ * TODO: gRPC / proto field names conflict with generated code
+ * e.g. "class" in java, "output" in python
+ */
+
+syntax = "proto3";
+
+/* TODO: Consider consolidating common components in another package
+ * and lanaguage namespaces for re-use with Runner Api.
+ */
+
+package org.apache.beam.fn.v1;
+
+option java_package = "org.apache.beam.fn.v1";
+option java_outer_classname = "BeamFnApi";
+
+import "google/protobuf/any.proto";
+import "google/protobuf/timestamp.proto";
+
+/*
+ * Constructs that define the pipeline shape.
+ *
+ * These are mostly unstable due to the missing pieces to be shared with
+ * the Runner Api like windowing strategy, display data, .... There are still
+ * some modelling questions related to whether a side input is modelled
+ * as another field on a PrimitiveTransform or as part of inputs and we
+ * still are missing things like the CompositeTransform.
+ */
+
+// A representation of an input or output definition on a primitive transform.
+// Stable
+message Target {
+  // A repeated list of target definitions.
+  message List {
+    repeated Target target = 1;
+  }
+
+  // (Required) The id of the PrimitiveTransform which is the target.
+  int64 primitive_transform_reference = 1;
+
+  // (Required) The local name of an input or output defined on the primitive
+  // transform.
+  string name = 2;
+}
+
+// Information defining a PCollection
+message PCollection {
+  // (Required) A reference to a coder.
+  int64 coder_reference = 1;
+
+  // TODO: Windowing strategy, ...
+}
+
+// A primitive transform within Apache Beam.
+message PrimitiveTransform {
+  // (Required) A pipeline level unique id which can be used as a reference to
+  // refer to this.
+  int64 id = 1;
+
+  // (Required) A function spec that is used by this primitive
+  // transform to process data.
+  FunctionSpec function_spec = 2;
+
+  // A map of distinct input names to target definitions.
+  // For example, in CoGbk this represents the tag name associated with each
+  // distinct input name and a list of primitive transforms that are associated
+  // with the specified input.
+  map<string, Target.List> inputs = 3;
+
+  // A map from local output name to PCollection definitions. For example, in
+  // DoFn this represents the tag name associated with each distinct output.
+  map<string, PCollection> outputs = 4;
+
+  // TODO: Should we model side inputs as a special type of input for a
+  // primitive transform or should it be modeled as the relationship that
+  // the predecessor input will be a view primitive transform.
+  // A map of from side input names to side inputs.
+  map<string, SideInput> side_inputs = 5;
+
+  // The user name of this step.
+  // TODO: This should really be in display data and not at this level
+  string step_name = 6;
+}
+
+/*
+ * User Definable Functions
+ *
+ * This is still unstable mainly due to how we model the side input.
+ */
+
+// Defines the common elements of user-definable functions, to allow the SDK to
+// express the information the runner needs to execute work.
+// Stable
+message FunctionSpec {
+  // (Required) A pipeline level unique id which can be used as a reference to
+  // refer to this.
+  int64 id = 1;
+
+  // (Required) A globally unique name representing this user definable
+  // function.
+  //
+  // User definable functions use the urn encodings registered such that another
+  // may implement the user definable function within another language.
+  //
+  // For example:
+  //    urn:org.apache.beam:coder:kv:1.0
+  string urn = 2;
+
+  // (Required) Reference to specification of execution environment required to
+  // invoke this function.
+  int64 environment_reference = 3;
+
+  // Data used to parameterize this function. Depending on the urn, this may be
+  // optional or required.
+  google.protobuf.Any data = 4;
+}
+
+message SideInput {
+  // TODO: Coder?
+
+  // For RunnerAPI.
+  Target input = 1;
+
+  // For FnAPI.
+  FunctionSpec view_fn = 2;
+}
+
+// Defines how to encode values into byte streams and decode values from byte
+// streams. A coder can be parameterized by additional properties which may or
+// may not be language agnostic.
+//
+// Coders using the urn:org.apache.beam:coder namespace must have their
+// encodings registered such that another may implement the encoding within
+// another language.
+//
+// For example:
+//    urn:org.apache.beam:coder:kv:1.0
+//    urn:org.apache.beam:coder:iterable:1.0
+// Stable
+message Coder {
+  // TODO: This looks weird when compared to the other function specs
+  // which use URN to differentiate themselves. Should "Coder" be embedded
+  // inside the FunctionSpec data block.
+
+  // The data associated with this coder used to reconstruct it.
+  FunctionSpec function_spec = 1;
+
+  // A list of component coder references.
+  //
+  // For a key-value coder, there must be exactly two component coder references
+  // where the first reference represents the key coder and the second reference
+  // is the value coder.
+  //
+  // For an iterable coder, there must be exactly one component coder reference
+  // representing the value coder.
+  //
+  // TODO: Perhaps this is redundant with the data of the FunctionSpec
+  // for known coders?
+  repeated int64 component_coder_reference = 2;
+}
+
+// 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
+message RemoteGrpcPort {
+  // (Required) An API descriptor which describes where to
+  // connect to including any authentication that is required.
+  ApiServiceDescriptor api_service_descriptor = 1;
+}
+
+/*
+ * Control Plane API
+ *
+ * Progress reporting and splitting still need further vetting. Also, this may change
+ * with the addition of new types of instructions/responses related to metrics.
+ */
+
+// An API that describes the work that a SDK Fn Harness is meant to do.
+// Stable
+service BeamFnControl {
+  // Instructions sent by the runner to the SDK requesting different types
+  // of work.
+  rpc Control(
+    // A stream of responses to instructions the SDK was asked to be performed.
+    stream InstructionResponse
+  ) returns (
+    // A stream of instructions requested of the SDK to be performed.
+    stream InstructionRequest
+  ) {}
+}
+
+// A request sent by a runner which it the SDK is asked to fulfill.
+// Stable
+message InstructionRequest {
+  // (Required) An unique identifier provided by the runner which represents
+  // this requests execution. The InstructionResponse MUST have the matching id.
+  int64 instruction_id = 1;
+
+  // (Required) A request that the SDK Harness needs to interpret.
+  oneof request {
+    RegisterRequest register = 1000;
+    ProcessBundleRequest process_bundle = 1001;
+    ProcessBundleProgressRequest process_bundle_progress = 1002;
+    ProcessBundleSplitRequest process_bundle_split = 1003;
+  }
+}
+
+// The response for an associated request the SDK had been asked to fulfill.
+// Stable
+message InstructionResponse {
+  // (Required) A reference provided by the runner which represents a requests
+  // execution. The InstructionResponse MUST have the matching id when
+  // responding to the runner.
+  int64 instruction_id = 1;
+
+  // If this is specified, then this instruction has failed.
+  // A human readable string representing the reason as to why processing has
+  // failed.
+  string error = 2;
+
+  // If the instruction did not fail, it is required to return an equivalent
+  // response type depending on the request this matches.
+  oneof response {
+    RegisterResponse register = 1000;
+    ProcessBundleResponse process_bundle = 1001;
+    ProcessBundleProgressResponse process_bundle_progress = 1002;
+    ProcessBundleSplitResponse process_bundle_split = 1003;
+  }
+}
+
+// A list of objects which can be referred to by the runner in
+// future requests.
+// Stable
+message RegisterRequest {
+  // (Optional) The set of descriptors used to process bundles.
+  repeated ProcessBundleDescriptor process_bundle_descriptor = 1;
+}
+
+// Stable
+message RegisterResponse {
+}
+
+// A descriptor of references used when processing a bundle.
+// Stable
+message ProcessBundleDescriptor {
+  // (Required) A pipeline level unique id which can be used as a reference to
+  // refer to this.
+  int64 id = 1;
+
+  // (Required) A list of primitive transforms that should
+  // be used to construct the bundle processing graph.
+  repeated PrimitiveTransform primitive_transform = 2;
+
+  // (Required) The set of all coders referenced in this bundle.
+  repeated Coder coders = 4;
+}
+
+// A request to process a given bundle.
+// Stable
+message ProcessBundleRequest {
+  int64 process_bundle_descriptor_reference = 1;
+}
+
+// Stable
+message ProcessBundleResponse {
+}
+
+message ProcessBundleProgressRequest {
+  // (Required) A reference to an active process bundle request with the given
+  // instruction id.
+  int64 instruction_reference = 1;
+}
+
+message ProcessBundleProgressResponse {
+  // (Required) The finished amount of work. A monotonically increasing
+  // unitless measure of work finished.
+  double finished_work = 1;
+
+  // (Required) The known amount of backlog for the process bundle request.
+  // Computed as:
+  //   (estimated known work - finish work) / finished work
+  double backlog = 2;
+}
+
+message ProcessBundleSplitRequest {
+  // (Required) A reference to an active process bundle request with the given
+  // instruction id.
+  int64 instruction_reference = 1;
+
+  // (Required) The fraction of work (when compared to the known amount of work)
+  // the process bundle request should try to split at.
+  double fraction = 2;
+}
+
+// urn:org.apache.beam:restriction:element-count:1.0
+message ElementCountRestriction {
+  // A restriction representing the number of elements that should be processed.
+  // Effectively the range [0, count]
+  int64 count = 1;
+}
+
+// urn:org.apache.beam:restriction:element-count-skip:1.0
+message ElementCountSkipRestriction {
+  // A restriction representing the number of elements that should be skipped.
+  // Effectively the range (count, infinity]
+  int64 count = 1;
+}
+
+// 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 \u22c3 R_todo
+message PrimitiveTransformSplit {
+  // (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.
+  int64 primitive_transform_reference = 1;
+
+  // (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.
+  FunctionSpec completed_restriction = 2;
+
+  // (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.
+  FunctionSpec remaining_restriction = 3;
+}
+
+message ProcessBundleSplitResponse {
+  // (Optional) A set of split responses for a currently active work item.
+  //
+  // If primitive transform B is a descendant of primitive transform A and both
+  // A and B report a split. Then B's restriction is reported as an element
+  // restriction pair and thus the fully reported restriction is:
+  //   R = A_done
+  //     \u22c3 (A_boundary \u22c2 B_done)
+  //     \u22c3 (A_boundary \u22c2 B_todo)
+  //     \u22c3 A_todo
+  // If there is a decendant of B named C, then C would similarly report a
+  // set of element pair restrictions.
+  //
+  // This restriction is processed and completed by the currently active process
+  // bundle request:
+  //   A_done \u22c3 (A_boundary \u22c2 B_done)
+  // and these restrictions will be processed by future process bundle requests:
+  //   A_boundary \u22c2 B_todo (passed to SDF B directly)
+  //   A_todo (passed to SDF A directly)
+
+  // 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
+  //     \u22c3 (A_boundary \u22c2 B_done)
+  //     \u22c3 (A_boundary \u22c2 B_todo)
+  //     \u22c3 (A_boundary \u22c2 B_todo)
+  //     \u22c3 (A_boundary \u22c2 C_todo)
+  //     \u22c3 A_todo
+  // If there is no descendant of B or C also reporting a split, than
+  //   B_boundary = \u2205 and C_boundary = \u2205
+  //
+  // This restriction is processed and completed by the currently active process
+  // bundle request:
+  //   A_done \u22c3 (A_boundary \u22c2 B_done)
+  //          \u22c3 (A_boundary \u22c2 C_done)
+  // and these restrictions will be processed by future process bundle requests:
+  //   A_boundary \u22c2 B_todo (passed to SDF B directly)
+  //   A_boundary \u22c2 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.
+  repeated PrimitiveTransformSplit splits = 1;
+}
+
+/*
+ * Data Plane API
+ */
+
+// Messages used to represent logical byte streams.
+// Stable
+message Elements {
+  // Represents multiple encoded elements in nested context for a given named
+  // instruction and target.
+  message Data {
+    // (Required) A reference to an active instruction request with the given
+    // instruction id.
+    int64 instruction_reference = 1;
+
+    // (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 = 2;
+
+    // (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.
+    bytes data = 3;
+  }
+
+  // (Required) A list containing parts of logical byte streams.
+  repeated Data data = 1;
+}
+
+// Stable
+service BeamFnData {
+  // Used to send data between harnesses.
+  rpc Data(
+    // A stream of data representing input.
+    stream Elements
+  ) returns (
+    // A stream of data representing output.
+    stream Elements
+  ) {}
+}
+
+/*
+ * State API
+ *
+ * This is just a high level sketch of how this could work. There is still
+ * a lot of work with respect to how the key spaces for the different types
+ * of access required (side inputs, user state, ...) and how state caching
+ * works across bundles.
+ */
+
+message StateRequest {
+  // (Required) An unique identifier provided by the SDK which represents this
+  // requests execution. The StateResponse must have the matching id.
+  int64 id = 1;
+
+  // (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.
+  int64 instruction_reference = 2;
+
+  // At least one of the following fields should be populated.
+  // Also, no request should use a state key referred to in another state key.
+
+  // (Optional) A request to get state.
+  repeated StateGetRequest get = 3;
+
+  // (Optional) A request to append to state.
+  repeated StateAppendRequest append = 4;
+
+  // (Optional) A request to clear state.
+  repeated StateClearRequest clear = 5;
+}
+
+message StateResponse {
+  // (Required) A reference provided by the SDK which represents a requests
+  // execution. The StateResponse must have the matching id when responding
+  // to the SDK.
+  int64 id = 1;
+
+  // (Required) The associated instruction id of the work that is currently
+  // being processed.
+  int64 instruction_reference = 2;
+
+  // (Required) A key to associate with the version of this state. Allows for
+  // SDKs to share state across work items if they have the same cache key and
+  // state key.
+  bytes cache_key = 3;
+
+  // (Optional) If this is specified, then the state request has failed.
+  // A human readable string representing the reason as to why the request
+  // failed.
+  string error = 4;
+
+  // For every field populated in the StateRequest, there is a matching field in
+  // the StateResponse.
+
+  // (Optional) A response to getting state.
+  repeated StateGetResponse get = 5;
+
+  // (Optional) A response to appending to state.
+  repeated StateAppendResponse append = 6;
+
+  // (Optional) A response to clearing state.
+  repeated StateClearResponse clear = 7;
+}
+
+service BeamFnState {
+  // Used to get/append/clear state stored by the runner on behalf of the SDK.
+  rpc State(
+    // A stream of state instructions requested of the runner.
+    stream StateRequest
+  ) returns (
+    // A stream of responses to state instructions the runner was asked to be
+    // performed.
+    stream StateResponse
+  ) {}
+}
+
+
+// TODO: Resolve with the other State API.
+service SimpleBeamFnState {
+  // Gets the elements associated with the given key.
+  rpc Get(StateKey) returns (Elements.Data) {}
+  // Appends elements to a given state bag.
+  rpc Append(SimpleStateAppendRequest) returns (Empty) {}
+  // Clears a given state bag.
+  rpc Clear(StateKey) returns (Empty) {}
+}
+
+message Empty {
+}
+
+message SimpleStateAppendRequest {
+  StateKey state_key = 1;
+  repeated bytes data = 2;
+}
+
+message StateKey {
+  // (Required) Represents the namespace for the state. If this state is for a
+  // DoFn, then this reference is expected to point to the DoFn. If this state
+  // is for a side input, then this is expected to reference the ViewFn.
+  int64 function_spec_reference = 1;
+
+  // (Required) The bytes of the window which this state request is for encoded
+  // in the outer context.
+  bytes window = 2;
+
+  // (Required) The user key for which the value was encoded in the outer
+  // context.
+  bytes key = 3;
+}
+
+message StateKeyOrIterable {
+  // One of the two fields below are required to be set.
+  // If state key is set, then the State API should be invoked to fetch the
+  // values allowing one to restart the iterable. Otherwise the bytes for the
+  // entire iterable are represented and should be decoded using an iterable
+  // coder using the outer context.
+  StateKey state_key = 1;
+  repeated bytes iterable = 2;
+}
+
+// A request to get state for the given state key.
+message StateGetRequest {
+  // A state key encoded in the outer context.
+  StateKey state_key = 1;
+}
+
+// A response to get state for the given state key.
+message StateGetResponse {
+  // A state key encoded in the outer context.
+  StateKey state_key = 1;
+
+  oneof state {
+    // A description of an input port which will stream the state data.
+    RemoteGrpcPort remote_grpc_port = 1000;
+  }
+}
+
+// A request to append state for the given state key.
+message StateAppendRequest {
+  // A state key encoded in the outer context.
+  StateKey state_key  = 1;
+}
+
+// A response to append state for the given state key.
+message StateAppendResponse {
+  // A state key encoded in the outer context.
+  StateKey state_key = 1;
+
+  oneof state {
+    // A description of an output port which to stream the state data to.
+    RemoteGrpcPort remote_grpc_port = 1000;
+  }
+}
+
+// A request to clear state for the given state key.
+message StateClearRequest {
+  // A state key encoded in the outer context.
+  StateKey state_key = 1;
+}
+
+// A response to clear state for the given state key.
+message StateClearResponse {
+}
+
+/*
+ * Logging API
+ *
+ * This is very stable. There can be some changes to how we define a LogEntry,
+ * to increase/decrease the severity types, the way we format an exception/stack
+ * trace, or the log site.
+ */
+
+// A log entry
+message LogEntry {
+  // A list of log entries, enables buffering and batching of multiple
+  // log messages using the logging API.
+  message List {
+    // (Required) One or or more log messages.
+    repeated LogEntry log_entries = 1;
+  }
+
+  // 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.
+  enum Severity {
+    // Trace level information, also the default log level unless
+    // another severity is specified.
+    TRACE = 0;
+    // Debugging information.
+    DEBUG = 10;
+    // Normal events.
+    INFO = 20;
+    // Normal but significant events, such as start up, shut down, or
+    // configuration.
+    NOTICE = 30;
+    // Warning events might cause problems.
+    WARN = 40;
+    // Error events are likely to cause problems.
+    ERROR = 50;
+    // Critical events cause severe problems or brief outages and may
+    // indicate that a person must take action.
+    CRITICAL = 60;
+  }
+
+  // (Required) The severity of the log statement.
+  Severity severity = 1;
+
+  // (Required) The time at which this log statement occurred.
+  google.protobuf.Timestamp timestamp = 2;
+
+  // (Required) A human readable message.
+  string message = 3;
+
+  // (Optional) An optional trace of the functions involved. For example, in
+  // Java this can include multiple causes and multiple suppressed exceptions.
+  string trace = 4;
+
+  // (Optional) A reference to the instruction this log statement is associated
+  // with.
+  int64 instruction_reference = 5;
+
+  // (Optional) A reference to the primitive transform this log statement is
+  // associated with.
+  int64 primitive_transform_reference = 6;
+
+  // (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++)
+  string log_location = 7;
+
+  // (Optional) The name of the thread this log statement is associated with.
+  string thread = 8;
+}
+
+message LogControl {
+}
+
+// Stable
+service BeamFnLogging {
+  // Allows for the SDK to emit log entries which the runner can
+  // associate with the active job.
+  rpc Logging(
+    // A stream of log entries batched into lists emitted by the SDK harness.
+    stream LogEntry.List
+  ) returns (
+    // A stream of log control messages used to configure the SDK.
+    stream LogControl
+  ) {}
+}
+
+/*
+ * Environment types
+ */
+message ApiServiceDescriptor {
+  // (Required) A pipeline level unique id which can be used as a reference to
+  // refer to this.
+  int64 id = 1;
+
+  // (Required) The URL to connect to.
+  string url = 2;
+
+  // (Optional) The method for authentication. If unspecified, access to the
+  // url is already being performed in a trusted context (e.g. localhost,
+  // private network).
+  oneof authentication {
+    OAuth2ClientCredentialsGrant oauth2_client_credentials_grant = 3;
+  }
+}
+
+message OAuth2ClientCredentialsGrant {
+  // (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.
+  string url = 1;
+}
+
+// A Docker container configuration for launching the SDK Fn Harness to execute
+// user specified functions.
+message DockerContainer {
+  // (Required) A pipeline level unique id which can be used as a reference to
+  // refer to this.
+  int64 id = 1;
+
+  // (Required) The Docker container URI
+  // For example "dataflow.gcr.io/v1beta3/java-batch:1.5.1"
+  string uri = 2;
+
+  // (Optional) Docker registry specification.
+  // If unspecified, the uri is expected to be able to be fetched without
+  // requiring additional configuration by a runner.
+  int64 registry_reference = 3;
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/sdks/common/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/common/pom.xml b/sdks/common/pom.xml
new file mode 100644
index 0000000..8364d9a
--- /dev/null
+++ b/sdks/common/pom.xml
@@ -0,0 +1,38 @@
+<?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-parent</artifactId>
+    <version>0.6.0-SNAPSHOT</version>
+    <relativePath>../pom.xml</relativePath>
+  </parent>
+
+  <artifactId>beam-sdks-common-parent</artifactId>
+
+  <packaging>pom</packaging>
+
+  <name>Apache Beam :: SDKs :: Common</name>
+
+  <modules>
+    <module>fn-api</module>
+  </modules>
+</project>

http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml
----------------------------------------------------------------------
diff --git a/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml b/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml
index 35b5ed3..91ab9be 100644
--- a/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml
+++ b/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml
@@ -34,12 +34,42 @@
           unapproved artifact license.
         -->
   <Match>
+    <Class name="org.apache.beam.fn.harness.control.BeamFnControlClient$InboundObserver"/>
+    <Method name="onCompleted"/>
+    <Bug pattern="NP_NONNULL_PARAM_VIOLATION"/>
+    <!--
+    Findbugs limitation when using Void typed CompletableFuture. This is a limitation of FindBugs as discussed here:
+    https://github.com/findbugsproject/findbugs/issues/79
+    -->
+  </Match>
+
+  <Match>
+    <Class name="org.apache.beam.fn.harness.data.BeamFnDataInboundObserver"/>
+    <Method name="accept"/>
+    <Bug pattern="NP_NONNULL_PARAM_VIOLATION"/>
+    <!--
+    Findbugs limitation when using Void typed CompletableFuture. This is a limitation of FindBugs as discussed here:
+    https://github.com/findbugsproject/findbugs/issues/79
+    -->
+  </Match>
+
+  <Match>
+    <Class name="org.apache.beam.fn.harness.logging.BeamFnLoggingClient$LogControlObserver"/>
+    <Method name="onCompleted"/>
+    <Bug pattern="NP_NONNULL_PARAM_VIOLATION"/>
+    <!--
+    Findbugs limitation when using Void typed CompletableFuture. This is a limitation of FindBugs as discussed here:
+    https://github.com/findbugsproject/findbugs/issues/79
+    -->
+  </Match>
+
+  <Match>
     <Class name="org.apache.beam.sdk.coders.AvroCoder$SerializableSchemaSupplier"/>
     <Field name="schema"/>
     <Bug pattern="SE_BAD_FIELD"/>
     <!--
     writeReplace makes this object serializable. This is a limitation of FindBugs as discussed here:
-     http://stackoverflow.com/questions/26156523/is-writeobject-not-neccesary-using-the-serialization-proxy-pattern
+    http://stackoverflow.com/questions/26156523/is-writeobject-not-neccesary-using-the-serialization-proxy-pattern
     -->
   </Match>
 

http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/sdks/java/harness/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/harness/pom.xml b/sdks/java/harness/pom.xml
new file mode 100644
index 0000000..e164ee0
--- /dev/null
+++ b/sdks/java/harness/pom.xml
@@ -0,0 +1,167 @@
+<?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>
+
+  <packaging>jar</packaging>
+
+  <parent>
+    <groupId>org.apache.beam</groupId>
+    <artifactId>beam-sdks-java-parent</artifactId>
+    <version>0.6.0-SNAPSHOT</version>
+    <relativePath>../pom.xml</relativePath>
+  </parent>
+
+  <artifactId>beam-sdks-java-harness</artifactId>
+  <name>Apache Beam :: SDKs :: Java :: Harness</name>
+  <description>This contains the SDK Fn Harness for Beam Java</description>
+
+  <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-sdks-java-core</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>beam-sdks-java-core</artifactId>
+      <version>${project.version}</version>
+      <classifier>tests</classifier>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>beam-runners-core-java</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>beam-runners-google-cloud-dataflow-java</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>beam-sdks-common-fn-api</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-databind</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.auto.value</groupId>
+      <artifactId>auto-value</artifactId>
+      <scope>provided</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.errorprone</groupId>
+      <artifactId>error_prone_annotations</artifactId>
+      <scope>provided</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.code.findbugs</groupId>
+      <artifactId>jsr305</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.protobuf</groupId>
+      <artifactId>protobuf-lite</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.protobuf</groupId>
+      <artifactId>protobuf-java</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-core</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-netty</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-stub</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>io.netty</groupId>
+      <artifactId>netty-transport-native-epoll</artifactId>
+      <classifier>linux-x86_64</classifier>
+    </dependency>
+
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-api</artifactId>
+    </dependency>
+
+    <!-- test dependencies -->
+    <dependency>
+      <groupId>org.hamcrest</groupId>
+      <artifactId>hamcrest-all</artifactId>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-jdk14</artifactId>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/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
new file mode 100644
index 0000000..3e06f38
--- /dev/null
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java
@@ -0,0 +1,131 @@
+/*
+ * 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;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.protobuf.TextFormat;
+import java.io.PrintStream;
+import java.util.EnumMap;
+import org.apache.beam.fn.harness.channel.ManagedChannelFactory;
+import org.apache.beam.fn.harness.control.BeamFnControlClient;
+import org.apache.beam.fn.harness.control.ProcessBundleHandler;
+import org.apache.beam.fn.harness.control.RegisterHandler;
+import org.apache.beam.fn.harness.data.BeamFnDataGrpcClient;
+import org.apache.beam.fn.harness.fn.ThrowingFunction;
+import org.apache.beam.fn.harness.logging.BeamFnLoggingClient;
+import org.apache.beam.fn.harness.stream.StreamObserverFactory;
+import org.apache.beam.fn.v1.BeamFnApi;
+import org.apache.beam.sdk.options.GcsOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.util.IOChannelUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Main entry point into the Beam SDK Fn Harness for Java.
+ *
+ * <p>This entry point expects the following environment variables:
+ * <ul>
+ *   <li>LOGGING_API_SERVICE_DESCRIPTOR: A
+ *   {@link org.apache.beam.fn.v1.BeamFnApi.ApiServiceDescriptor} encoded as text
+ *   representing the endpoint that is to be connected to for the Beam Fn Logging service.</li>
+ *   <li>CONTROL_API_SERVICE_DESCRIPTOR: A
+ *   {@link org.apache.beam.fn.v1.BeamFnApi.ApiServiceDescriptor} encoded as text
+ *   representing the endpoint that is to be connected to for the Beam Fn Control service.</li>
+ *   <li>PIPELINE_OPTIONS: A serialized form of {@link PipelineOptions}. See {@link PipelineOptions}
+ *   for further details.</li>
+ * </ul>
+ */
+public class FnHarness {
+  private static final String CONTROL_API_SERVICE_DESCRIPTOR = "CONTROL_API_SERVICE_DESCRIPTOR";
+  private static final String LOGGING_API_SERVICE_DESCRIPTOR = "LOGGING_API_SERVICE_DESCRIPTOR";
+  private static final String PIPELINE_OPTIONS = "PIPELINE_OPTIONS";
+  private static final Logger LOGGER = LoggerFactory.getLogger(FnHarness.class);
+
+  private static BeamFnApi.ApiServiceDescriptor getApiServiceDescriptor(String env)
+      throws TextFormat.ParseException {
+    BeamFnApi.ApiServiceDescriptor.Builder apiServiceDescriptorBuilder =
+        BeamFnApi.ApiServiceDescriptor.newBuilder();
+    TextFormat.merge(System.getenv(env), apiServiceDescriptorBuilder);
+    return apiServiceDescriptorBuilder.build();
+  }
+
+  public static void main(String[] args) throws Exception {
+    System.out.format("SDK Fn Harness started%n");
+    System.out.format("Logging location %s%n", System.getenv(LOGGING_API_SERVICE_DESCRIPTOR));
+    System.out.format("Control location %s%n", System.getenv(CONTROL_API_SERVICE_DESCRIPTOR));
+    System.out.format("Pipeline options %s%n", System.getenv(PIPELINE_OPTIONS));
+
+    PipelineOptions options = new ObjectMapper().readValue(
+        System.getenv(PIPELINE_OPTIONS), PipelineOptions.class);
+
+    BeamFnApi.ApiServiceDescriptor loggingApiServiceDescriptor =
+        getApiServiceDescriptor(LOGGING_API_SERVICE_DESCRIPTOR);
+
+    BeamFnApi.ApiServiceDescriptor controlApiServiceDescriptor =
+        getApiServiceDescriptor(CONTROL_API_SERVICE_DESCRIPTOR);
+
+    main(options, loggingApiServiceDescriptor, controlApiServiceDescriptor);
+  }
+
+  public static void main(PipelineOptions options,
+      BeamFnApi.ApiServiceDescriptor loggingApiServiceDescriptor,
+      BeamFnApi.ApiServiceDescriptor controlApiServiceDescriptor) throws Exception {
+    IOChannelUtils.registerIOFactories(options);
+
+    ManagedChannelFactory channelFactory = ManagedChannelFactory.from(options);
+    StreamObserverFactory streamObserverFactory = StreamObserverFactory.fromOptions(options);
+    PrintStream originalErrStream = System.err;
+
+    try (BeamFnLoggingClient logging = new BeamFnLoggingClient(
+        options,
+        loggingApiServiceDescriptor,
+        channelFactory::forDescriptor,
+        streamObserverFactory::from)) {
+
+      LOGGER.info("Fn Harness started");
+      EnumMap<BeamFnApi.InstructionRequest.RequestCase,
+              ThrowingFunction<BeamFnApi.InstructionRequest,
+                               BeamFnApi.InstructionResponse.Builder>> handlers =
+          new EnumMap<>(BeamFnApi.InstructionRequest.RequestCase.class);
+
+      RegisterHandler fnApiRegistry = new RegisterHandler();
+      BeamFnDataGrpcClient beamFnDataMultiplexer = new BeamFnDataGrpcClient(
+          options, channelFactory::forDescriptor, streamObserverFactory::from);
+
+      ProcessBundleHandler processBundleHandler =
+          new ProcessBundleHandler(options, fnApiRegistry::getById, beamFnDataMultiplexer);
+      handlers.put(BeamFnApi.InstructionRequest.RequestCase.REGISTER,
+          fnApiRegistry::register);
+      handlers.put(BeamFnApi.InstructionRequest.RequestCase.PROCESS_BUNDLE,
+          processBundleHandler::processBundle);
+      BeamFnControlClient control = new BeamFnControlClient(controlApiServiceDescriptor,
+          channelFactory::forDescriptor,
+          streamObserverFactory::from,
+          handlers);
+
+      LOGGER.info("Entering instruction processing loop");
+      control.processInstructionRequests(options.as(GcsOptions.class).getExecutorService());
+    } catch (Throwable t) {
+      t.printStackTrace(originalErrStream);
+    } finally {
+      originalErrStream.println("Shutting SDK harness down.");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/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
new file mode 100644
index 0000000..d26f4a5
--- /dev/null
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/channel/ManagedChannelFactory.java
@@ -0,0 +1,80 @@
+/*
+ * 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.fn.v1.BeamFnApi.ApiServiceDescriptor;
+import org.apache.beam.runners.dataflow.options.DataflowPipelineDebugOptions;
+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(DataflowPipelineDebugOptions.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)
+          .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)
+          .build();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/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
new file mode 100644
index 0000000..a27d542
--- /dev/null
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/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.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/0b4b2bec/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
new file mode 100644
index 0000000..6323166
--- /dev/null
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/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.fn.harness.channel;

http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/BeamFnControlClient.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/BeamFnControlClient.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/BeamFnControlClient.java
new file mode 100644
index 0000000..7f44a01
--- /dev/null
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/BeamFnControlClient.java
@@ -0,0 +1,165 @@
+/*
+ * 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.control;
+
+import static com.google.common.base.Throwables.getStackTraceAsString;
+
+import com.google.common.util.concurrent.Uninterruptibles;
+import io.grpc.ManagedChannel;
+import io.grpc.stub.StreamObserver;
+import java.util.EnumMap;
+import java.util.concurrent.BlockingDeque;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.LinkedBlockingDeque;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+import org.apache.beam.fn.harness.fn.ThrowingFunction;
+import org.apache.beam.fn.v1.BeamFnApi;
+import org.apache.beam.fn.v1.BeamFnControlGrpc;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A client for the Beam Fn Control API. Uses an unbounded internal queue to pull down
+ * an unbounded number of requests.
+ *
+ * <p>Also can delegate to a set of handlers based upon the
+ * {@link org.apache.beam.fn.v1.BeamFnApi.InstructionRequest.RequestCase request type}.
+ *
+ * <p>When the inbound instruction stream finishes successfully, the {@code onFinish} is
+ * completed successfully signaling to the caller that this client will not produce any more
+ * {@link org.apache.beam.fn.v1.BeamFnApi.InstructionRequest}s. If the inbound instruction stream
+ * errors, the {@code onFinish} is completed exceptionally propagating the failure reason
+ * to the caller and signaling that this client will not produce any more
+ * {@link org.apache.beam.fn.v1.BeamFnApi.InstructionRequest}s.
+ */
+public class BeamFnControlClient {
+  private static final Logger LOGGER = LoggerFactory.getLogger(BeamFnControlClient.class);
+  private static final BeamFnApi.InstructionRequest POISON_PILL =
+      BeamFnApi.InstructionRequest.newBuilder().setInstructionId(Long.MIN_VALUE).build();
+
+  private final StreamObserver<BeamFnApi.InstructionResponse> outboundObserver;
+  private final BlockingDeque<BeamFnApi.InstructionRequest> bufferedInstructions;
+  private final EnumMap<BeamFnApi.InstructionRequest.RequestCase,
+                        ThrowingFunction<BeamFnApi.InstructionRequest,
+                                         BeamFnApi.InstructionResponse.Builder>> handlers;
+  private final CompletableFuture<Void> onFinish;
+
+  public BeamFnControlClient(
+      BeamFnApi.ApiServiceDescriptor apiServiceDescriptor,
+      Function<BeamFnApi.ApiServiceDescriptor, ManagedChannel> channelFactory,
+      BiFunction<Function<StreamObserver<BeamFnApi.InstructionRequest>,
+                          StreamObserver<BeamFnApi.InstructionResponse>>,
+                 StreamObserver<BeamFnApi.InstructionRequest>,
+                 StreamObserver<BeamFnApi.InstructionResponse>> streamObserverFactory,
+      EnumMap<BeamFnApi.InstructionRequest.RequestCase,
+              ThrowingFunction<BeamFnApi.InstructionRequest,
+                               BeamFnApi.InstructionResponse.Builder>> handlers) {
+    this.bufferedInstructions = new LinkedBlockingDeque<>();
+    this.outboundObserver = streamObserverFactory.apply(
+        BeamFnControlGrpc.newStub(channelFactory.apply(apiServiceDescriptor))::control,
+        new InboundObserver());
+    this.handlers = handlers;
+    this.onFinish = new CompletableFuture<>();
+  }
+
+  /**
+   * A {@link StreamObserver} for the inbound stream that completes the future on stream
+   * termination.
+   */
+  private class InboundObserver implements StreamObserver<BeamFnApi.InstructionRequest> {
+    @Override
+    public void onNext(BeamFnApi.InstructionRequest value) {
+      LOGGER.info("InstructionRequest received {}", value);
+      Uninterruptibles.putUninterruptibly(bufferedInstructions, value);
+    }
+
+    @Override
+    public void onError(Throwable t) {
+      placePoisonPillIntoQueue();
+      onFinish.completeExceptionally(t);
+    }
+
+    @Override
+    public void onCompleted() {
+      placePoisonPillIntoQueue();
+      onFinish.complete(null);
+    }
+
+    /**
+     * This method emulates {@link Uninterruptibles#putUninterruptibly} but placing the
+     * element at the front of the queue.
+     *
+     * <p>We place the poison pill at the front of the queue because if the server shutdown,
+     * any remaining instructions can be discarded.
+     */
+    private void placePoisonPillIntoQueue() {
+      while (true) {
+        try {
+          bufferedInstructions.putFirst(POISON_PILL);
+          return;
+        } catch (InterruptedException e) {
+          // Ignored until we place the poison pill into the queue
+        }
+      }
+    }
+  }
+
+  /**
+   * Note that this method continuously submits work to the supplied executor until the
+   * Beam Fn Control server hangs up or fails exceptionally.
+   */
+  public void processInstructionRequests(Executor executor)
+      throws InterruptedException, ExecutionException {
+    BeamFnApi.InstructionRequest request;
+    while ((request = bufferedInstructions.take()) != POISON_PILL) {
+      BeamFnApi.InstructionRequest currentRequest = request;
+      executor.execute(
+          () -> sendInstructionResponse(delegateOnInstructionRequestType(currentRequest)));
+    }
+    onFinish.get();
+  }
+
+  public BeamFnApi.InstructionResponse delegateOnInstructionRequestType(
+      BeamFnApi.InstructionRequest value) {
+    try {
+      return handlers.getOrDefault(value.getRequestCase(), this::missingHandler)
+          .apply(value)
+          .setInstructionId(value.getInstructionId())
+          .build();
+    } catch (Exception e) {
+      return BeamFnApi.InstructionResponse.newBuilder()
+          .setInstructionId(value.getInstructionId())
+          .setError(getStackTraceAsString(e))
+          .build();
+    }
+  }
+
+  public void sendInstructionResponse(BeamFnApi.InstructionResponse value) {
+    outboundObserver.onNext(value);
+  }
+
+  private BeamFnApi.InstructionResponse.Builder missingHandler(
+      BeamFnApi.InstructionRequest request) {
+    return BeamFnApi.InstructionResponse.newBuilder().setError(
+        String.format("Unknown InstructionRequest type %s", request.getRequestCase()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java
new file mode 100644
index 0000000..05c2aab
--- /dev/null
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java
@@ -0,0 +1,334 @@
+/*
+ * 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.control;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.collect.Iterables.getOnlyElement;
+
+import com.google.common.collect.Collections2;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableMultimap;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Multimap;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.BytesValue;
+import com.google.protobuf.InvalidProtocolBufferException;
+import com.google.protobuf.Message;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.function.Supplier;
+import org.apache.beam.fn.harness.data.BeamFnDataClient;
+import org.apache.beam.fn.harness.fake.FakeAggregatorFactory;
+import org.apache.beam.fn.harness.fake.FakeStepContext;
+import org.apache.beam.fn.harness.fn.ThrowingConsumer;
+import org.apache.beam.fn.harness.fn.ThrowingRunnable;
+import org.apache.beam.fn.v1.BeamFnApi;
+import org.apache.beam.runners.core.BeamFnDataReadRunner;
+import org.apache.beam.runners.core.BeamFnDataWriteRunner;
+import org.apache.beam.runners.core.BoundedSourceRunner;
+import org.apache.beam.runners.core.DoFnRunner;
+import org.apache.beam.runners.core.DoFnRunners;
+import org.apache.beam.runners.core.DoFnRunners.OutputManager;
+import org.apache.beam.runners.dataflow.util.DoFnInfo;
+import org.apache.beam.sdk.io.BoundedSource;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.util.NullSideInputReader;
+import org.apache.beam.sdk.util.SerializableUtils;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.values.TupleTag;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Processes {@link org.apache.beam.fn.v1.BeamFnApi.ProcessBundleRequest}s by materializing
+ * the set of required runners for each {@link org.apache.beam.fn.v1.BeamFnApi.FunctionSpec},
+ * wiring them together based upon the {@code input} and {@code output} map definitions.
+ *
+ * <p>Finally executes the DAG based graph by starting all runners in reverse topological order,
+ * and finishing all runners in forward topological order.
+ */
+public class ProcessBundleHandler {
+  // TODO: What should the initial set of URNs be?
+  private static final String DATA_INPUT_URN = "urn:org.apache.beam:source:runner:0.1";
+  private static final String DATA_OUTPUT_URN = "urn:org.apache.beam:sink:runner:0.1";
+  private static final String JAVA_DO_FN_URN = "urn:org.apache.beam:dofn:java:0.1";
+  private static final String JAVA_SOURCE_URN = "urn:org.apache.beam:source:java:0.1";
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(ProcessBundleHandler.class);
+
+  private final PipelineOptions options;
+  private final Function<Long, Message> fnApiRegistry;
+  private final BeamFnDataClient beamFnDataClient;
+
+  public ProcessBundleHandler(
+      PipelineOptions options,
+      Function<Long, Message> fnApiRegistry,
+      BeamFnDataClient beamFnDataClient) {
+    this.options = options;
+    this.fnApiRegistry = fnApiRegistry;
+    this.beamFnDataClient = beamFnDataClient;
+  }
+
+  protected <InputT, OutputT> void createConsumersForPrimitiveTransform(
+      BeamFnApi.PrimitiveTransform primitiveTransform,
+      Supplier<Long> processBundleInstructionId,
+      Function<BeamFnApi.Target, Collection<ThrowingConsumer<WindowedValue<OutputT>>>> consumers,
+      BiConsumer<BeamFnApi.Target, ThrowingConsumer<WindowedValue<InputT>>> addConsumer,
+      Consumer<ThrowingRunnable> addStartFunction,
+      Consumer<ThrowingRunnable> addFinishFunction) throws IOException {
+
+    BeamFnApi.FunctionSpec functionSpec = primitiveTransform.getFunctionSpec();
+
+    // For every output PCollection, create a map from output name to Consumer
+    ImmutableMap.Builder<String, Collection<ThrowingConsumer<WindowedValue<OutputT>>>>
+        outputMapBuilder = ImmutableMap.builder();
+    for (Map.Entry<String, BeamFnApi.PCollection> entry :
+        primitiveTransform.getOutputsMap().entrySet()) {
+      outputMapBuilder.put(
+          entry.getKey(),
+          consumers.apply(
+              BeamFnApi.Target.newBuilder()
+                  .setPrimitiveTransformReference(primitiveTransform.getId())
+                  .setName(entry.getKey())
+                  .build()));
+    }
+    ImmutableMap<String, Collection<ThrowingConsumer<WindowedValue<OutputT>>>> outputMap =
+        outputMapBuilder.build();
+
+    // Based upon the function spec, populate the start/finish/consumer information.
+    ThrowingConsumer<WindowedValue<InputT>> consumer;
+    switch (functionSpec.getUrn()) {
+      default:
+        BeamFnApi.Target target;
+        BeamFnApi.Coder coderSpec;
+        throw new IllegalArgumentException(
+            String.format("Unknown FunctionSpec %s", functionSpec));
+
+      case DATA_OUTPUT_URN:
+        target = BeamFnApi.Target.newBuilder()
+            .setPrimitiveTransformReference(primitiveTransform.getId())
+            .setName(getOnlyElement(primitiveTransform.getOutputsMap().keySet()))
+            .build();
+        coderSpec = (BeamFnApi.Coder) fnApiRegistry.apply(
+            getOnlyElement(primitiveTransform.getOutputsMap().values()).getCoderReference());
+        BeamFnDataWriteRunner<InputT> remoteGrpcWriteRunner =
+            new BeamFnDataWriteRunner<>(
+                functionSpec,
+                processBundleInstructionId,
+                target,
+                coderSpec,
+                beamFnDataClient);
+        addStartFunction.accept(remoteGrpcWriteRunner::registerForOutput);
+        consumer = remoteGrpcWriteRunner::consume;
+        addFinishFunction.accept(remoteGrpcWriteRunner::close);
+        break;
+
+      case DATA_INPUT_URN:
+        target = BeamFnApi.Target.newBuilder()
+            .setPrimitiveTransformReference(primitiveTransform.getId())
+            .setName(getOnlyElement(primitiveTransform.getInputsMap().keySet()))
+            .build();
+        coderSpec = (BeamFnApi.Coder) fnApiRegistry.apply(
+            getOnlyElement(primitiveTransform.getOutputsMap().values()).getCoderReference());
+        BeamFnDataReadRunner<OutputT> remoteGrpcReadRunner =
+            new BeamFnDataReadRunner<>(
+                functionSpec,
+                processBundleInstructionId,
+                target,
+                coderSpec,
+                beamFnDataClient,
+                outputMap);
+        addStartFunction.accept(remoteGrpcReadRunner::registerInputLocation);
+        consumer = null;
+        addFinishFunction.accept(remoteGrpcReadRunner::blockTillReadFinishes);
+        break;
+
+      case JAVA_DO_FN_URN:
+        DoFnRunner<InputT, OutputT> doFnRunner = createDoFnRunner(functionSpec, outputMap);
+        addStartFunction.accept(doFnRunner::startBundle);
+        addFinishFunction.accept(doFnRunner::finishBundle);
+        consumer = doFnRunner::processElement;
+        break;
+
+      case JAVA_SOURCE_URN:
+        @SuppressWarnings({"unchecked", "rawtypes"})
+        BoundedSourceRunner<BoundedSource<OutputT>, OutputT> sourceRunner =
+            createBoundedSourceRunner(functionSpec, outputMap);
+        @SuppressWarnings({"unchecked", "rawtypes"})
+        ThrowingConsumer<WindowedValue<?>> sourceConsumer =
+            (ThrowingConsumer)
+                (ThrowingConsumer<WindowedValue<BoundedSource<OutputT>>>)
+                    sourceRunner::runReadLoop;
+        // TODO: Remove and replace with source being sent across gRPC port
+        addStartFunction.accept(sourceRunner::start);
+        consumer = (ThrowingConsumer) sourceConsumer;
+        break;
+    }
+
+    if (consumer != null) {
+      for (Map.Entry<String, BeamFnApi.Target.List> entry :
+          primitiveTransform.getInputsMap().entrySet()) {
+        for (BeamFnApi.Target target : entry.getValue().getTargetList()) {
+          addConsumer.accept(target, consumer);
+        }
+      }
+    }
+  }
+
+  public BeamFnApi.InstructionResponse.Builder processBundle(BeamFnApi.InstructionRequest request)
+      throws Exception {
+    BeamFnApi.InstructionResponse.Builder response =
+        BeamFnApi.InstructionResponse.newBuilder()
+            .setProcessBundle(BeamFnApi.ProcessBundleResponse.getDefaultInstance());
+
+    long bundleId = request.getProcessBundle().getProcessBundleDescriptorReference();
+    BeamFnApi.ProcessBundleDescriptor bundleDescriptor =
+        (BeamFnApi.ProcessBundleDescriptor) fnApiRegistry.apply(bundleId);
+
+    Multimap<BeamFnApi.Target,
+             ThrowingConsumer<WindowedValue<Object>>> outputTargetToConsumer =
+             HashMultimap.create();
+    List<ThrowingRunnable> startFunctions = new ArrayList<>();
+    List<ThrowingRunnable> finishFunctions = new ArrayList<>();
+    // We process the primitive transform list in reverse order
+    // because we assume that the runner provides it in topologically order.
+    // This means that all the start/finish functions will be in reverse topological order.
+    for (BeamFnApi.PrimitiveTransform primitiveTransform :
+        Lists.reverse(bundleDescriptor.getPrimitiveTransformList())) {
+      createConsumersForPrimitiveTransform(
+          primitiveTransform,
+          request::getInstructionId,
+          outputTargetToConsumer::get,
+          outputTargetToConsumer::put,
+          startFunctions::add,
+          finishFunctions::add);
+    }
+
+    // Already in reverse order so we don't need to do anything.
+    for (ThrowingRunnable startFunction : startFunctions) {
+      LOGGER.debug("Starting function {}", startFunction);
+      startFunction.run();
+    }
+
+    // Need to reverse this since we want to call finish in topological order.
+    for (ThrowingRunnable finishFunction : Lists.reverse(finishFunctions)) {
+      LOGGER.debug("Finishing function {}", finishFunction);
+      finishFunction.run();
+    }
+
+    return response;
+  }
+
+  /**
+   * Converts a {@link org.apache.beam.fn.v1.BeamFnApi.FunctionSpec} into a {@link DoFnRunner}.
+   */
+  private <InputT, OutputT> DoFnRunner<InputT, OutputT> createDoFnRunner(
+      BeamFnApi.FunctionSpec functionSpec,
+      Map<String, Collection<ThrowingConsumer<WindowedValue<OutputT>>>> outputMap) {
+    ByteString serializedFn;
+    try {
+      serializedFn = functionSpec.getData().unpack(BytesValue.class).getValue();
+    } catch (InvalidProtocolBufferException e) {
+      throw new IllegalArgumentException(
+          String.format("Unable to unwrap DoFn %s", functionSpec), e);
+    }
+    DoFnInfo<?, ?> doFnInfo =
+        (DoFnInfo<?, ?>)
+            SerializableUtils.deserializeFromByteArray(serializedFn.toByteArray(), "DoFnInfo");
+
+    checkArgument(
+        Objects.equals(
+            new HashSet<>(Collections2.transform(outputMap.keySet(), Long::parseLong)),
+            doFnInfo.getOutputMap().keySet()),
+        "Unexpected mismatch between transform output map %s and DoFnInfo output map %s.",
+        outputMap.keySet(),
+        doFnInfo.getOutputMap());
+
+    ImmutableMultimap.Builder<TupleTag<?>,
+                              ThrowingConsumer<WindowedValue<OutputT>>> tagToOutput =
+                              ImmutableMultimap.builder();
+    for (Map.Entry<Long, TupleTag<?>> entry : doFnInfo.getOutputMap().entrySet()) {
+      tagToOutput.putAll(entry.getValue(), outputMap.get(Long.toString(entry.getKey())));
+    }
+    @SuppressWarnings({"unchecked", "rawtypes"})
+    final Map<TupleTag<?>, Collection<ThrowingConsumer<WindowedValue<?>>>> tagBasedOutputMap =
+        (Map) tagToOutput.build().asMap();
+
+    OutputManager outputManager =
+        new OutputManager() {
+          Map<TupleTag<?>, Collection<ThrowingConsumer<WindowedValue<?>>>> tupleTagToOutput =
+              tagBasedOutputMap;
+
+          @Override
+          public <T> void output(TupleTag<T> tag, WindowedValue<T> output) {
+            try {
+              Collection<ThrowingConsumer<WindowedValue<?>>> consumers =
+                  tupleTagToOutput.get(tag);
+              if (consumers == null) {
+                // TODO: Should we handle undeclared outputs, if so how?
+                throw new UnsupportedOperationException(String.format(
+                    "Unable to output %s on unknown output %s", output, tag));
+              }
+              for (ThrowingConsumer<WindowedValue<?>> consumer : consumers) {
+                consumer.accept(output);
+              }
+            } catch (Throwable t) {
+              throw new RuntimeException(t);
+            }
+          }
+        };
+
+    @SuppressWarnings({"unchecked", "rawtypes", "deprecation"})
+    DoFnRunner<InputT, OutputT> runner =
+        DoFnRunners.simpleRunner(
+            PipelineOptionsFactory.create(), /* TODO */
+            (DoFn) doFnInfo.getDoFn(),
+            NullSideInputReader.empty(), /* TODO */
+            outputManager,
+            (TupleTag) doFnInfo.getOutputMap().get(doFnInfo.getMainOutput()),
+            new ArrayList<>(doFnInfo.getOutputMap().values()),
+            new FakeStepContext(),
+            new FakeAggregatorFactory(),
+            (WindowingStrategy) doFnInfo.getWindowingStrategy());
+    return runner;
+  }
+
+  private <InputT extends BoundedSource<OutputT>, OutputT>
+      BoundedSourceRunner<InputT, OutputT> createBoundedSourceRunner(
+          BeamFnApi.FunctionSpec functionSpec,
+          Map<String, Collection<ThrowingConsumer<WindowedValue<OutputT>>>> outputMap) {
+
+    @SuppressWarnings({"rawtypes", "unchecked"})
+    BoundedSourceRunner<InputT, OutputT> runner =
+        new BoundedSourceRunner(options, functionSpec, outputMap);
+    return runner;
+  }
+}


[2/6] beam git commit: A proposal for a portability framework to execute user definable functions.

Posted by lc...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/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
new file mode 100644
index 0000000..20566ea
--- /dev/null
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataGrpcClientTest.java
@@ -0,0 +1,309 @@
+/*
+ * 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.data;
+
+import static org.apache.beam.sdk.util.CoderUtils.encodeToByteArray;
+import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow;
+import static org.hamcrest.Matchers.contains;
+import static org.hamcrest.collection.IsEmptyCollection.empty;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+
+import com.google.protobuf.ByteString;
+import io.grpc.ManagedChannel;
+import io.grpc.Server;
+import io.grpc.inprocess.InProcessChannelBuilder;
+import io.grpc.inprocess.InProcessServerBuilder;
+import io.grpc.stub.CallStreamObserver;
+import io.grpc.stub.StreamObserver;
+import java.util.Collection;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentLinkedQueue;
+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.fn.harness.test.TestStreams;
+import org.apache.beam.fn.v1.BeamFnApi;
+import org.apache.beam.fn.v1.BeamFnDataGrpc;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.LengthPrefixCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.KV;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Tests for {@link BeamFnDataGrpcClient}. */
+@RunWith(JUnit4.class)
+public class BeamFnDataGrpcClientTest {
+  private static final Coder<WindowedValue<String>> CODER =
+      LengthPrefixCoder.of(
+          WindowedValue.getFullCoder(StringUtf8Coder.of(),
+              GlobalWindow.Coder.INSTANCE));
+  private static final KV<Long, BeamFnApi.Target> KEY_A = KV.of(
+      12L,
+      BeamFnApi.Target.newBuilder().setPrimitiveTransformReference(34L).setName("targetA").build());
+  private static final KV<Long, BeamFnApi.Target> KEY_B = KV.of(
+      56L,
+      BeamFnApi.Target.newBuilder().setPrimitiveTransformReference(78L).setName("targetB").build());
+
+  private static final BeamFnApi.Elements ELEMENTS_A_1;
+  private static final BeamFnApi.Elements ELEMENTS_A_2;
+  private static final BeamFnApi.Elements ELEMENTS_B_1;
+  static {
+    try {
+    ELEMENTS_A_1 = BeamFnApi.Elements.newBuilder()
+        .addData(BeamFnApi.Elements.Data.newBuilder()
+            .setInstructionReference(KEY_A.getKey())
+            .setTarget(KEY_A.getValue())
+            .setData(ByteString.copyFrom(encodeToByteArray(CODER, valueInGlobalWindow("ABC")))
+                .concat(ByteString.copyFrom(encodeToByteArray(CODER, valueInGlobalWindow("DEF"))))))
+        .build();
+    ELEMENTS_A_2 = BeamFnApi.Elements.newBuilder()
+        .addData(BeamFnApi.Elements.Data.newBuilder()
+            .setInstructionReference(KEY_A.getKey())
+            .setTarget(KEY_A.getValue())
+            .setData(ByteString.copyFrom(encodeToByteArray(CODER, valueInGlobalWindow("GHI")))))
+        .addData(BeamFnApi.Elements.Data.newBuilder()
+            .setInstructionReference(KEY_A.getKey())
+            .setTarget(KEY_A.getValue()))
+        .build();
+    ELEMENTS_B_1 = BeamFnApi.Elements.newBuilder()
+        .addData(BeamFnApi.Elements.Data.newBuilder()
+            .setInstructionReference(KEY_B.getKey())
+            .setTarget(KEY_B.getValue())
+            .setData(ByteString.copyFrom(encodeToByteArray(CODER, valueInGlobalWindow("JKL")))
+                .concat(ByteString.copyFrom(encodeToByteArray(CODER, valueInGlobalWindow("MNO"))))))
+        .addData(BeamFnApi.Elements.Data.newBuilder()
+            .setInstructionReference(KEY_B.getKey())
+            .setTarget(KEY_B.getValue()))
+        .build();
+    } catch (Exception e) {
+      throw new ExceptionInInitializerError(e);
+    }
+  }
+
+  @Test
+  public void testForInboundConsumer() throws Exception {
+    CountDownLatch waitForClientToConnect = new CountDownLatch(1);
+    Collection<WindowedValue<String>> inboundValuesA = new ConcurrentLinkedQueue<>();
+    Collection<WindowedValue<String>> inboundValuesB = new ConcurrentLinkedQueue<>();
+    Collection<BeamFnApi.Elements> inboundServerValues = new ConcurrentLinkedQueue<>();
+    AtomicReference<StreamObserver<BeamFnApi.Elements>> outboundServerObserver =
+        new AtomicReference<>();
+    CallStreamObserver<BeamFnApi.Elements> inboundServerObserver =
+        TestStreams.withOnNext(inboundServerValues::add).build();
+
+    BeamFnApi.ApiServiceDescriptor apiServiceDescriptor =
+        BeamFnApi.ApiServiceDescriptor.newBuilder()
+            .setUrl(this.getClass().getName() + "-" + UUID.randomUUID().toString())
+            .build();
+    Server server = InProcessServerBuilder.forName(apiServiceDescriptor.getUrl())
+            .addService(new BeamFnDataGrpc.BeamFnDataImplBase() {
+              @Override
+              public StreamObserver<BeamFnApi.Elements> data(
+                  StreamObserver<BeamFnApi.Elements> outboundObserver) {
+                outboundServerObserver.set(outboundObserver);
+                waitForClientToConnect.countDown();
+                return inboundServerObserver;
+              }
+            })
+            .build();
+    server.start();
+    try {
+      ManagedChannel channel =
+          InProcessChannelBuilder.forName(apiServiceDescriptor.getUrl()).build();
+
+    BeamFnDataGrpcClient clientFactory = new BeamFnDataGrpcClient(
+        PipelineOptionsFactory.create(),
+        (BeamFnApi.ApiServiceDescriptor descriptor) -> channel,
+        this::createStreamForTest);
+
+    CompletableFuture<Void> readFutureA = clientFactory.forInboundConsumer(
+        apiServiceDescriptor,
+        KEY_A,
+        CODER,
+        inboundValuesA::add);
+
+      waitForClientToConnect.await();
+      outboundServerObserver.get().onNext(ELEMENTS_A_1);
+      // Purposefully transmit some data before the consumer for B is bound showing that
+      // data is not lost
+      outboundServerObserver.get().onNext(ELEMENTS_B_1);
+      Thread.sleep(100);
+
+      CompletableFuture<Void> readFutureB = clientFactory.forInboundConsumer(
+          apiServiceDescriptor,
+          KEY_B,
+          CODER,
+          inboundValuesB::add);
+
+      // Show that out of order stream completion can occur.
+      readFutureB.get();
+      assertThat(inboundValuesB, contains(
+          valueInGlobalWindow("JKL"), valueInGlobalWindow("MNO")));
+
+      outboundServerObserver.get().onNext(ELEMENTS_A_2);
+      readFutureA.get();
+      assertThat(inboundValuesA, contains(
+          valueInGlobalWindow("ABC"), valueInGlobalWindow("DEF"), valueInGlobalWindow("GHI")));
+    } finally {
+      server.shutdownNow();
+    }
+  }
+
+  @Test
+  public void testForInboundConsumerThatThrows() throws Exception {
+    CountDownLatch waitForClientToConnect = new CountDownLatch(1);
+    AtomicInteger consumerInvoked = new AtomicInteger();
+    Collection<BeamFnApi.Elements> inboundServerValues = new ConcurrentLinkedQueue<>();
+    AtomicReference<StreamObserver<BeamFnApi.Elements>> outboundServerObserver =
+        new AtomicReference<>();
+    CallStreamObserver<BeamFnApi.Elements> inboundServerObserver =
+        TestStreams.withOnNext(inboundServerValues::add).build();
+
+    BeamFnApi.ApiServiceDescriptor apiServiceDescriptor =
+        BeamFnApi.ApiServiceDescriptor.newBuilder()
+            .setUrl(this.getClass().getName() + "-" + UUID.randomUUID().toString())
+            .build();
+    Server server = InProcessServerBuilder.forName(apiServiceDescriptor.getUrl())
+            .addService(new BeamFnDataGrpc.BeamFnDataImplBase() {
+              @Override
+              public StreamObserver<BeamFnApi.Elements> data(
+                  StreamObserver<BeamFnApi.Elements> outboundObserver) {
+                outboundServerObserver.set(outboundObserver);
+                waitForClientToConnect.countDown();
+                return inboundServerObserver;
+              }
+            })
+            .build();
+    server.start();
+    RuntimeException exceptionToThrow = new RuntimeException("TestFailure");
+    try {
+      ManagedChannel channel =
+          InProcessChannelBuilder.forName(apiServiceDescriptor.getUrl()).build();
+
+      BeamFnDataGrpcClient clientFactory = new BeamFnDataGrpcClient(
+          PipelineOptionsFactory.create(),
+          (BeamFnApi.ApiServiceDescriptor descriptor) -> channel,
+          this::createStreamForTest);
+
+      CompletableFuture<Void> readFuture = clientFactory.forInboundConsumer(
+          apiServiceDescriptor,
+          KEY_A,
+          CODER,
+          new ThrowingConsumer<WindowedValue<String>>() {
+            @Override
+            public void accept(WindowedValue<String> t) throws Exception {
+              consumerInvoked.incrementAndGet();
+              throw exceptionToThrow;
+            }
+          });
+
+      waitForClientToConnect.await();
+
+      // This first message should cause a failure afterwards all other messages are dropped.
+      outboundServerObserver.get().onNext(ELEMENTS_A_1);
+      outboundServerObserver.get().onNext(ELEMENTS_A_2);
+
+      try {
+        readFuture.get();
+        fail("Expected channel to fail");
+      } catch (ExecutionException e) {
+        assertEquals(exceptionToThrow, e.getCause());
+      }
+      // The server should not have received any values
+      assertThat(inboundServerValues, empty());
+      // The consumer should have only been invoked once
+      assertEquals(1, consumerInvoked.get());
+    } finally {
+      server.shutdownNow();
+    }
+  }
+
+  @Test
+  public void testForOutboundConsumer() throws Exception {
+    CountDownLatch waitForInboundServerValuesCompletion = new CountDownLatch(2);
+    Collection<BeamFnApi.Elements> inboundServerValues = new ConcurrentLinkedQueue<>();
+    CallStreamObserver<BeamFnApi.Elements> inboundServerObserver =
+        TestStreams.withOnNext(
+            new Consumer<BeamFnApi.Elements>() {
+              @Override
+              public void accept(BeamFnApi.Elements t) {
+                inboundServerValues.add(t);
+                waitForInboundServerValuesCompletion.countDown();
+              }
+            }
+            ).build();
+
+    BeamFnApi.ApiServiceDescriptor apiServiceDescriptor =
+        BeamFnApi.ApiServiceDescriptor.newBuilder()
+            .setUrl(this.getClass().getName() + "-" + UUID.randomUUID().toString())
+            .build();
+    Server server = InProcessServerBuilder.forName(apiServiceDescriptor.getUrl())
+            .addService(new BeamFnDataGrpc.BeamFnDataImplBase() {
+              @Override
+              public StreamObserver<BeamFnApi.Elements> data(
+                  StreamObserver<BeamFnApi.Elements> outboundObserver) {
+                return inboundServerObserver;
+              }
+            })
+            .build();
+    server.start();
+    try {
+      ManagedChannel channel =
+          InProcessChannelBuilder.forName(apiServiceDescriptor.getUrl()).build();
+
+      BeamFnDataGrpcClient clientFactory = new BeamFnDataGrpcClient(
+          PipelineOptionsFactory.fromArgs(
+              new String[]{ "--experiments=beam_fn_api_data_buffer_limit=20" }).create(),
+          (BeamFnApi.ApiServiceDescriptor descriptor) -> channel,
+          this::createStreamForTest);
+
+      try (CloseableThrowingConsumer<WindowedValue<String>> consumer =
+          clientFactory.forOutboundConsumer(apiServiceDescriptor, KEY_A, CODER)) {
+        consumer.accept(valueInGlobalWindow("ABC"));
+        consumer.accept(valueInGlobalWindow("DEF"));
+        consumer.accept(valueInGlobalWindow("GHI"));
+      }
+
+      waitForInboundServerValuesCompletion.await();
+
+      assertThat(inboundServerValues, contains(ELEMENTS_A_1, ELEMENTS_A_2));
+    } finally {
+      server.shutdownNow();
+    }
+  }
+
+  private <ReqT, RespT> StreamObserver<RespT> createStreamForTest(
+      Function<StreamObserver<ReqT>, StreamObserver<RespT>> clientFactory,
+      StreamObserver<ReqT> handler) {
+    return clientFactory.apply(handler);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/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
new file mode 100644
index 0000000..38d9e2c
--- /dev/null
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataGrpcMultiplexerTest.java
@@ -0,0 +1,96 @@
+/*
+ * 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.data;
+
+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 com.google.common.util.concurrent.Uninterruptibles;
+import com.google.protobuf.ByteString;
+import io.grpc.stub.StreamObserver;
+import java.util.ArrayList;
+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.fn.v1.BeamFnApi;
+import org.apache.beam.sdk.values.KV;
+import org.junit.Test;
+
+/** Tests for {@link BeamFnDataGrpcMultiplexer}. */
+public class BeamFnDataGrpcMultiplexerTest {
+  private static final BeamFnApi.ApiServiceDescriptor DESCRIPTOR =
+      BeamFnApi.ApiServiceDescriptor.newBuilder().setUrl("test").build();
+  private static final KV<Long, BeamFnApi.Target> OUTPUT_LOCATION = KV.of(777L,
+      BeamFnApi.Target.newBuilder()
+      .setName("name")
+      .setPrimitiveTransformReference(888L)
+      .build());
+  private static final BeamFnApi.Elements ELEMENTS = BeamFnApi.Elements.newBuilder()
+      .addData(BeamFnApi.Elements.Data.newBuilder()
+          .setInstructionReference(OUTPUT_LOCATION.getKey())
+          .setTarget(OUTPUT_LOCATION.getValue())
+          .setData(ByteString.copyFrom(new byte[1])))
+      .build();
+  private static final BeamFnApi.Elements TERMINAL_ELEMENTS = BeamFnApi.Elements.newBuilder()
+      .addData(BeamFnApi.Elements.Data.newBuilder()
+          .setInstructionReference(OUTPUT_LOCATION.getKey())
+          .setTarget(OUTPUT_LOCATION.getValue()))
+      .build();
+
+  @Test
+  public void testOutboundObserver() {
+    Collection<BeamFnApi.Elements> values = new ArrayList<>();
+    BeamFnDataGrpcMultiplexer multiplexer = new BeamFnDataGrpcMultiplexer(
+        DESCRIPTOR,
+        (StreamObserver<BeamFnApi.Elements> inboundObserver)
+            -> TestStreams.withOnNext(values::add).build());
+    multiplexer.getOutboundObserver().onNext(ELEMENTS);
+    assertThat(values, contains(ELEMENTS));
+  }
+
+  @Test
+  public void testInboundObserverBlocksTillConsumerConnects() throws Exception {
+    Collection<BeamFnApi.Elements> outboundValues = new ArrayList<>();
+    Collection<BeamFnApi.Elements.Data> inboundValues = new ArrayList<>();
+    BeamFnDataGrpcMultiplexer multiplexer = new BeamFnDataGrpcMultiplexer(
+        DESCRIPTOR,
+        (StreamObserver<BeamFnApi.Elements> inboundObserver)
+            -> TestStreams.withOnNext(outboundValues::add).build());
+    ExecutorService executorService = Executors.newCachedThreadPool();
+    executorService.submit(new Runnable() {
+      @Override
+      public void run() {
+        // Purposefully sleep to simulate a delay in a consumer connecting.
+        Uninterruptibles.sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
+        multiplexer.futureForKey(OUTPUT_LOCATION).complete(inboundValues::add);
+      }
+    });
+    multiplexer.getInboundObserver().onNext(ELEMENTS);
+    assertTrue(multiplexer.consumers.containsKey(OUTPUT_LOCATION));
+    // Ensure that when we see a terminal Elements object, we remove the consumer
+    multiplexer.getInboundObserver().onNext(TERMINAL_ELEMENTS);
+    assertFalse(multiplexer.consumers.containsKey(OUTPUT_LOCATION));
+
+    // Assert that normal and terminal Elements are passed to the consumer
+    assertThat(inboundValues, contains(ELEMENTS.getData(0), TERMINAL_ELEMENTS.getData(0)));
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataInboundObserverTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataInboundObserverTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataInboundObserverTest.java
new file mode 100644
index 0000000..ff0e083
--- /dev/null
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataInboundObserverTest.java
@@ -0,0 +1,116 @@
+/*
+ * 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.data;
+
+import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow;
+import static org.hamcrest.Matchers.contains;
+import static org.hamcrest.Matchers.empty;
+import static org.hamcrest.Matchers.instanceOf;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import com.google.protobuf.ByteString;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import org.apache.beam.fn.v1.BeamFnApi;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.Coder.Context;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Tests for {@link BeamFnDataInboundObserver}. */
+@RunWith(JUnit4.class)
+public class BeamFnDataInboundObserverTest {
+  private static final Coder<WindowedValue<String>> CODER =
+      WindowedValue.getFullCoder(StringUtf8Coder.of(), GlobalWindow.Coder.INSTANCE);
+
+  @Test
+  public void testDecodingElements() throws Exception {
+    Collection<WindowedValue<String>> values = new ArrayList<>();
+    CompletableFuture<Void> readFuture = new CompletableFuture<>();
+    BeamFnDataInboundObserver<String> observer = new BeamFnDataInboundObserver<>(
+        CODER,
+        values::add,
+        readFuture);
+
+    // Test decoding multiple messages
+    observer.accept(dataWith("ABC", "DEF", "GHI"));
+    assertThat(values, contains(
+        valueInGlobalWindow("ABC"), valueInGlobalWindow("DEF"), valueInGlobalWindow("GHI")));
+    values.clear();
+
+    // Test empty message signaling end of stream
+    assertFalse(readFuture.isDone());
+    observer.accept(dataWith());
+    assertTrue(readFuture.isDone());
+
+    // Test messages after stream is finished are discarded
+    observer.accept(dataWith("ABC", "DEF", "GHI"));
+    assertThat(values, empty());
+  }
+
+  @Test
+  public void testConsumptionFailureCompletesReadFutureAndDiscardsMessages() throws Exception {
+    CompletableFuture<Void> readFuture = new CompletableFuture<>();
+    BeamFnDataInboundObserver<String> observer = new BeamFnDataInboundObserver<>(
+        CODER,
+        this::throwOnDefValue,
+        readFuture);
+
+    assertFalse(readFuture.isDone());
+    observer.accept(dataWith("ABC", "DEF", "GHI"));
+    assertTrue(readFuture.isCompletedExceptionally());
+
+    try {
+      readFuture.get();
+      fail("Expected failure");
+    } catch (ExecutionException e) {
+      assertThat(e.getCause(), instanceOf(RuntimeException.class));
+      assertEquals("Failure", e.getCause().getMessage());
+    }
+  }
+
+  private void throwOnDefValue(WindowedValue<String> value) {
+    if ("DEF".equals(value.getValue())) {
+      throw new RuntimeException("Failure");
+    }
+  }
+
+  private BeamFnApi.Elements.Data dataWith(String ... values) throws Exception {
+    BeamFnApi.Elements.Data.Builder builder = BeamFnApi.Elements.Data.newBuilder()
+        .setInstructionReference(777L)
+        .setTarget(BeamFnApi.Target.newBuilder()
+            .setPrimitiveTransformReference(999L)
+            .setName("Test"));
+    ByteString.Output output = ByteString.newOutput();
+    for (String value : values) {
+      CODER.encode(valueInGlobalWindow(value), output, Context.NESTED);
+    }
+    builder.setData(output.toByteString());
+    return builder.build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/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
new file mode 100644
index 0000000..bb6a501
--- /dev/null
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/logging/BeamFnLoggingClientTest.java
@@ -0,0 +1,169 @@
+/*
+ * 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.logging;
+
+import static com.google.common.base.Throwables.getStackTraceAsString;
+import static org.hamcrest.Matchers.contains;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+import com.google.protobuf.Timestamp;
+import io.grpc.ManagedChannel;
+import io.grpc.Server;
+import io.grpc.inprocess.InProcessChannelBuilder;
+import io.grpc.inprocess.InProcessServerBuilder;
+import io.grpc.stub.CallStreamObserver;
+import io.grpc.stub.StreamObserver;
+import java.util.Collection;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Function;
+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.fn.v1.BeamFnApi;
+import org.apache.beam.fn.v1.BeamFnLoggingGrpc;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Tests for {@link BeamFnLoggingClient}. */
+@RunWith(JUnit4.class)
+public class BeamFnLoggingClientTest {
+
+  private static final LogRecord FILTERED_RECORD;
+  private static final LogRecord TEST_RECORD;
+  private static final LogRecord TEST_RECORD_WITH_EXCEPTION;
+
+  static {
+    FILTERED_RECORD = new LogRecord(Level.SEVERE, "FilteredMessage");
+
+    TEST_RECORD = new LogRecord(Level.FINE, "Message");
+    TEST_RECORD.setLoggerName("LoggerName");
+    TEST_RECORD.setMillis(1234567890L);
+    TEST_RECORD.setThreadID(12345);
+
+    TEST_RECORD_WITH_EXCEPTION = new LogRecord(Level.WARNING, "MessageWithException");
+    TEST_RECORD_WITH_EXCEPTION.setLoggerName("LoggerName");
+    TEST_RECORD_WITH_EXCEPTION.setMillis(1234567890L);
+    TEST_RECORD_WITH_EXCEPTION.setThreadID(12345);
+    TEST_RECORD_WITH_EXCEPTION.setThrown(new RuntimeException("ExceptionMessage"));
+  }
+
+  private static final BeamFnApi.LogEntry TEST_ENTRY =
+      BeamFnApi.LogEntry.newBuilder()
+          .setSeverity(BeamFnApi.LogEntry.Severity.DEBUG)
+          .setMessage("Message")
+          .setThread("12345")
+          .setTimestamp(Timestamp.newBuilder().setSeconds(1234567).setNanos(890000000).build())
+          .setLogLocation("LoggerName")
+          .build();
+  private static final BeamFnApi.LogEntry TEST_ENTRY_WITH_EXCEPTION =
+      BeamFnApi.LogEntry.newBuilder()
+          .setSeverity(BeamFnApi.LogEntry.Severity.WARN)
+          .setMessage("MessageWithException")
+          .setTrace(getStackTraceAsString(TEST_RECORD_WITH_EXCEPTION.getThrown()))
+          .setThread("12345")
+          .setTimestamp(Timestamp.newBuilder().setSeconds(1234567).setNanos(890000000).build())
+          .setLogLocation("LoggerName")
+          .build();
+
+  @Test
+  public void testLogging() throws Exception {
+    AtomicBoolean clientClosedStream = new AtomicBoolean();
+    Collection<BeamFnApi.LogEntry> values = new ConcurrentLinkedQueue<>();
+    AtomicReference<StreamObserver<BeamFnApi.LogControl>> outboundServerObserver =
+        new AtomicReference<>();
+    CallStreamObserver<BeamFnApi.LogEntry.List> inboundServerObserver =
+        TestStreams.withOnNext(
+            (BeamFnApi.LogEntry.List logEntries) -> values.addAll(logEntries.getLogEntriesList()))
+        .withOnCompleted(new Runnable() {
+          @Override
+          public void run() {
+            // Remember that the client told us that this stream completed
+            clientClosedStream.set(true);
+            outboundServerObserver.get().onCompleted();
+          }
+        }).build();
+
+    BeamFnApi.ApiServiceDescriptor apiServiceDescriptor =
+        BeamFnApi.ApiServiceDescriptor.newBuilder()
+            .setUrl(this.getClass().getName() + "-" + UUID.randomUUID().toString())
+            .build();
+    Server server = InProcessServerBuilder.forName(apiServiceDescriptor.getUrl())
+            .addService(new BeamFnLoggingGrpc.BeamFnLoggingImplBase() {
+              @Override
+              public StreamObserver<BeamFnApi.LogEntry.List> logging(
+                  StreamObserver<BeamFnApi.LogControl> outboundObserver) {
+                outboundServerObserver.set(outboundObserver);
+                return inboundServerObserver;
+              }
+            })
+            .build();
+    server.start();
+    try {
+      ManagedChannel channel =
+          InProcessChannelBuilder.forName(apiServiceDescriptor.getUrl()).build();
+
+      BeamFnLoggingClient client = new BeamFnLoggingClient(
+          PipelineOptionsFactory.fromArgs(new String[] {
+              "--defaultWorkerLogLevel=OFF",
+              "--workerLogLevelOverrides={\"ConfiguredLogger\": \"DEBUG\"}"
+          }).create(),
+          apiServiceDescriptor,
+          (BeamFnApi.ApiServiceDescriptor descriptor) -> channel,
+          this::createStreamForTest);
+
+      // Ensure that log levels were correctly set.
+      assertEquals(Level.OFF,
+          LogManager.getLogManager().getLogger("").getLevel());
+      assertEquals(Level.FINE,
+          LogManager.getLogManager().getLogger("ConfiguredLogger").getLevel());
+
+      // Should be filtered because the default log level override is OFF
+      LogManager.getLogManager().getLogger("").log(FILTERED_RECORD);
+      // Should not be filtered because the default log level override for ConfiguredLogger is DEBUG
+      LogManager.getLogManager().getLogger("ConfiguredLogger").log(TEST_RECORD);
+      LogManager.getLogManager().getLogger("ConfiguredLogger").log(TEST_RECORD_WITH_EXCEPTION);
+      client.close();
+
+      // Verify that after close, log levels are reset.
+      assertEquals(Level.INFO, LogManager.getLogManager().getLogger("").getLevel());
+      assertNull(LogManager.getLogManager().getLogger("ConfiguredLogger").getLevel());
+
+      assertTrue(clientClosedStream.get());
+      assertTrue(channel.isShutdown());
+      assertThat(values, contains(TEST_ENTRY, TEST_ENTRY_WITH_EXCEPTION));
+    } finally {
+      server.shutdownNow();
+    }
+  }
+
+  private <ReqT, RespT> StreamObserver<RespT> createStreamForTest(
+      Function<StreamObserver<ReqT>, StreamObserver<RespT>> clientFactory,
+      StreamObserver<ReqT> handler) {
+    return clientFactory.apply(handler);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/stream/AdvancingPhaserTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/stream/AdvancingPhaserTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/stream/AdvancingPhaserTest.java
new file mode 100644
index 0000000..3dd1b42
--- /dev/null
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/stream/AdvancingPhaserTest.java
@@ -0,0 +1,48 @@
+/*
+ * 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.stream;
+
+import static org.hamcrest.collection.IsEmptyCollection.empty;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThat;
+
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Tests for {@link AdvancingPhaser}. */
+@RunWith(JUnit4.class)
+public class AdvancingPhaserTest {
+  @Test
+  public void testAdvancement() throws Exception {
+    AdvancingPhaser phaser = new AdvancingPhaser(1);
+    int currentPhase = phaser.getPhase();
+    ExecutorService service = Executors.newSingleThreadExecutor();
+    service.submit(phaser::arrive);
+    phaser.awaitAdvance(currentPhase);
+    assertFalse(phaser.isTerminated());
+    service.shutdown();
+    if (!service.awaitTermination(10, TimeUnit.SECONDS)) {
+      assertThat(service.shutdownNow(), empty());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/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
new file mode 100644
index 0000000..76b7ef0
--- /dev/null
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/stream/BufferingStreamObserverTest.java
@@ -0,0 +1,146 @@
+/*
+ * 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.stream;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.util.concurrent.Uninterruptibles;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.Callable;
+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.fn.harness.test.TestExecutors;
+import org.apache.beam.fn.harness.test.TestExecutors.TestExecutorService;
+import org.apache.beam.fn.harness.test.TestStreams;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Tests for {@link BufferingStreamObserver}. */
+@RunWith(JUnit4.class)
+public class BufferingStreamObserverTest {
+  @Rule public TestExecutorService executor = TestExecutors.from(Executors::newCachedThreadPool);
+
+  @Test
+  public void testThreadSafety() throws Exception {
+    List<String> onNextValues = new ArrayList<>();
+    AdvancingPhaser phaser = new AdvancingPhaser(1);
+    final AtomicBoolean isCriticalSectionShared = new AtomicBoolean();
+    final BufferingStreamObserver<String> streamObserver =
+        new BufferingStreamObserver<>(
+            phaser,
+            TestStreams.withOnNext(
+                new Consumer<String>() {
+                  @Override
+                  public void accept(String t) {
+                    // Use the atomic boolean to detect if multiple threads are in this
+                    // critical section. Any thread that enters purposefully blocks by sleeping
+                    // to increase the contention between threads artificially.
+                    assertFalse(isCriticalSectionShared.getAndSet(true));
+                    Uninterruptibles.sleepUninterruptibly(50, TimeUnit.MILLISECONDS);
+                    onNextValues.add(t);
+                    assertTrue(isCriticalSectionShared.getAndSet(false));
+                  }
+                }).build(),
+            executor,
+            3);
+
+    List<String> prefixes = ImmutableList.of("0", "1", "2", "3", "4");
+    List<Callable<String>> tasks = new ArrayList<>();
+    for (String prefix : prefixes) {
+      tasks.add(
+          new Callable<String>() {
+            @Override
+            public String call() throws Exception {
+              for (int i = 0; i < 10; i++) {
+                streamObserver.onNext(prefix + i);
+              }
+              return prefix;
+            }
+          });
+    }
+    List<Future<String>> results = executor.invokeAll(tasks);
+    for (Future<String> result : results) {
+      result.get();
+    }
+    streamObserver.onCompleted();
+
+    // Check that order was maintained.
+    int[] prefixesIndex = new int[prefixes.size()];
+    assertEquals(50, onNextValues.size());
+    for (String onNextValue : onNextValues) {
+      int prefix = Integer.parseInt(onNextValue.substring(0, 1));
+      int suffix = Integer.parseInt(onNextValue.substring(1, 2));
+      assertEquals(prefixesIndex[prefix], suffix);
+      prefixesIndex[prefix] += 1;
+    }
+  }
+
+  @Test
+  public void testIsReadyIsHonored() throws Exception {
+    AdvancingPhaser phaser = new AdvancingPhaser(1);
+    final AtomicBoolean elementsAllowed = new AtomicBoolean();
+    final BufferingStreamObserver<String> streamObserver =
+        new BufferingStreamObserver<>(
+            phaser,
+            TestStreams.withOnNext(
+                new Consumer<String>() {
+                  @Override
+                  public void accept(String t) {
+                    assertTrue(elementsAllowed.get());
+                  }
+                }).withIsReady(elementsAllowed::get).build(),
+            executor,
+            3);
+
+    // Start all the tasks
+    List<Future<String>> results = new ArrayList<>();
+    for (String prefix : ImmutableList.of("0", "1", "2", "3", "4")) {
+      results.add(
+          executor.submit(
+              new Callable<String>() {
+                @Override
+                public String call() throws Exception {
+                  for (int i = 0; i < 10; i++) {
+                    streamObserver.onNext(prefix + i);
+                  }
+                  return prefix;
+                }
+              }));
+    }
+
+    // Have them wait and then flip that we do allow elements and wake up those awaiting
+    Uninterruptibles.sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
+    elementsAllowed.set(true);
+    phaser.arrive();
+
+    for (Future<String> result : results) {
+      result.get();
+    }
+    streamObserver.onCompleted();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/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
new file mode 100644
index 0000000..b5d3ec1
--- /dev/null
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/stream/DirectStreamObserverTest.java
@@ -0,0 +1,139 @@
+/*
+ * 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.stream;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.util.concurrent.Uninterruptibles;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.Callable;
+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.fn.harness.test.TestExecutors;
+import org.apache.beam.fn.harness.test.TestExecutors.TestExecutorService;
+import org.apache.beam.fn.harness.test.TestStreams;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Tests for {@link DirectStreamObserver}. */
+@RunWith(JUnit4.class)
+public class DirectStreamObserverTest {
+  @Rule public TestExecutorService executor = TestExecutors.from(Executors::newCachedThreadPool);
+
+  @Test
+  public void testThreadSafety() throws Exception {
+    List<String> onNextValues = new ArrayList<>();
+    AdvancingPhaser phaser = new AdvancingPhaser(1);
+    final AtomicBoolean isCriticalSectionShared = new AtomicBoolean();
+    final DirectStreamObserver<String> streamObserver =
+        new DirectStreamObserver<>(
+            phaser,
+            TestStreams.withOnNext(
+                new Consumer<String>() {
+                  @Override
+                  public void accept(String t) {
+                    // Use the atomic boolean to detect if multiple threads are in this
+                    // critical section. Any thread that enters purposefully blocks by sleeping
+                    // to increase the contention between threads artificially.
+                    assertFalse(isCriticalSectionShared.getAndSet(true));
+                    Uninterruptibles.sleepUninterruptibly(50, TimeUnit.MILLISECONDS);
+                    onNextValues.add(t);
+                    assertTrue(isCriticalSectionShared.getAndSet(false));
+                  }
+                }).build());
+
+    List<String> prefixes = ImmutableList.of("0", "1", "2", "3", "4");
+    List<Callable<String>> tasks = new ArrayList<>();
+    for (String prefix : prefixes) {
+      tasks.add(
+          new Callable<String>() {
+            @Override
+            public String call() throws Exception {
+              for (int i = 0; i < 10; i++) {
+                streamObserver.onNext(prefix + i);
+              }
+              return prefix;
+            }
+          });
+    }
+    executor.invokeAll(tasks);
+    streamObserver.onCompleted();
+
+    // Check that order was maintained.
+    int[] prefixesIndex = new int[prefixes.size()];
+    assertEquals(50, onNextValues.size());
+    for (String onNextValue : onNextValues) {
+      int prefix = Integer.parseInt(onNextValue.substring(0, 1));
+      int suffix = Integer.parseInt(onNextValue.substring(1, 2));
+      assertEquals(prefixesIndex[prefix], suffix);
+      prefixesIndex[prefix] += 1;
+    }
+  }
+
+  @Test
+  public void testIsReadyIsHonored() throws Exception {
+    AdvancingPhaser phaser = new AdvancingPhaser(1);
+    final AtomicBoolean elementsAllowed = new AtomicBoolean();
+    final DirectStreamObserver<String> streamObserver =
+        new DirectStreamObserver<>(
+            phaser,
+            TestStreams.withOnNext(
+                new Consumer<String>() {
+                  @Override
+                  public void accept(String t) {
+                    assertTrue(elementsAllowed.get());
+                  }
+                }).withIsReady(elementsAllowed::get).build());
+
+    // Start all the tasks
+    List<Future<String>> results = new ArrayList<>();
+    for (String prefix : ImmutableList.of("0", "1", "2", "3", "4")) {
+      results.add(
+          executor.submit(
+              new Callable<String>() {
+                @Override
+                public String call() throws Exception {
+                  for (int i = 0; i < 10; i++) {
+                    streamObserver.onNext(prefix + i);
+                  }
+                  return prefix;
+                }
+              }));
+    }
+
+    // Have them wait and then flip that we do allow elements and wake up those awaiting
+    Uninterruptibles.sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
+    elementsAllowed.set(true);
+    phaser.arrive();
+
+    for (Future<String> result : results) {
+      result.get();
+    }
+    streamObserver.onCompleted();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/stream/ForwardingClientResponseObserverTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/stream/ForwardingClientResponseObserverTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/stream/ForwardingClientResponseObserverTest.java
new file mode 100644
index 0000000..598d7f3
--- /dev/null
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/stream/ForwardingClientResponseObserverTest.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.fn.harness.stream;
+
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoMoreInteractions;
+
+import io.grpc.stub.ClientCallStreamObserver;
+import io.grpc.stub.ClientResponseObserver;
+import io.grpc.stub.StreamObserver;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+import org.mockito.Mockito;
+
+/** Tests for {@link ForwardingClientResponseObserver}. */
+@RunWith(JUnit4.class)
+public class ForwardingClientResponseObserverTest {
+  @Test
+  public void testCallsAreForwardedAndOnReadyHandlerBound() {
+    @SuppressWarnings("unchecked")
+    StreamObserver<Object> delegateObserver = Mockito.mock(StreamObserver.class);
+    @SuppressWarnings("unchecked")
+    ClientCallStreamObserver<Object> callStreamObserver =
+        Mockito.mock(ClientCallStreamObserver.class);
+    Runnable onReadyHandler = new Runnable() {
+      @Override
+      public void run() {
+      }
+    };
+    ClientResponseObserver<Object, Object> observer =
+        new ForwardingClientResponseObserver<>(delegateObserver, onReadyHandler);
+    observer.onNext("A");
+    verify(delegateObserver).onNext("A");
+    Throwable t = new RuntimeException();
+    observer.onError(t);
+    verify(delegateObserver).onError(t);
+    observer.onCompleted();
+    verify(delegateObserver).onCompleted();
+    observer.beforeStart(callStreamObserver);
+    verify(callStreamObserver).setOnReadyHandler(onReadyHandler);
+    verifyNoMoreInteractions(delegateObserver, callStreamObserver);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/stream/StreamObserverFactoryTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/stream/StreamObserverFactoryTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/stream/StreamObserverFactoryTest.java
new file mode 100644
index 0000000..9331079
--- /dev/null
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/stream/StreamObserverFactoryTest.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.fn.harness.stream;
+
+import static org.hamcrest.Matchers.instanceOf;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+
+import io.grpc.stub.CallStreamObserver;
+import io.grpc.stub.StreamObserver;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+import org.mockito.Mock;
+import org.mockito.MockitoAnnotations;
+
+/** Tests for {@link StreamObserverFactory}. */
+@RunWith(JUnit4.class)
+public class StreamObserverFactoryTest {
+  @Mock private StreamObserver<Integer> mockRequestObserver;
+  @Mock private CallStreamObserver<String> mockResponseObserver;
+
+  @Before
+  public void setUp() {
+    MockitoAnnotations.initMocks(this);
+  }
+
+  @Test
+  public void testDefaultInstantiation() {
+    StreamObserver<String> observer =
+        StreamObserverFactory.fromOptions(PipelineOptionsFactory.create())
+            .from(this::fakeFactory, mockRequestObserver);
+    assertThat(observer, instanceOf(DirectStreamObserver.class));
+  }
+
+  @Test
+  public void testBufferedStreamInstantiation() {
+    StreamObserver<String> observer =
+        StreamObserverFactory.fromOptions(
+                PipelineOptionsFactory.fromArgs(
+                        new String[] {"--experiments=beam_fn_api_buffered_stream"})
+                    .create())
+            .from(this::fakeFactory, mockRequestObserver);
+    assertThat(observer, instanceOf(BufferingStreamObserver.class));
+  }
+
+  @Test
+  public void testBufferedStreamWithLimitInstantiation() {
+    StreamObserver<String> observer =
+        StreamObserverFactory.fromOptions(
+                PipelineOptionsFactory.fromArgs(
+                        new String[] {
+                          "--experiments=beam_fn_api_buffered_stream,"
+                          + "beam_fn_api_buffered_stream_buffer_size=1"
+                        })
+                    .create())
+            .from(this::fakeFactory, mockRequestObserver);
+    assertThat(observer, instanceOf(BufferingStreamObserver.class));
+    assertEquals(1, ((BufferingStreamObserver<String>) observer).getBufferSize());
+  }
+
+  private StreamObserver<String> fakeFactory(StreamObserver<Integer> observer) {
+    assertThat(observer, instanceOf(ForwardingClientResponseObserver.class));
+    return mockResponseObserver;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/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
new file mode 100644
index 0000000..f846466
--- /dev/null
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/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.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;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/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
new file mode 100644
index 0000000..85c64d0
--- /dev/null
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/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.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/0b4b2bec/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
new file mode 100644
index 0000000..f398286
--- /dev/null
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/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.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/0b4b2bec/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
new file mode 100644
index 0000000..b684c90
--- /dev/null
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/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.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/0b4b2bec/sdks/java/harness/src/test/java/org/apache/beam/runners/core/BeamFnDataReadRunnerTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/test/java/org/apache/beam/runners/core/BeamFnDataReadRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/runners/core/BeamFnDataReadRunnerTest.java
new file mode 100644
index 0000000..511cc3f
--- /dev/null
+++ b/sdks/java/harness/src/test/java/org/apache/beam/runners/core/BeamFnDataReadRunnerTest.java
@@ -0,0 +1,187 @@
+/*
+ * 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.core;
+
+import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow;
+import static org.hamcrest.Matchers.contains;
+import static org.junit.Assert.assertThat;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoMoreInteractions;
+import static org.mockito.Mockito.when;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.util.concurrent.Uninterruptibles;
+import com.google.protobuf.Any;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.BytesValue;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.beam.fn.harness.data.BeamFnDataClient;
+import org.apache.beam.fn.harness.fn.ThrowingConsumer;
+import org.apache.beam.fn.harness.test.TestExecutors;
+import org.apache.beam.fn.harness.test.TestExecutors.TestExecutorService;
+import org.apache.beam.fn.v1.BeamFnApi;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.KV;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Captor;
+import org.mockito.Mock;
+import org.mockito.MockitoAnnotations;
+
+/** Tests for {@link BeamFnDataReadRunner}. */
+@RunWith(JUnit4.class)
+public class BeamFnDataReadRunnerTest {
+  private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+  private static final BeamFnApi.RemoteGrpcPort PORT_SPEC = BeamFnApi.RemoteGrpcPort.newBuilder()
+      .setApiServiceDescriptor(BeamFnApi.ApiServiceDescriptor.getDefaultInstance()).build();
+  private static final BeamFnApi.FunctionSpec FUNCTION_SPEC = BeamFnApi.FunctionSpec.newBuilder()
+      .setData(Any.pack(PORT_SPEC)).build();
+  private static final Coder<WindowedValue<String>> CODER =
+      WindowedValue.getFullCoder(StringUtf8Coder.of(), GlobalWindow.Coder.INSTANCE);
+  private static final BeamFnApi.Coder CODER_SPEC;
+  static {
+    try {
+      CODER_SPEC = BeamFnApi.Coder.newBuilder().setFunctionSpec(BeamFnApi.FunctionSpec.newBuilder()
+          .setData(Any.pack(BytesValue.newBuilder().setValue(ByteString.copyFrom(
+              OBJECT_MAPPER.writeValueAsBytes(CODER.asCloudObject()))).build())))
+          .build();
+    } catch (IOException e) {
+      throw new ExceptionInInitializerError(e);
+    }
+  }
+  private static final BeamFnApi.Target INPUT_TARGET = BeamFnApi.Target.newBuilder()
+      .setPrimitiveTransformReference(1)
+      .setName("out")
+      .build();
+
+  @Rule public TestExecutorService executor = TestExecutors.from(Executors::newCachedThreadPool);
+  @Mock private BeamFnDataClient mockBeamFnDataClientFactory;
+  @Captor private ArgumentCaptor<ThrowingConsumer<WindowedValue<String>>> consumerCaptor;
+
+  @Before
+  public void setUp() {
+    MockitoAnnotations.initMocks(this);
+  }
+
+  @Test
+  public void testReuseForMultipleBundles() throws Exception {
+    CompletableFuture<Void> bundle1Future = new CompletableFuture<>();
+    CompletableFuture<Void> bundle2Future = new CompletableFuture<>();
+    when(mockBeamFnDataClientFactory.forInboundConsumer(
+        any(),
+        any(),
+        any(),
+        any())).thenReturn(bundle1Future).thenReturn(bundle2Future);
+    List<WindowedValue<String>> valuesA = new ArrayList<>();
+    List<WindowedValue<String>> valuesB = new ArrayList<>();
+    Map<String, Collection<ThrowingConsumer<WindowedValue<String>>>> outputMap = ImmutableMap.of(
+        "outA", ImmutableList.of(valuesA::add),
+        "outB", ImmutableList.of(valuesB::add));
+    AtomicLong bundleId = new AtomicLong(0);
+    BeamFnDataReadRunner<String> readRunner = new BeamFnDataReadRunner<>(
+        FUNCTION_SPEC,
+        bundleId::get,
+        INPUT_TARGET,
+        CODER_SPEC,
+        mockBeamFnDataClientFactory,
+        outputMap);
+
+    // Process for bundle id 0
+    readRunner.registerInputLocation();
+
+    verify(mockBeamFnDataClientFactory).forInboundConsumer(
+        eq(PORT_SPEC.getApiServiceDescriptor()),
+        eq(KV.of(bundleId.get(), INPUT_TARGET)),
+        eq(CODER),
+        consumerCaptor.capture());
+
+    executor.submit(new Runnable() {
+      @Override
+      public void run() {
+        // Sleep for some small amount of time simulating the parent blocking
+        Uninterruptibles.sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
+        try {
+          consumerCaptor.getValue().accept(valueInGlobalWindow("ABC"));
+          consumerCaptor.getValue().accept(valueInGlobalWindow("DEF"));
+        } catch (Exception e) {
+          bundle1Future.completeExceptionally(e);
+        } finally {
+          bundle1Future.complete(null);
+        }
+      }
+    });
+
+    readRunner.blockTillReadFinishes();
+    assertThat(valuesA, contains(valueInGlobalWindow("ABC"), valueInGlobalWindow("DEF")));
+    assertThat(valuesB, contains(valueInGlobalWindow("ABC"), valueInGlobalWindow("DEF")));
+
+    // Process for bundle id 1
+    bundleId.incrementAndGet();
+    valuesA.clear();
+    valuesB.clear();
+    readRunner.registerInputLocation();
+
+    verify(mockBeamFnDataClientFactory).forInboundConsumer(
+        eq(PORT_SPEC.getApiServiceDescriptor()),
+        eq(KV.of(bundleId.get(), INPUT_TARGET)),
+        eq(CODER),
+        consumerCaptor.capture());
+
+    executor.submit(new Runnable() {
+      @Override
+      public void run() {
+        // Sleep for some small amount of time simulating the parent blocking
+        Uninterruptibles.sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
+        try {
+          consumerCaptor.getValue().accept(valueInGlobalWindow("GHI"));
+          consumerCaptor.getValue().accept(valueInGlobalWindow("JKL"));
+        } catch (Exception e) {
+          bundle2Future.completeExceptionally(e);
+        } finally {
+          bundle2Future.complete(null);
+        }
+      }
+    });
+
+    readRunner.blockTillReadFinishes();
+    assertThat(valuesA, contains(valueInGlobalWindow("GHI"), valueInGlobalWindow("JKL")));
+    assertThat(valuesB, contains(valueInGlobalWindow("GHI"), valueInGlobalWindow("JKL")));
+
+    verifyNoMoreInteractions(mockBeamFnDataClientFactory);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/sdks/java/harness/src/test/java/org/apache/beam/runners/core/BeamFnDataWriteRunnerTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/test/java/org/apache/beam/runners/core/BeamFnDataWriteRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/runners/core/BeamFnDataWriteRunnerTest.java
new file mode 100644
index 0000000..ed67b14
--- /dev/null
+++ b/sdks/java/harness/src/test/java/org/apache/beam/runners/core/BeamFnDataWriteRunnerTest.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.runners.core;
+
+import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow;
+import static org.hamcrest.Matchers.contains;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoMoreInteractions;
+import static org.mockito.Mockito.when;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.protobuf.Any;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.BytesValue;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.beam.fn.harness.data.BeamFnDataClient;
+import org.apache.beam.fn.harness.fn.CloseableThrowingConsumer;
+import org.apache.beam.fn.v1.BeamFnApi;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.KV;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+import org.mockito.Matchers;
+import org.mockito.Mock;
+import org.mockito.MockitoAnnotations;
+
+/** Tests for {@link BeamFnDataWriteRunner}. */
+@RunWith(JUnit4.class)
+public class BeamFnDataWriteRunnerTest {
+  private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+  private static final BeamFnApi.RemoteGrpcPort PORT_SPEC = BeamFnApi.RemoteGrpcPort.newBuilder()
+      .setApiServiceDescriptor(BeamFnApi.ApiServiceDescriptor.getDefaultInstance()).build();
+  private static final BeamFnApi.FunctionSpec FUNCTION_SPEC = BeamFnApi.FunctionSpec.newBuilder()
+      .setData(Any.pack(PORT_SPEC)).build();
+  private static final Coder<WindowedValue<String>> CODER =
+      WindowedValue.getFullCoder(StringUtf8Coder.of(), GlobalWindow.Coder.INSTANCE);
+  private static final BeamFnApi.Coder CODER_SPEC;
+  static {
+    try {
+      CODER_SPEC = BeamFnApi.Coder.newBuilder().setFunctionSpec(BeamFnApi.FunctionSpec.newBuilder()
+      .setData(Any.pack(BytesValue.newBuilder().setValue(ByteString.copyFrom(
+          OBJECT_MAPPER.writeValueAsBytes(CODER.asCloudObject()))).build())))
+      .build();
+    } catch (IOException e) {
+      throw new ExceptionInInitializerError(e);
+    }
+  }
+  private static final BeamFnApi.Target OUTPUT_TARGET = BeamFnApi.Target.newBuilder()
+      .setPrimitiveTransformReference(1)
+      .setName("out")
+      .build();
+
+  @Mock private BeamFnDataClient mockBeamFnDataClientFactory;
+
+  @Before
+  public void setUp() {
+    MockitoAnnotations.initMocks(this);
+  }
+
+  @Test
+  public void testReuseForMultipleBundles() throws Exception {
+    RecordingConsumer<WindowedValue<String>> valuesA = new RecordingConsumer<>();
+    RecordingConsumer<WindowedValue<String>> valuesB = new RecordingConsumer<>();
+    when(mockBeamFnDataClientFactory.forOutboundConsumer(
+        any(),
+        any(),
+        Matchers.<Coder<WindowedValue<String>>>any())).thenReturn(valuesA).thenReturn(valuesB);
+    AtomicLong bundleId = new AtomicLong(0);
+    BeamFnDataWriteRunner<String> writeRunner = new BeamFnDataWriteRunner<>(
+        FUNCTION_SPEC,
+        bundleId::get,
+        OUTPUT_TARGET,
+        CODER_SPEC,
+        mockBeamFnDataClientFactory);
+
+    // Process for bundle id 0
+    writeRunner.registerForOutput();
+
+    verify(mockBeamFnDataClientFactory).forOutboundConsumer(
+        eq(PORT_SPEC.getApiServiceDescriptor()),
+        eq(KV.of(bundleId.get(), OUTPUT_TARGET)),
+        eq(CODER));
+
+    writeRunner.consume(valueInGlobalWindow("ABC"));
+    writeRunner.consume(valueInGlobalWindow("DEF"));
+    writeRunner.close();
+
+    assertTrue(valuesA.closed);
+    assertThat(valuesA, contains(valueInGlobalWindow("ABC"), valueInGlobalWindow("DEF")));
+
+    // Process for bundle id 1
+    bundleId.incrementAndGet();
+    valuesA.clear();
+    valuesB.clear();
+    writeRunner.registerForOutput();
+
+    verify(mockBeamFnDataClientFactory).forOutboundConsumer(
+        eq(PORT_SPEC.getApiServiceDescriptor()),
+        eq(KV.of(bundleId.get(), OUTPUT_TARGET)),
+        eq(CODER));
+
+    writeRunner.consume(valueInGlobalWindow("GHI"));
+    writeRunner.consume(valueInGlobalWindow("JKL"));
+    writeRunner.close();
+
+    assertTrue(valuesB.closed);
+    assertThat(valuesB, contains(valueInGlobalWindow("GHI"), valueInGlobalWindow("JKL")));
+    verifyNoMoreInteractions(mockBeamFnDataClientFactory);
+  }
+
+  private static class RecordingConsumer<T> extends ArrayList<T>
+      implements CloseableThrowingConsumer<T> {
+    private boolean closed;
+    @Override
+    public void close() throws Exception {
+      closed = true;
+    }
+
+    @Override
+    public void accept(T t) throws Exception {
+      if (closed) {
+        throw new IllegalStateException("Consumer is closed but attempting to consume " + t);
+      }
+      add(t);
+    }
+
+  }
+}


[6/6] beam git commit: [BEAM-1347, BEAM-1348] Beam Fn API Basic Java Harness and Proto Model

Posted by lc...@apache.org.
[BEAM-1347, BEAM-1348] Beam Fn API Basic Java Harness and Proto Model

This closes #1801


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

Branch: refs/heads/master
Commit: 343176c008832f4d3776a6e591d36be84dfb022a
Parents: 582c4a8 0b4b2be
Author: Luke Cwik <lc...@google.com>
Authored: Mon Jan 30 12:48:35 2017 -0800
Committer: Luke Cwik <lc...@google.com>
Committed: Mon Jan 30 12:48:35 2017 -0800

----------------------------------------------------------------------
 pom.xml                                         |  36 +-
 runners/apex/pom.xml                            |   2 +-
 sdks/common/fn-api/pom.xml                      | 111 +++
 .../fn-api/src/main/proto/beam_fn_api.proto     | 771 +++++++++++++++++++
 sdks/common/pom.xml                             |  38 +
 .../src/main/resources/beam/findbugs-filter.xml |  32 +-
 sdks/java/harness/pom.xml                       | 167 ++++
 .../org/apache/beam/fn/harness/FnHarness.java   | 131 ++++
 .../harness/channel/ManagedChannelFactory.java  |  80 ++
 .../harness/channel/SocketAddressFactory.java   |  64 ++
 .../beam/fn/harness/channel/package-info.java   |  22 +
 .../fn/harness/control/BeamFnControlClient.java | 165 ++++
 .../harness/control/ProcessBundleHandler.java   | 334 ++++++++
 .../fn/harness/control/RegisterHandler.java     |  92 +++
 .../beam/fn/harness/control/package-info.java   |  22 +
 .../BeamFnDataBufferingOutboundObserver.java    | 135 ++++
 .../beam/fn/harness/data/BeamFnDataClient.java  |  64 ++
 .../fn/harness/data/BeamFnDataGrpcClient.java   | 122 +++
 .../harness/data/BeamFnDataGrpcMultiplexer.java | 140 ++++
 .../harness/data/BeamFnDataInboundObserver.java |  81 ++
 .../beam/fn/harness/data/package-info.java      |  22 +
 .../fn/harness/fake/FakeAggregatorFactory.java  |  52 ++
 .../beam/fn/harness/fake/FakeStepContext.java   |  70 ++
 .../beam/fn/harness/fake/package-info.java      |  22 +
 .../harness/fn/CloseableThrowingConsumer.java   |  23 +
 .../beam/fn/harness/fn/ThrowingBiFunction.java  |  32 +
 .../beam/fn/harness/fn/ThrowingConsumer.java    |  32 +
 .../beam/fn/harness/fn/ThrowingFunction.java    |  32 +
 .../beam/fn/harness/fn/ThrowingRunnable.java    |  30 +
 .../apache/beam/fn/harness/fn/package-info.java |  22 +
 .../fn/harness/logging/BeamFnLoggingClient.java | 308 ++++++++
 .../beam/fn/harness/logging/package-info.java   |  22 +
 .../apache/beam/fn/harness/package-info.java    |  22 +
 .../beam/fn/harness/stream/AdvancingPhaser.java |  36 +
 .../harness/stream/BufferingStreamObserver.java | 166 ++++
 .../fn/harness/stream/DirectStreamObserver.java |  71 ++
 .../ForwardingClientResponseObserver.java       |  63 ++
 .../harness/stream/StreamObserverFactory.java   |  91 +++
 .../beam/fn/harness/stream/package-info.java    |  22 +
 .../beam/runners/core/BeamFnDataReadRunner.java | 104 +++
 .../runners/core/BeamFnDataWriteRunner.java     |  87 +++
 .../beam/runners/core/BoundedSourceRunner.java  | 105 +++
 .../apache/beam/runners/core/package-info.java  |  22 +
 .../apache/beam/fn/harness/FnHarnessTest.java   | 130 ++++
 .../channel/ManagedChannelFactoryTest.java      |  74 ++
 .../channel/SocketAddressFactoryTest.java       |  56 ++
 .../control/BeamFnControlClientTest.java        | 182 +++++
 .../control/ProcessBundleHandlerTest.java       | 674 ++++++++++++++++
 .../fn/harness/control/RegisterHandlerTest.java |  80 ++
 ...BeamFnDataBufferingOutboundObserverTest.java | 142 ++++
 .../harness/data/BeamFnDataGrpcClientTest.java  | 309 ++++++++
 .../data/BeamFnDataGrpcMultiplexerTest.java     |  96 +++
 .../data/BeamFnDataInboundObserverTest.java     | 116 +++
 .../logging/BeamFnLoggingClientTest.java        | 169 ++++
 .../fn/harness/stream/AdvancingPhaserTest.java  |  48 ++
 .../stream/BufferingStreamObserverTest.java     | 146 ++++
 .../stream/DirectStreamObserverTest.java        | 139 ++++
 .../ForwardingClientResponseObserverTest.java   |  60 ++
 .../stream/StreamObserverFactoryTest.java       |  84 ++
 .../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 ++
 .../runners/core/BeamFnDataReadRunnerTest.java  | 187 +++++
 .../runners/core/BeamFnDataWriteRunnerTest.java | 155 ++++
 .../runners/core/BoundedSourceRunnerTest.java   | 113 +++
 sdks/java/pom.xml                               |   1 +
 sdks/pom.xml                                    |   1 +
 68 files changed, 7514 insertions(+), 4 deletions(-)
----------------------------------------------------------------------



[3/6] beam git commit: A proposal for a portability framework to execute user definable functions.

Posted by lc...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/sdks/java/harness/src/main/java/org/apache/beam/runners/core/BeamFnDataReadRunner.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/runners/core/BeamFnDataReadRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/runners/core/BeamFnDataReadRunner.java
new file mode 100644
index 0000000..92042d0
--- /dev/null
+++ b/sdks/java/harness/src/main/java/org/apache/beam/runners/core/BeamFnDataReadRunner.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.core;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.collect.FluentIterable;
+import com.google.common.collect.ImmutableList;
+import com.google.protobuf.BytesValue;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Supplier;
+import org.apache.beam.fn.harness.data.BeamFnDataClient;
+import org.apache.beam.fn.harness.fn.ThrowingConsumer;
+import org.apache.beam.fn.v1.BeamFnApi;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.util.Serializer;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.KV;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Registers as a consumer for data over the Beam Fn API. Multiplexes any received data
+ * to all consumers in the specified output map.
+ *
+ * <p>Can be re-used serially across {@link org.apache.beam.fn.v1.BeamFnApi.ProcessBundleRequest}s.
+ * For each request, call {@link #registerInputLocation()} to start and call
+ * {@link #blockTillReadFinishes()} to finish.
+ */
+public class BeamFnDataReadRunner<OutputT> {
+  private static final Logger LOGGER = LoggerFactory.getLogger(BeamFnDataReadRunner.class);
+  private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+
+  private final BeamFnApi.ApiServiceDescriptor apiServiceDescriptor;
+  private final Collection<ThrowingConsumer<WindowedValue<OutputT>>> consumers;
+  private final Supplier<Long> processBundleInstructionIdSupplier;
+  private final BeamFnDataClient beamFnDataClientFactory;
+  private final Coder<WindowedValue<OutputT>> coder;
+  private final BeamFnApi.Target inputTarget;
+
+  private CompletableFuture<Void> readFuture;
+
+  public BeamFnDataReadRunner(
+      BeamFnApi.FunctionSpec functionSpec,
+      Supplier<Long> processBundleInstructionIdSupplier,
+      BeamFnApi.Target inputTarget,
+      BeamFnApi.Coder coderSpec,
+      BeamFnDataClient beamFnDataClientFactory,
+      Map<String, Collection<ThrowingConsumer<WindowedValue<OutputT>>>> outputMap)
+          throws IOException {
+    this.apiServiceDescriptor = functionSpec.getData().unpack(BeamFnApi.RemoteGrpcPort.class)
+        .getApiServiceDescriptor();
+    this.inputTarget = inputTarget;
+    this.processBundleInstructionIdSupplier = processBundleInstructionIdSupplier;
+    this.beamFnDataClientFactory = beamFnDataClientFactory;
+    this.consumers = ImmutableList.copyOf(FluentIterable.concat(outputMap.values()));
+
+    @SuppressWarnings("unchecked")
+    Coder<WindowedValue<OutputT>> coder = Serializer.deserialize(
+        OBJECT_MAPPER.readValue(
+            coderSpec.getFunctionSpec().getData().unpack(BytesValue.class).getValue().newInput(),
+            Map.class),
+        Coder.class);
+    this.coder = coder;
+  }
+
+  public void registerInputLocation() {
+    this.readFuture = beamFnDataClientFactory.forInboundConsumer(
+        apiServiceDescriptor,
+        KV.of(processBundleInstructionIdSupplier.get(), inputTarget),
+        coder,
+        this::multiplexToConsumers);
+  }
+
+  public void blockTillReadFinishes() throws Exception {
+    LOGGER.debug("Waiting for process bundle instruction {} and target {} to close.",
+        processBundleInstructionIdSupplier.get(), inputTarget);
+    readFuture.get();
+  }
+
+  private void multiplexToConsumers(WindowedValue<OutputT> value) throws Exception {
+    for (ThrowingConsumer<WindowedValue<OutputT>> consumer : consumers) {
+      consumer.accept(value);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/sdks/java/harness/src/main/java/org/apache/beam/runners/core/BeamFnDataWriteRunner.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/runners/core/BeamFnDataWriteRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/runners/core/BeamFnDataWriteRunner.java
new file mode 100644
index 0000000..596afe5
--- /dev/null
+++ b/sdks/java/harness/src/main/java/org/apache/beam/runners/core/BeamFnDataWriteRunner.java
@@ -0,0 +1,87 @@
+/*
+ * 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.core;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.protobuf.BytesValue;
+import java.io.IOException;
+import java.util.Map;
+import java.util.function.Supplier;
+import org.apache.beam.fn.harness.data.BeamFnDataClient;
+import org.apache.beam.fn.harness.fn.CloseableThrowingConsumer;
+import org.apache.beam.fn.v1.BeamFnApi;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.util.Serializer;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.KV;
+
+/**
+ * Registers as a consumer with the Beam Fn Data API. Propagates and elements consumed to
+ * the the registered consumer.
+ *
+ * <p>Can be re-used serially across {@link org.apache.beam.fn.v1.BeamFnApi.ProcessBundleRequest}s.
+ * For each request, call {@link #registerForOutput()} to start and call {@link #close()} to finish.
+ */
+public class BeamFnDataWriteRunner<InputT> {
+  private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+  private final BeamFnApi.ApiServiceDescriptor apiServiceDescriptor;
+  private final BeamFnApi.Target outputTarget;
+  private final Coder<WindowedValue<InputT>> coder;
+  private final BeamFnDataClient beamFnDataClientFactory;
+  private final Supplier<Long> processBundleInstructionIdSupplier;
+
+  private CloseableThrowingConsumer<WindowedValue<InputT>> consumer;
+
+  public BeamFnDataWriteRunner(
+      BeamFnApi.FunctionSpec functionSpec,
+      Supplier<Long> processBundleInstructionIdSupplier,
+      BeamFnApi.Target outputTarget,
+      BeamFnApi.Coder coderSpec,
+      BeamFnDataClient beamFnDataClientFactory)
+          throws IOException {
+    this.apiServiceDescriptor = functionSpec.getData().unpack(BeamFnApi.RemoteGrpcPort.class)
+        .getApiServiceDescriptor();
+    this.beamFnDataClientFactory = beamFnDataClientFactory;
+    this.processBundleInstructionIdSupplier = processBundleInstructionIdSupplier;
+    this.outputTarget = outputTarget;
+
+    @SuppressWarnings("unchecked")
+    Coder<WindowedValue<InputT>> coder = Serializer.deserialize(
+        OBJECT_MAPPER.readValue(
+            coderSpec.getFunctionSpec().getData().unpack(BytesValue.class).getValue().newInput(),
+            Map.class),
+        Coder.class);
+    this.coder = coder;
+  }
+
+  public void registerForOutput() {
+    consumer = beamFnDataClientFactory.forOutboundConsumer(
+        apiServiceDescriptor,
+        KV.of(processBundleInstructionIdSupplier.get(), outputTarget),
+        coder);
+  }
+
+  public void close() throws Exception {
+    consumer.close();
+  }
+
+  public void consume(WindowedValue<InputT> value) throws Exception {
+    consumer.accept(value);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/sdks/java/harness/src/main/java/org/apache/beam/runners/core/BoundedSourceRunner.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/runners/core/BoundedSourceRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/runners/core/BoundedSourceRunner.java
new file mode 100644
index 0000000..9d9c433
--- /dev/null
+++ b/sdks/java/harness/src/main/java/org/apache/beam/runners/core/BoundedSourceRunner.java
@@ -0,0 +1,105 @@
+/*
+ * 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.core;
+
+import com.google.common.collect.FluentIterable;
+import com.google.common.collect.ImmutableList;
+import com.google.protobuf.BytesValue;
+import com.google.protobuf.InvalidProtocolBufferException;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Map;
+import org.apache.beam.fn.harness.fn.ThrowingConsumer;
+import org.apache.beam.fn.v1.BeamFnApi;
+import org.apache.beam.sdk.io.BoundedSource;
+import org.apache.beam.sdk.io.Source.Reader;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.util.SerializableUtils;
+import org.apache.beam.sdk.util.WindowedValue;
+
+/**
+ * A runner which creates {@link Reader}s for each {@link BoundedSource} and executes
+ * the {@link Reader}s read loop.
+ */
+public class BoundedSourceRunner<InputT extends BoundedSource<OutputT>, OutputT> {
+  private final PipelineOptions pipelineOptions;
+  private final BeamFnApi.FunctionSpec definition;
+  private final Collection<ThrowingConsumer<WindowedValue<OutputT>>> consumers;
+
+  public BoundedSourceRunner(
+      PipelineOptions pipelineOptions,
+      BeamFnApi.FunctionSpec definition,
+      Map<String, Collection<ThrowingConsumer<WindowedValue<OutputT>>>> outputMap) {
+    this.pipelineOptions = pipelineOptions;
+    this.definition = definition;
+    this.consumers = ImmutableList.copyOf(FluentIterable.concat(outputMap.values()));
+  }
+
+  /**
+   * The runner harness is meant to send the source over the Beam Fn Data API which would be
+   * consumed by the {@link #runReadLoop}. Drop this method once the runner harness sends the
+   * source instead of unpacking it from the data block of the function specification.
+   */
+  @Deprecated
+  public void start() throws Exception {
+    try {
+      // The representation here is defined as the java serialized representation of the
+      // bounded source object packed into a protobuf Any using a protobuf BytesValue wrapper.
+      byte[] bytes = definition.getData().unpack(BytesValue.class).getValue().toByteArray();
+      @SuppressWarnings("unchecked")
+      InputT boundedSource =
+          (InputT) SerializableUtils.deserializeFromByteArray(bytes, definition.toString());
+      runReadLoop(WindowedValue.valueInGlobalWindow(boundedSource));
+    } catch (InvalidProtocolBufferException e) {
+      throw new IOException(
+          String.format("Failed to decode %s, expected %s",
+              definition.getData().getTypeUrl(), BytesValue.getDescriptor().getFullName()),
+          e);
+    }
+  }
+
+  /**
+   * Creates a {@link Reader} for each {@link BoundedSource} and executes the {@link Reader}s
+   * read loop. See {@link Reader} for further details of the read loop.
+   *
+   * <p>Propagates any exceptions caused during reading or processing via a consumer to the
+   * caller.
+   */
+  public void runReadLoop(WindowedValue<InputT> value) throws Exception {
+    try (Reader<OutputT> reader = value.getValue().createReader(pipelineOptions)) {
+      if (!reader.start()) {
+        // Reader has no data, immediately return
+        return;
+      }
+      do {
+        // TODO: Should this use the input window as the window for all the outputs?
+        WindowedValue<OutputT> nextValue = WindowedValue.timestampedValueInGlobalWindow(
+            reader.getCurrent(), reader.getCurrentTimestamp());
+        for (ThrowingConsumer<WindowedValue<OutputT>> consumer : consumers) {
+          consumer.accept(nextValue);
+        }
+      } while (reader.advance());
+    }
+  }
+
+  @Override
+  public String toString() {
+    return definition.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/sdks/java/harness/src/main/java/org/apache/beam/runners/core/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/runners/core/package-info.java b/sdks/java/harness/src/main/java/org/apache/beam/runners/core/package-info.java
new file mode 100644
index 0000000..d250a6a
--- /dev/null
+++ b/sdks/java/harness/src/main/java/org/apache/beam/runners/core/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.
+ */
+
+/**
+ * Provides utilities for Beam runner authors.
+ */
+package org.apache.beam.runners.core;

http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/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
new file mode 100644
index 0000000..ff05225
--- /dev/null
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnHarnessTest.java
@@ -0,0 +1,130 @@
+/*
+ * 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;
+
+import static org.hamcrest.Matchers.contains;
+import static org.junit.Assert.assertThat;
+
+import com.google.common.util.concurrent.Uninterruptibles;
+import io.grpc.Server;
+import io.grpc.ServerBuilder;
+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.fn.harness.test.TestStreams;
+import org.apache.beam.fn.v1.BeamFnApi;
+import org.apache.beam.fn.v1.BeamFnApi.InstructionRequest;
+import org.apache.beam.fn.v1.BeamFnApi.InstructionResponse;
+import org.apache.beam.fn.v1.BeamFnApi.LogControl;
+import org.apache.beam.fn.v1.BeamFnControlGrpc;
+import org.apache.beam.fn.v1.BeamFnLoggingGrpc;
+import org.apache.beam.sdk.options.GcsOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Tests for {@link FnHarness}. */
+@RunWith(JUnit4.class)
+public class FnHarnessTest {
+  private static final BeamFnApi.InstructionRequest INSTRUCTION_REQUEST =
+      BeamFnApi.InstructionRequest.newBuilder()
+      .setInstructionId(999L)
+      .setRegister(BeamFnApi.RegisterRequest.getDefaultInstance())
+      .build();
+  private static final BeamFnApi.InstructionResponse INSTRUCTION_RESPONSE =
+      BeamFnApi.InstructionResponse.newBuilder()
+      .setInstructionId(999L)
+      .setRegister(BeamFnApi.RegisterResponse.getDefaultInstance())
+      .build();
+
+  @Test
+  public void testLaunchFnHarnessAndTeardownCleanly() throws Exception {
+    PipelineOptions options = PipelineOptionsFactory.create();
+
+    List<BeamFnApi.LogEntry> logEntries = new ArrayList<>();
+    List<BeamFnApi.InstructionResponse> instructionResponses = new ArrayList<>();
+
+    BeamFnLoggingGrpc.BeamFnLoggingImplBase loggingService =
+        new BeamFnLoggingGrpc.BeamFnLoggingImplBase() {
+      @Override
+      public StreamObserver<BeamFnApi.LogEntry.List> logging(
+          StreamObserver<LogControl> responseObserver) {
+        return TestStreams.withOnNext(
+            (BeamFnApi.LogEntry.List entries) -> logEntries.addAll(entries.getLogEntriesList()))
+            .withOnCompleted(() -> responseObserver.onCompleted())
+            .build();
+      }
+    };
+
+    BeamFnControlGrpc.BeamFnControlImplBase controlService =
+        new BeamFnControlGrpc.BeamFnControlImplBase() {
+      @Override
+      public StreamObserver<InstructionResponse> control(
+          StreamObserver<InstructionRequest> responseObserver) {
+        CountDownLatch waitForResponses = new CountDownLatch(1 /* number of responses expected */);
+        options.as(GcsOptions.class).getExecutorService().submit(new Runnable() {
+          @Override
+          public void run() {
+            responseObserver.onNext(INSTRUCTION_REQUEST);
+            Uninterruptibles.awaitUninterruptibly(waitForResponses);
+            responseObserver.onCompleted();
+          }
+        });
+        return TestStreams.withOnNext(new Consumer<BeamFnApi.InstructionResponse>() {
+          @Override
+          public void accept(InstructionResponse t) {
+            instructionResponses.add(t);
+            waitForResponses.countDown();
+          }
+        }).withOnCompleted(waitForResponses::countDown).build();
+      }
+    };
+
+    Server loggingServer = ServerBuilder.forPort(0).addService(loggingService).build();
+    loggingServer.start();
+    try {
+      Server controlServer = ServerBuilder.forPort(0).addService(controlService).build();
+      controlServer.start();
+      try {
+        BeamFnApi.ApiServiceDescriptor loggingDescriptor = BeamFnApi.ApiServiceDescriptor
+            .newBuilder()
+            .setId(1L)
+            .setUrl("localhost:" + loggingServer.getPort())
+            .build();
+        BeamFnApi.ApiServiceDescriptor controlDescriptor = BeamFnApi.ApiServiceDescriptor
+            .newBuilder()
+            .setId(2L)
+            .setUrl("localhost:" + controlServer.getPort())
+            .build();
+
+        FnHarness.main(options, loggingDescriptor, controlDescriptor);
+        assertThat(instructionResponses, contains(INSTRUCTION_RESPONSE));
+      } finally {
+        controlServer.shutdownNow();
+      }
+    } finally {
+      loggingServer.shutdownNow();
+    }
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/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
new file mode 100644
index 0000000..9f634c9
--- /dev/null
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/channel/ManagedChannelFactoryTest.java
@@ -0,0 +1,74 @@
+/*
+ * 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.fn.v1.BeamFnApi.ApiServiceDescriptor;
+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() {
+    ApiServiceDescriptor apiServiceDescriptor = 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());
+    ApiServiceDescriptor apiServiceDescriptor = 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());
+    ApiServiceDescriptor apiServiceDescriptor = 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/0b4b2bec/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
new file mode 100644
index 0000000..610a8ea
--- /dev/null
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/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.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/0b4b2bec/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
new file mode 100644
index 0000000..fc3af49
--- /dev/null
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/BeamFnControlClientTest.java
@@ -0,0 +1,182 @@
+/*
+ * 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.control;
+
+import static com.google.common.base.Throwables.getStackTraceAsString;
+import static org.junit.Assert.assertEquals;
+
+import com.google.common.util.concurrent.Uninterruptibles;
+import io.grpc.ManagedChannel;
+import io.grpc.Server;
+import io.grpc.inprocess.InProcessChannelBuilder;
+import io.grpc.inprocess.InProcessServerBuilder;
+import io.grpc.stub.CallStreamObserver;
+import io.grpc.stub.StreamObserver;
+import java.util.EnumMap;
+import java.util.UUID;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+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.fn.v1.BeamFnApi;
+import org.apache.beam.fn.v1.BeamFnControlGrpc;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Tests for {@link BeamFnControlClient}. */
+@RunWith(JUnit4.class)
+public class BeamFnControlClientTest {
+  private static final BeamFnApi.InstructionRequest SUCCESSFUL_REQUEST =
+      BeamFnApi.InstructionRequest.newBuilder()
+      .setInstructionId(1L)
+      .setProcessBundle(BeamFnApi.ProcessBundleRequest.getDefaultInstance())
+      .build();
+  private static final BeamFnApi.InstructionResponse SUCCESSFUL_RESPONSE =
+      BeamFnApi.InstructionResponse.newBuilder()
+      .setInstructionId(1L)
+      .setProcessBundle(BeamFnApi.ProcessBundleResponse.getDefaultInstance())
+      .build();
+  private static final BeamFnApi.InstructionRequest UNKNOWN_HANDLER_REQUEST =
+      BeamFnApi.InstructionRequest.newBuilder()
+      .setInstructionId(2L)
+      .build();
+  private static final BeamFnApi.InstructionResponse UNKNOWN_HANDLER_RESPONSE =
+      BeamFnApi.InstructionResponse.newBuilder()
+      .setInstructionId(2L)
+      .setError("Unknown InstructionRequest type "
+          + BeamFnApi.InstructionRequest.RequestCase.REQUEST_NOT_SET)
+      .build();
+  private static final RuntimeException FAILURE = new RuntimeException("TestFailure");
+  private static final BeamFnApi.InstructionRequest FAILURE_REQUEST =
+      BeamFnApi.InstructionRequest.newBuilder()
+      .setInstructionId(3L)
+      .setRegister(BeamFnApi.RegisterRequest.getDefaultInstance())
+      .build();
+  private static final BeamFnApi.InstructionResponse FAILURE_RESPONSE =
+      BeamFnApi.InstructionResponse.newBuilder()
+      .setInstructionId(3L)
+      .setError(getStackTraceAsString(FAILURE))
+      .build();
+
+  @Test
+  public void testDelegation() throws Exception {
+    AtomicBoolean clientClosedStream = new AtomicBoolean();
+    BlockingQueue<BeamFnApi.InstructionResponse> values = new LinkedBlockingQueue<>();
+    BlockingQueue<StreamObserver<BeamFnApi.InstructionRequest>> outboundServerObservers =
+        new LinkedBlockingQueue<>();
+    CallStreamObserver<BeamFnApi.InstructionResponse> inboundServerObserver =
+        TestStreams.withOnNext(values::add)
+        .withOnCompleted(() -> clientClosedStream.set(true)).build();
+
+    BeamFnApi.ApiServiceDescriptor apiServiceDescriptor =
+        BeamFnApi.ApiServiceDescriptor.newBuilder()
+            .setUrl(this.getClass().getName() + "-" + UUID.randomUUID().toString())
+            .build();
+    Server server = InProcessServerBuilder.forName(apiServiceDescriptor.getUrl())
+        .addService(new BeamFnControlGrpc.BeamFnControlImplBase() {
+          @Override
+          public StreamObserver<BeamFnApi.InstructionResponse> control(
+              StreamObserver<BeamFnApi.InstructionRequest> outboundObserver) {
+            Uninterruptibles.putUninterruptibly(outboundServerObservers, outboundObserver);
+            return inboundServerObserver;
+          }
+        })
+        .build();
+    server.start();
+    try {
+      ManagedChannel channel =
+          InProcessChannelBuilder.forName(apiServiceDescriptor.getUrl()).build();
+
+      EnumMap<BeamFnApi.InstructionRequest.RequestCase,
+      ThrowingFunction<BeamFnApi.InstructionRequest,
+                       BeamFnApi.InstructionResponse.Builder>> handlers =
+                       new EnumMap<>(BeamFnApi.InstructionRequest.RequestCase.class);
+      handlers.put(BeamFnApi.InstructionRequest.RequestCase.PROCESS_BUNDLE,
+        new ThrowingFunction<BeamFnApi.InstructionRequest,
+                             BeamFnApi.InstructionResponse.Builder>() {
+          @Override
+          public BeamFnApi.InstructionResponse.Builder apply(BeamFnApi.InstructionRequest value)
+              throws Exception {
+            return BeamFnApi.InstructionResponse.newBuilder()
+                .setProcessBundle(BeamFnApi.ProcessBundleResponse.getDefaultInstance());
+          }
+        });
+      handlers.put(BeamFnApi.InstructionRequest.RequestCase.REGISTER,
+          new ThrowingFunction<BeamFnApi.InstructionRequest,
+                               BeamFnApi.InstructionResponse.Builder>() {
+            @Override
+            public BeamFnApi.InstructionResponse.Builder apply(BeamFnApi.InstructionRequest value)
+                throws Exception {
+              throw FAILURE;
+            }
+          });
+
+      BeamFnControlClient client = new BeamFnControlClient(
+                apiServiceDescriptor,
+                (BeamFnApi.ApiServiceDescriptor descriptor) -> channel,
+                this::createStreamForTest,
+                handlers);
+
+      // Get the connected client and attempt to send and receive an instruction
+      StreamObserver<BeamFnApi.InstructionRequest> outboundServerObserver =
+          outboundServerObservers.take();
+
+      ExecutorService executor = Executors.newCachedThreadPool();
+      Future<Void> future = executor.submit(new Callable<Void>() {
+        @Override
+        public Void call() throws Exception {
+          client.processInstructionRequests(executor);
+          return null;
+        }
+      });
+
+      outboundServerObserver.onNext(SUCCESSFUL_REQUEST);
+      assertEquals(SUCCESSFUL_RESPONSE, values.take());
+
+      // Ensure that conversion of an unknown request type is properly converted to a
+      // failure response.
+      outboundServerObserver.onNext(UNKNOWN_HANDLER_REQUEST);
+      assertEquals(UNKNOWN_HANDLER_RESPONSE, values.take());
+
+      // Ensure that all exceptions are caught and translated to failures
+      outboundServerObserver.onNext(FAILURE_REQUEST);
+      assertEquals(FAILURE_RESPONSE, values.take());
+
+      // Ensure that the server completing the stream translates to the completable future
+      // being completed allowing for a successful shutdown of the client.
+      outboundServerObserver.onCompleted();
+      future.get();
+    } finally {
+      server.shutdownNow();
+    }
+  }
+
+  private <ReqT, RespT> StreamObserver<RespT> createStreamForTest(
+      Function<StreamObserver<ReqT>, StreamObserver<RespT>> clientFactory,
+      StreamObserver<ReqT> handler) {
+    return clientFactory.apply(handler);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/ProcessBundleHandlerTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/ProcessBundleHandlerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/ProcessBundleHandlerTest.java
new file mode 100644
index 0000000..1d451b5
--- /dev/null
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/ProcessBundleHandlerTest.java
@@ -0,0 +1,674 @@
+/*
+ * 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.control;
+
+import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow;
+import static org.hamcrest.Matchers.contains;
+import static org.hamcrest.Matchers.empty;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoMoreInteractions;
+import static org.mockito.Mockito.verifyZeroInteractions;
+import static org.mockito.Mockito.when;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Suppliers;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableMultimap;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Multimap;
+import com.google.protobuf.Any;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.BytesValue;
+import com.google.protobuf.Message;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.function.Supplier;
+import org.apache.beam.fn.harness.data.BeamFnDataClient;
+import org.apache.beam.fn.harness.fn.CloseableThrowingConsumer;
+import org.apache.beam.fn.harness.fn.ThrowingConsumer;
+import org.apache.beam.fn.harness.fn.ThrowingRunnable;
+import org.apache.beam.fn.v1.BeamFnApi;
+import org.apache.beam.runners.dataflow.util.DoFnInfo;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.VarLongCoder;
+import org.apache.beam.sdk.io.BoundedSource;
+import org.apache.beam.sdk.io.CountingSource;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
+import org.apache.beam.sdk.util.SerializableUtils;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.TupleTag;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Captor;
+import org.mockito.Matchers;
+import org.mockito.Mock;
+import org.mockito.MockitoAnnotations;
+
+/** Tests for {@link ProcessBundleHandler}. */
+@RunWith(JUnit4.class)
+public class ProcessBundleHandlerTest {
+  private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+  private static final Coder<WindowedValue<String>> STRING_CODER =
+      WindowedValue.getFullCoder(StringUtf8Coder.of(), GlobalWindow.Coder.INSTANCE);
+  private static final long LONG_CODER_SPEC_ID = 998L;
+  private static final long STRING_CODER_SPEC_ID = 999L;
+  private static final BeamFnApi.RemoteGrpcPort REMOTE_PORT = BeamFnApi.RemoteGrpcPort.newBuilder()
+      .setApiServiceDescriptor(BeamFnApi.ApiServiceDescriptor.newBuilder()
+          .setId(58L)
+          .setUrl("TestUrl"))
+      .build();
+  private static final BeamFnApi.Coder LONG_CODER_SPEC;
+  private static final BeamFnApi.Coder STRING_CODER_SPEC;
+  static {
+    try {
+      STRING_CODER_SPEC =
+          BeamFnApi.Coder.newBuilder().setFunctionSpec(BeamFnApi.FunctionSpec.newBuilder()
+          .setId(STRING_CODER_SPEC_ID)
+          .setData(Any.pack(BytesValue.newBuilder().setValue(ByteString.copyFrom(
+              OBJECT_MAPPER.writeValueAsBytes(STRING_CODER.asCloudObject()))).build())))
+          .build();
+      LONG_CODER_SPEC =
+          BeamFnApi.Coder.newBuilder().setFunctionSpec(BeamFnApi.FunctionSpec.newBuilder()
+          .setId(STRING_CODER_SPEC_ID)
+          .setData(Any.pack(BytesValue.newBuilder().setValue(ByteString.copyFrom(
+              OBJECT_MAPPER.writeValueAsBytes(WindowedValue.getFullCoder(
+                  VarLongCoder.of(), GlobalWindow.Coder.INSTANCE).asCloudObject()))).build())))
+          .build();
+    } catch (IOException e) {
+      throw new ExceptionInInitializerError(e);
+    }
+  }
+
+  private static final String DATA_INPUT_URN = "urn:org.apache.beam:source:runner:0.1";
+  private static final String DATA_OUTPUT_URN = "urn:org.apache.beam:sink:runner:0.1";
+  private static final String JAVA_DO_FN_URN = "urn:org.apache.beam:dofn:java:0.1";
+  private static final String JAVA_SOURCE_URN = "urn:org.apache.beam:source:java:0.1";
+
+  @Rule public ExpectedException thrown = ExpectedException.none();
+
+  @Mock private BeamFnDataClient beamFnDataClient;
+  @Captor private ArgumentCaptor<ThrowingConsumer<WindowedValue<String>>> consumerCaptor;
+
+  @Before
+  public void setUp() {
+    MockitoAnnotations.initMocks(this);
+  }
+
+  @Test
+  public void testOrderOfStartAndFinishCalls() throws Exception {
+    BeamFnApi.ProcessBundleDescriptor processBundleDescriptor =
+        BeamFnApi.ProcessBundleDescriptor.newBuilder()
+        .addPrimitiveTransform(BeamFnApi.PrimitiveTransform.newBuilder().setId(2L))
+        .addPrimitiveTransform(BeamFnApi.PrimitiveTransform.newBuilder().setId(3L))
+        .build();
+    Map<Long, Message> fnApiRegistry = ImmutableMap.of(1L, processBundleDescriptor);
+
+    List<BeamFnApi.PrimitiveTransform> transformsProcessed = new ArrayList<>();
+    List<String> orderOfOperations = new ArrayList<>();
+
+    ProcessBundleHandler handler = new ProcessBundleHandler(
+        PipelineOptionsFactory.create(),
+        fnApiRegistry::get,
+        beamFnDataClient) {
+      @Override
+      protected <InputT, OutputT> void createConsumersForPrimitiveTransform(
+          BeamFnApi.PrimitiveTransform primitiveTransform,
+          Supplier<Long> processBundleInstructionId,
+          Function<BeamFnApi.Target,
+                   Collection<ThrowingConsumer<WindowedValue<OutputT>>>> consumers,
+          BiConsumer<BeamFnApi.Target, ThrowingConsumer<WindowedValue<InputT>>> addConsumer,
+          Consumer<ThrowingRunnable> addStartFunction,
+          Consumer<ThrowingRunnable> addFinishFunction)
+          throws IOException {
+
+        assertEquals((Long) 999L, processBundleInstructionId.get());
+
+        transformsProcessed.add(primitiveTransform);
+        addStartFunction.accept(
+            () -> orderOfOperations.add("Start" + primitiveTransform.getId()));
+        addFinishFunction.accept(
+            () -> orderOfOperations.add("Finish" + primitiveTransform.getId()));
+      }
+    };
+    handler.processBundle(BeamFnApi.InstructionRequest.newBuilder()
+        .setInstructionId(999L)
+        .setProcessBundle(
+            BeamFnApi.ProcessBundleRequest.newBuilder().setProcessBundleDescriptorReference(1L))
+        .build());
+
+    // Processing of primitive transforms is performed in reverse order.
+    assertThat(transformsProcessed, contains(
+        processBundleDescriptor.getPrimitiveTransform(1),
+        processBundleDescriptor.getPrimitiveTransform(0)));
+    // Start should occur in reverse order while finish calls should occur in forward order
+    assertThat(orderOfOperations, contains("Start3", "Start2", "Finish2", "Finish3"));
+  }
+
+  @Test
+  public void testCreatingPrimitiveTransformExceptionsArePropagated() throws Exception {
+    BeamFnApi.ProcessBundleDescriptor processBundleDescriptor =
+        BeamFnApi.ProcessBundleDescriptor.newBuilder()
+        .addPrimitiveTransform(BeamFnApi.PrimitiveTransform.newBuilder().setId(2L))
+        .addPrimitiveTransform(BeamFnApi.PrimitiveTransform.newBuilder().setId(3L))
+        .build();
+    Map<Long, Message> fnApiRegistry = ImmutableMap.of(1L, processBundleDescriptor);
+
+    ProcessBundleHandler handler = new ProcessBundleHandler(
+        PipelineOptionsFactory.create(),
+        fnApiRegistry::get,
+        beamFnDataClient) {
+      @Override
+      protected <InputT, OutputT> void createConsumersForPrimitiveTransform(
+          BeamFnApi.PrimitiveTransform primitiveTransform,
+          Supplier<Long> processBundleInstructionId,
+          Function<BeamFnApi.Target,
+                   Collection<ThrowingConsumer<WindowedValue<OutputT>>>> consumers,
+          BiConsumer<BeamFnApi.Target, ThrowingConsumer<WindowedValue<InputT>>> addConsumer,
+          Consumer<ThrowingRunnable> addStartFunction,
+          Consumer<ThrowingRunnable> addFinishFunction)
+          throws IOException {
+        thrown.expect(IllegalStateException.class);
+        thrown.expectMessage("TestException");
+        throw new IllegalStateException("TestException");
+      }
+    };
+    handler.processBundle(
+        BeamFnApi.InstructionRequest.newBuilder().setProcessBundle(
+            BeamFnApi.ProcessBundleRequest.newBuilder().setProcessBundleDescriptorReference(1L))
+        .build());
+  }
+
+  @Test
+  public void testPrimitiveTransformStartExceptionsArePropagated() throws Exception {
+    BeamFnApi.ProcessBundleDescriptor processBundleDescriptor =
+        BeamFnApi.ProcessBundleDescriptor.newBuilder()
+        .addPrimitiveTransform(BeamFnApi.PrimitiveTransform.newBuilder().setId(2L))
+        .addPrimitiveTransform(BeamFnApi.PrimitiveTransform.newBuilder().setId(3L))
+        .build();
+    Map<Long, Message> fnApiRegistry = ImmutableMap.of(1L, processBundleDescriptor);
+
+    ProcessBundleHandler handler = new ProcessBundleHandler(
+        PipelineOptionsFactory.create(),
+        fnApiRegistry::get,
+        beamFnDataClient) {
+      @Override
+      protected <InputT, OutputT> void createConsumersForPrimitiveTransform(
+          BeamFnApi.PrimitiveTransform primitiveTransform,
+          Supplier<Long> processBundleInstructionId,
+          Function<BeamFnApi.Target,
+                   Collection<ThrowingConsumer<WindowedValue<OutputT>>>> consumers,
+          BiConsumer<BeamFnApi.Target, ThrowingConsumer<WindowedValue<InputT>>> addConsumer,
+          Consumer<ThrowingRunnable> addStartFunction,
+          Consumer<ThrowingRunnable> addFinishFunction)
+          throws IOException {
+        thrown.expect(IllegalStateException.class);
+        thrown.expectMessage("TestException");
+        addStartFunction.accept(this::throwException);
+      }
+
+      private void throwException() {
+        throw new IllegalStateException("TestException");
+      }
+    };
+    handler.processBundle(
+        BeamFnApi.InstructionRequest.newBuilder().setProcessBundle(
+            BeamFnApi.ProcessBundleRequest.newBuilder().setProcessBundleDescriptorReference(1L))
+        .build());
+  }
+
+  @Test
+  public void testPrimitiveTransformFinishExceptionsArePropagated() throws Exception {
+    BeamFnApi.ProcessBundleDescriptor processBundleDescriptor =
+        BeamFnApi.ProcessBundleDescriptor.newBuilder()
+        .addPrimitiveTransform(BeamFnApi.PrimitiveTransform.newBuilder().setId(2L))
+        .addPrimitiveTransform(BeamFnApi.PrimitiveTransform.newBuilder().setId(3L))
+        .build();
+    Map<Long, Message> fnApiRegistry = ImmutableMap.of(1L, processBundleDescriptor);
+
+    ProcessBundleHandler handler = new ProcessBundleHandler(
+        PipelineOptionsFactory.create(),
+        fnApiRegistry::get,
+        beamFnDataClient) {
+      @Override
+      protected <InputT, OutputT> void createConsumersForPrimitiveTransform(
+          BeamFnApi.PrimitiveTransform primitiveTransform,
+          Supplier<Long> processBundleInstructionId,
+          Function<BeamFnApi.Target,
+                   Collection<ThrowingConsumer<WindowedValue<OutputT>>>> consumers,
+          BiConsumer<BeamFnApi.Target, ThrowingConsumer<WindowedValue<InputT>>> addConsumer,
+          Consumer<ThrowingRunnable> addStartFunction,
+          Consumer<ThrowingRunnable> addFinishFunction)
+          throws IOException {
+        thrown.expect(IllegalStateException.class);
+        thrown.expectMessage("TestException");
+        addFinishFunction.accept(this::throwException);
+      }
+
+      private void throwException() {
+        throw new IllegalStateException("TestException");
+      }
+    };
+    handler.processBundle(
+        BeamFnApi.InstructionRequest.newBuilder().setProcessBundle(
+            BeamFnApi.ProcessBundleRequest.newBuilder().setProcessBundleDescriptorReference(1L))
+        .build());
+  }
+
+  private static class TestDoFn extends DoFn<String, String> {
+    private static final TupleTag<String> mainOutput = new TupleTag<>("mainOutput");
+    private static final TupleTag<String> sideOutput = new TupleTag<>("sideOutput");
+
+    @StartBundle
+    public void startBundle(Context context) {
+      context.output("StartBundle");
+    }
+
+    @ProcessElement
+    public void processElement(ProcessContext context) {
+      context.output("MainOutput" + context.element());
+      context.sideOutput(sideOutput, "SideOutput" + context.element());
+    }
+
+    @FinishBundle
+    public void finishBundle(Context context) {
+      context.output("FinishBundle");
+    }
+  }
+
+  /**
+   * Create a DoFn that has 3 inputs (inputATarget1, inputATarget2, inputBTarget) and 2 outputs
+   * (mainOutput, sideOutput). Validate that inputs are fed to the {@link DoFn} and that outputs
+   * are directed to the correct consumers.
+   */
+  @Test
+  public void testCreatingAndProcessingDoFn() throws Exception {
+    Map<Long, Message> fnApiRegistry = ImmutableMap.of(STRING_CODER_SPEC_ID, STRING_CODER_SPEC);
+    long primitiveTransformId = 100L;
+    long mainOutputId = 101L;
+    long sideOutputId = 102L;
+
+    DoFnInfo<?, ?> doFnInfo = DoFnInfo.forFn(
+        new TestDoFn(),
+        WindowingStrategy.globalDefault(),
+        ImmutableList.of(),
+        STRING_CODER,
+        mainOutputId,
+        ImmutableMap.of(
+            mainOutputId, TestDoFn.mainOutput,
+            sideOutputId, TestDoFn.sideOutput));
+    BeamFnApi.FunctionSpec functionSpec = BeamFnApi.FunctionSpec.newBuilder()
+        .setId(1L)
+        .setUrn(JAVA_DO_FN_URN)
+        .setData(Any.pack(BytesValue.newBuilder()
+            .setValue(ByteString.copyFrom(SerializableUtils.serializeToByteArray(doFnInfo)))
+            .build()))
+        .build();
+    BeamFnApi.Target inputATarget1 = BeamFnApi.Target.newBuilder()
+        .setPrimitiveTransformReference(1000L)
+        .setName("inputATarget1")
+        .build();
+    BeamFnApi.Target inputATarget2 = BeamFnApi.Target.newBuilder()
+        .setPrimitiveTransformReference(1001L)
+        .setName("inputATarget1")
+        .build();
+    BeamFnApi.Target inputBTarget = BeamFnApi.Target.newBuilder()
+        .setPrimitiveTransformReference(1002L)
+        .setName("inputBTarget")
+        .build();
+    BeamFnApi.PrimitiveTransform primitiveTransform = BeamFnApi.PrimitiveTransform.newBuilder()
+        .setId(primitiveTransformId)
+        .setFunctionSpec(functionSpec)
+        .putInputs("inputA", BeamFnApi.Target.List.newBuilder()
+            .addTarget(inputATarget1)
+            .addTarget(inputATarget2)
+            .build())
+        .putInputs("inputB", BeamFnApi.Target.List.newBuilder()
+            .addTarget(inputBTarget)
+            .build())
+        .putOutputs(Long.toString(mainOutputId), BeamFnApi.PCollection.newBuilder()
+            .setCoderReference(STRING_CODER_SPEC_ID)
+            .build())
+        .putOutputs(Long.toString(sideOutputId), BeamFnApi.PCollection.newBuilder()
+            .setCoderReference(STRING_CODER_SPEC_ID)
+            .build())
+        .build();
+
+    List<WindowedValue<String>> mainOutputValues = new ArrayList<>();
+    List<WindowedValue<String>> sideOutputValues = new ArrayList<>();
+    BeamFnApi.Target mainOutputTarget = BeamFnApi.Target.newBuilder()
+        .setPrimitiveTransformReference(primitiveTransformId)
+        .setName(Long.toString(mainOutputId))
+        .build();
+    BeamFnApi.Target sideOutputTarget = BeamFnApi.Target.newBuilder()
+        .setPrimitiveTransformReference(primitiveTransformId)
+        .setName(Long.toString(sideOutputId))
+        .build();
+    Multimap<BeamFnApi.Target, ThrowingConsumer<WindowedValue<String>>> existingConsumers =
+        ImmutableMultimap.of(
+            mainOutputTarget, mainOutputValues::add,
+            sideOutputTarget, sideOutputValues::add);
+    Multimap<BeamFnApi.Target, ThrowingConsumer<WindowedValue<String>>> newConsumers =
+        HashMultimap.create();
+    List<ThrowingRunnable> startFunctions = new ArrayList<>();
+    List<ThrowingRunnable> finishFunctions = new ArrayList<>();
+
+    ProcessBundleHandler handler = new ProcessBundleHandler(
+        PipelineOptionsFactory.create(),
+        fnApiRegistry::get,
+        beamFnDataClient);
+    handler.createConsumersForPrimitiveTransform(
+        primitiveTransform,
+        Suppliers.ofInstance(57L)::get,
+        existingConsumers::get,
+        newConsumers::put,
+        startFunctions::add,
+        finishFunctions::add);
+
+    Iterables.getOnlyElement(startFunctions).run();
+    assertThat(mainOutputValues, contains(valueInGlobalWindow("StartBundle")));
+    mainOutputValues.clear();
+
+    assertEquals(newConsumers.keySet(),
+        ImmutableSet.of(inputATarget1, inputATarget2, inputBTarget));
+
+    Iterables.getOnlyElement(newConsumers.get(inputATarget1)).accept(valueInGlobalWindow("A1"));
+    Iterables.getOnlyElement(newConsumers.get(inputATarget1)).accept(valueInGlobalWindow("A2"));
+    Iterables.getOnlyElement(newConsumers.get(inputATarget1)).accept(valueInGlobalWindow("B"));
+    assertThat(mainOutputValues, contains(
+        valueInGlobalWindow("MainOutputA1"),
+        valueInGlobalWindow("MainOutputA2"),
+        valueInGlobalWindow("MainOutputB")));
+    assertThat(sideOutputValues, contains(
+        valueInGlobalWindow("SideOutputA1"),
+        valueInGlobalWindow("SideOutputA2"),
+        valueInGlobalWindow("SideOutputB")));
+    mainOutputValues.clear();
+    sideOutputValues.clear();
+
+    Iterables.getOnlyElement(finishFunctions).run();
+    assertThat(mainOutputValues, contains(valueInGlobalWindow("FinishBundle")));
+    mainOutputValues.clear();
+  }
+
+  @Test
+  public void testCreatingAndProcessingSource() throws Exception {
+    Map<Long, Message> fnApiRegistry = ImmutableMap.of(LONG_CODER_SPEC_ID, LONG_CODER_SPEC);
+    long primitiveTransformId = 100L;
+    long outputId = 101L;
+
+    BeamFnApi.Target inputTarget = BeamFnApi.Target.newBuilder()
+        .setPrimitiveTransformReference(1000L)
+        .setName("inputTarget")
+        .build();
+
+    List<WindowedValue<String>> outputValues = new ArrayList<>();
+    BeamFnApi.Target outputTarget = BeamFnApi.Target.newBuilder()
+        .setPrimitiveTransformReference(primitiveTransformId)
+        .setName(Long.toString(outputId))
+        .build();
+
+    Multimap<BeamFnApi.Target, ThrowingConsumer<WindowedValue<String>>> existingConsumers =
+        ImmutableMultimap.of(outputTarget, outputValues::add);
+    Multimap<BeamFnApi.Target,
+             ThrowingConsumer<WindowedValue<BoundedSource<Long>>>> newConsumers =
+             HashMultimap.create();
+    List<ThrowingRunnable> startFunctions = new ArrayList<>();
+    List<ThrowingRunnable> finishFunctions = new ArrayList<>();
+
+    BeamFnApi.FunctionSpec functionSpec = BeamFnApi.FunctionSpec.newBuilder()
+        .setId(1L)
+        .setUrn(JAVA_SOURCE_URN)
+        .setData(Any.pack(BytesValue.newBuilder()
+            .setValue(ByteString.copyFrom(
+                SerializableUtils.serializeToByteArray(CountingSource.upTo(3))))
+            .build()))
+        .build();
+
+    BeamFnApi.PrimitiveTransform primitiveTransform = BeamFnApi.PrimitiveTransform.newBuilder()
+        .setId(primitiveTransformId)
+        .setFunctionSpec(functionSpec)
+        .putInputs("input",
+            BeamFnApi.Target.List.newBuilder().addTarget(inputTarget).build())
+        .putOutputs(Long.toString(outputId),
+            BeamFnApi.PCollection.newBuilder().setCoderReference(LONG_CODER_SPEC_ID).build())
+        .build();
+
+    ProcessBundleHandler handler = new ProcessBundleHandler(
+        PipelineOptionsFactory.create(),
+        fnApiRegistry::get,
+        beamFnDataClient);
+
+    handler.createConsumersForPrimitiveTransform(
+        primitiveTransform,
+        Suppliers.ofInstance(57L)::get,
+        existingConsumers::get,
+        newConsumers::put,
+        startFunctions::add,
+        finishFunctions::add);
+
+    // This is testing a deprecated way of running sources and should be removed
+    // once all source definitions are instead propagated along the input edge.
+    Iterables.getOnlyElement(startFunctions).run();
+    assertThat(outputValues, contains(
+        valueInGlobalWindow(0L),
+        valueInGlobalWindow(1L),
+        valueInGlobalWindow(2L)));
+    outputValues.clear();
+
+    // Check that when passing a source along as an input, the source is processed.
+    assertEquals(newConsumers.keySet(), ImmutableSet.of(inputTarget));
+    Iterables.getOnlyElement(newConsumers.get(inputTarget)).accept(
+        valueInGlobalWindow(CountingSource.upTo(2)));
+    assertThat(outputValues, contains(
+        valueInGlobalWindow(0L),
+        valueInGlobalWindow(1L)));
+
+    assertThat(finishFunctions, empty());
+  }
+
+  @Test
+  public void testCreatingAndProcessingBeamFnDataReadRunner() throws Exception {
+    Map<Long, Message> fnApiRegistry = ImmutableMap.of(STRING_CODER_SPEC_ID, STRING_CODER_SPEC);
+    long bundleId = 57L;
+    long primitiveTransformId = 100L;
+    long outputId = 101L;
+
+    List<WindowedValue<String>> outputValues = new ArrayList<>();
+    BeamFnApi.Target outputTarget = BeamFnApi.Target.newBuilder()
+        .setPrimitiveTransformReference(primitiveTransformId)
+        .setName(Long.toString(outputId))
+        .build();
+
+    Multimap<BeamFnApi.Target, ThrowingConsumer<WindowedValue<String>>> existingConsumers =
+        ImmutableMultimap.of(outputTarget, outputValues::add);
+    Multimap<BeamFnApi.Target, ThrowingConsumer<WindowedValue<String>>> newConsumers =
+        HashMultimap.create();
+    List<ThrowingRunnable> startFunctions = new ArrayList<>();
+    List<ThrowingRunnable> finishFunctions = new ArrayList<>();
+
+    BeamFnApi.FunctionSpec functionSpec = BeamFnApi.FunctionSpec.newBuilder()
+        .setId(1L)
+        .setUrn(DATA_INPUT_URN)
+        .setData(Any.pack(REMOTE_PORT))
+        .build();
+
+    BeamFnApi.PrimitiveTransform primitiveTransform = BeamFnApi.PrimitiveTransform.newBuilder()
+        .setId(primitiveTransformId)
+        .setFunctionSpec(functionSpec)
+        .putInputs("input", BeamFnApi.Target.List.getDefaultInstance())
+        .putOutputs(Long.toString(outputId),
+            BeamFnApi.PCollection.newBuilder().setCoderReference(STRING_CODER_SPEC_ID).build())
+        .build();
+
+    ProcessBundleHandler handler = new ProcessBundleHandler(
+        PipelineOptionsFactory.create(),
+        fnApiRegistry::get,
+        beamFnDataClient);
+
+    handler.createConsumersForPrimitiveTransform(
+        primitiveTransform,
+        Suppliers.ofInstance(bundleId)::get,
+        existingConsumers::get,
+        newConsumers::put,
+        startFunctions::add,
+        finishFunctions::add);
+
+    verifyZeroInteractions(beamFnDataClient);
+
+    CompletableFuture<Void> completionFuture = new CompletableFuture<>();
+    when(beamFnDataClient.forInboundConsumer(any(), any(), any(), any()))
+        .thenReturn(completionFuture);
+    Iterables.getOnlyElement(startFunctions).run();
+    verify(beamFnDataClient).forInboundConsumer(
+        eq(REMOTE_PORT.getApiServiceDescriptor()),
+        eq(KV.of(bundleId, BeamFnApi.Target.newBuilder()
+            .setPrimitiveTransformReference(primitiveTransformId)
+            .setName("input")
+            .build())),
+        eq(STRING_CODER),
+        consumerCaptor.capture());
+
+    consumerCaptor.getValue().accept(valueInGlobalWindow("TestValue"));
+    assertThat(outputValues, contains(valueInGlobalWindow("TestValue")));
+    outputValues.clear();
+
+    assertThat(newConsumers.keySet(), empty());
+
+    completionFuture.complete(null);
+    Iterables.getOnlyElement(finishFunctions).run();
+
+    verifyNoMoreInteractions(beamFnDataClient);
+  }
+
+  @Test
+  public void testCreatingAndProcessingBeamFnDataWriteRunner() throws Exception {
+    Map<Long, Message> fnApiRegistry = ImmutableMap.of(STRING_CODER_SPEC_ID, STRING_CODER_SPEC);
+    long bundleId = 57L;
+    long primitiveTransformId = 100L;
+    long outputId = 101L;
+
+    BeamFnApi.Target inputTarget = BeamFnApi.Target.newBuilder()
+        .setPrimitiveTransformReference(1000L)
+        .setName("inputTarget")
+        .build();
+
+    Multimap<BeamFnApi.Target, ThrowingConsumer<WindowedValue<String>>> existingConsumers =
+        ImmutableMultimap.of();
+    Multimap<BeamFnApi.Target, ThrowingConsumer<WindowedValue<String>>> newConsumers =
+        HashMultimap.create();
+    List<ThrowingRunnable> startFunctions = new ArrayList<>();
+    List<ThrowingRunnable> finishFunctions = new ArrayList<>();
+
+    BeamFnApi.FunctionSpec functionSpec = BeamFnApi.FunctionSpec.newBuilder()
+        .setId(1L)
+        .setUrn(DATA_OUTPUT_URN)
+        .setData(Any.pack(REMOTE_PORT))
+        .build();
+
+    BeamFnApi.PrimitiveTransform primitiveTransform = BeamFnApi.PrimitiveTransform.newBuilder()
+        .setId(primitiveTransformId)
+        .setFunctionSpec(functionSpec)
+        .putInputs("input", BeamFnApi.Target.List.newBuilder().addTarget(inputTarget).build())
+        .putOutputs(Long.toString(outputId),
+            BeamFnApi.PCollection.newBuilder().setCoderReference(STRING_CODER_SPEC_ID).build())
+        .build();
+
+    ProcessBundleHandler handler = new ProcessBundleHandler(
+        PipelineOptionsFactory.create(),
+        fnApiRegistry::get,
+        beamFnDataClient);
+
+    handler.createConsumersForPrimitiveTransform(
+        primitiveTransform,
+        Suppliers.ofInstance(bundleId)::get,
+        existingConsumers::get,
+        newConsumers::put,
+        startFunctions::add,
+        finishFunctions::add);
+
+    verifyZeroInteractions(beamFnDataClient);
+
+    List<WindowedValue<String>> outputValues = new ArrayList<>();
+    AtomicBoolean wasCloseCalled = new AtomicBoolean();
+    CloseableThrowingConsumer<WindowedValue<String>> outputConsumer =
+        new CloseableThrowingConsumer<WindowedValue<String>>(){
+      @Override
+      public void close() throws Exception {
+        wasCloseCalled.set(true);
+      }
+
+      @Override
+      public void accept(WindowedValue<String> t) throws Exception {
+        outputValues.add(t);
+      }
+    };
+
+    when(beamFnDataClient.forOutboundConsumer(
+        any(),
+        any(),
+        Matchers.<Coder<WindowedValue<String>>>any())).thenReturn(outputConsumer);
+    Iterables.getOnlyElement(startFunctions).run();
+    verify(beamFnDataClient).forOutboundConsumer(
+        eq(REMOTE_PORT.getApiServiceDescriptor()),
+        eq(KV.of(bundleId, BeamFnApi.Target.newBuilder()
+            .setPrimitiveTransformReference(primitiveTransformId)
+            .setName(Long.toString(outputId))
+            .build())),
+        eq(STRING_CODER));
+
+    assertEquals(newConsumers.keySet(), ImmutableSet.of(inputTarget));
+    Iterables.getOnlyElement(newConsumers.get(inputTarget)).accept(
+        valueInGlobalWindow("TestValue"));
+    assertThat(outputValues, contains(valueInGlobalWindow("TestValue")));
+    outputValues.clear();
+
+    assertFalse(wasCloseCalled.get());
+    Iterables.getOnlyElement(finishFunctions).run();
+    assertTrue(wasCloseCalled.get());
+
+    verifyNoMoreInteractions(beamFnDataClient);
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/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
new file mode 100644
index 0000000..7b07a08
--- /dev/null
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/RegisterHandlerTest.java
@@ -0,0 +1,80 @@
+/*
+ * 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.control;
+
+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.fn.v1.BeamFnApi;
+import org.apache.beam.fn.v1.BeamFnApi.RegisterResponse;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Tests for {@link RegisterHandler}. */
+@RunWith(JUnit4.class)
+public class RegisterHandlerTest {
+  @Rule public TestExecutorService executor = TestExecutors.from(Executors::newCachedThreadPool);
+
+  private static final BeamFnApi.InstructionRequest REGISTER_REQUEST =
+      BeamFnApi.InstructionRequest.newBuilder()
+      .setInstructionId(1L)
+      .setRegister(BeamFnApi.RegisterRequest.newBuilder()
+          .addProcessBundleDescriptor(BeamFnApi.ProcessBundleDescriptor.newBuilder().setId(1L)
+              .addCoders(BeamFnApi.Coder.newBuilder().setFunctionSpec(
+                  BeamFnApi.FunctionSpec.newBuilder().setId(10L)).build()))
+          .addProcessBundleDescriptor(BeamFnApi.ProcessBundleDescriptor.newBuilder().setId(2L)
+              .addCoders(BeamFnApi.Coder.newBuilder().setFunctionSpec(
+                  BeamFnApi.FunctionSpec.newBuilder().setId(20L)).build()))
+          .build())
+      .build();
+  private static final BeamFnApi.InstructionResponse REGISTER_RESPONSE =
+      BeamFnApi.InstructionResponse.newBuilder()
+      .setRegister(RegisterResponse.getDefaultInstance())
+      .build();
+
+  @Test
+  public void testRegistration() throws Exception {
+    RegisterHandler handler = new RegisterHandler();
+    Future<BeamFnApi.InstructionResponse> responseFuture =
+        executor.submit(new Callable<BeamFnApi.InstructionResponse>() {
+          @Override
+          public BeamFnApi.InstructionResponse call() throws Exception {
+            // Purposefully wait a small amount of time making it likely that
+            // a downstream caller needs to block.
+            Thread.sleep(100);
+            return handler.register(REGISTER_REQUEST).build();
+          }
+    });
+    assertEquals(REGISTER_REQUEST.getRegister().getProcessBundleDescriptor(0),
+        handler.getById(1L));
+    assertEquals(REGISTER_REQUEST.getRegister().getProcessBundleDescriptor(1),
+        handler.getById(2L));
+    assertEquals(REGISTER_REQUEST.getRegister().getProcessBundleDescriptor(0).getCoders(0),
+        handler.getById(10L));
+    assertEquals(REGISTER_REQUEST.getRegister().getProcessBundleDescriptor(1).getCoders(0),
+        handler.getById(20L));
+    assertEquals(REGISTER_RESPONSE, responseFuture.get());
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/0b4b2bec/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
new file mode 100644
index 0000000..64a0e11
--- /dev/null
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataBufferingOutboundObserverTest.java
@@ -0,0 +1,142 @@
+/*
+ * 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.data;
+
+import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow;
+import static org.hamcrest.Matchers.empty;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+
+import com.google.common.collect.Iterables;
+import com.google.protobuf.ByteString;
+import java.io.IOException;
+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.fn.v1.BeamFnApi;
+import org.apache.beam.sdk.coders.ByteArrayCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.Coder.Context;
+import org.apache.beam.sdk.coders.LengthPrefixCoder;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.KV;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Tests for {@link BeamFnDataBufferingOutboundObserver}. */
+@RunWith(JUnit4.class)
+public class BeamFnDataBufferingOutboundObserverTest {
+  private static final int DEFAULT_BUFFER_LIMIT = 1_000_000;
+  private static final KV<Long, BeamFnApi.Target> OUTPUT_LOCATION = KV.of(777L,
+      BeamFnApi.Target.newBuilder().setPrimitiveTransformReference(555L).setName("Test").build());
+  private static final Coder<WindowedValue<byte[]>> CODER =
+      LengthPrefixCoder.of(WindowedValue.getValueOnlyCoder(ByteArrayCoder.of()));
+
+  @Test
+  public void testWithDefaultBuffer() throws Exception {
+    Collection<BeamFnApi.Elements> values = new ArrayList<>();
+    AtomicBoolean onCompletedWasCalled = new AtomicBoolean();
+    CloseableThrowingConsumer<WindowedValue<byte[]>> consumer =
+        new BeamFnDataBufferingOutboundObserver<>(
+        PipelineOptionsFactory.create(),
+        OUTPUT_LOCATION,
+        CODER,
+        TestStreams.withOnNext(values::add)
+            .withOnCompleted(() -> onCompletedWasCalled.set(true))
+            .build());
+
+    // Test that nothing is emitted till the default buffer size is surpassed.
+    consumer.accept(valueInGlobalWindow(new byte[DEFAULT_BUFFER_LIMIT - 50]));
+    assertThat(values, empty());
+
+    // Test that when we cross the buffer, we emit.
+    consumer.accept(valueInGlobalWindow(new byte[50]));
+    assertEquals(
+        messageWithData(new byte[DEFAULT_BUFFER_LIMIT - 50], new byte[50]),
+        Iterables.get(values, 0));
+
+    // Test that nothing is emitted till the default buffer size is surpassed after a reset
+    consumer.accept(valueInGlobalWindow(new byte[DEFAULT_BUFFER_LIMIT - 50]));
+    assertEquals(1, values.size());
+
+    // Test that when we cross the buffer, we emit.
+    consumer.accept(valueInGlobalWindow(new byte[50]));
+    assertEquals(
+        messageWithData(new byte[DEFAULT_BUFFER_LIMIT - 50], new byte[50]),
+        Iterables.get(values, 1));
+
+    // Test that when we close with an empty buffer we only have one end of stream
+    consumer.close();
+    assertEquals(messageWithData(),
+        Iterables.get(values, 2));
+  }
+
+  @Test
+  public void testExperimentConfiguresBufferLimit() throws Exception {
+    Collection<BeamFnApi.Elements> values = new ArrayList<>();
+    AtomicBoolean onCompletedWasCalled = new AtomicBoolean();
+    CloseableThrowingConsumer<WindowedValue<byte[]>> consumer =
+        new BeamFnDataBufferingOutboundObserver<>(
+        PipelineOptionsFactory.fromArgs(
+            new String[] { "--experiments=beam_fn_api_data_buffer_limit=100" }).create(),
+        OUTPUT_LOCATION,
+        CODER,
+        TestStreams.withOnNext(values::add)
+            .withOnCompleted(() -> onCompletedWasCalled.set(true))
+            .build());
+
+    // Test that nothing is emitted till the default buffer size is surpassed.
+    consumer.accept(valueInGlobalWindow(new byte[51]));
+    assertThat(values, empty());
+
+    // Test that when we cross the buffer, we emit.
+    consumer.accept(valueInGlobalWindow(new byte[49]));
+    assertEquals(
+        messageWithData(new byte[51], new byte[49]),
+        Iterables.get(values, 0));
+
+    // Test that when we close we empty the value, and then the stream terminator as part
+    // of the same message
+    consumer.accept(valueInGlobalWindow(new byte[1]));
+    consumer.close();
+    assertEquals(
+        BeamFnApi.Elements.newBuilder(messageWithData(new byte[1]))
+            .addData(BeamFnApi.Elements.Data.newBuilder()
+                .setInstructionReference(OUTPUT_LOCATION.getKey())
+                .setTarget(OUTPUT_LOCATION.getValue()))
+            .build(),
+        Iterables.get(values, 1));
+  }
+
+  private static BeamFnApi.Elements messageWithData(byte[] ... datum) throws IOException {
+    ByteString.Output output = ByteString.newOutput();
+    for (byte[] data : datum) {
+      CODER.encode(valueInGlobalWindow(data), output, Context.NESTED);
+    }
+    return BeamFnApi.Elements.newBuilder()
+        .addData(BeamFnApi.Elements.Data.newBuilder()
+            .setInstructionReference(OUTPUT_LOCATION.getKey())
+            .setTarget(OUTPUT_LOCATION.getValue())
+            .setData(output.toByteString()))
+        .build();
+  }
+}