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 (> 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();
+ }
+}