[ 
https://issues.apache.org/jira/browse/BEAM-3327?focusedWorklogId=97661&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-97661
 ]

ASF GitHub Bot logged work on BEAM-3327:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 02/May/18 19:12
            Start Date: 02/May/18 19:12
    Worklog Time Spent: 10m 
      Work Description: tgroh closed pull request #5152: [BEAM-3327] Harness 
Manager Interfaces
URL: https://github.com/apache/beam/pull/5152
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git 
a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/artifact/ArtifactSource.java
 
b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/artifact/ArtifactSource.java
new file mode 100644
index 00000000000..a8b582640b5
--- /dev/null
+++ 
b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/artifact/ArtifactSource.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.fnexecution.artifact;
+
+import io.grpc.stub.StreamObserver;
+import java.io.IOException;
+import org.apache.beam.model.jobmanagement.v1.ArtifactApi.ArtifactChunk;
+import org.apache.beam.model.jobmanagement.v1.ArtifactApi.Manifest;
+
+/**
+ * Makes artifacts available to an ArtifactRetrievalService by
+ * encapsulating runner-specific resources.
+ */
+public interface ArtifactSource {
+
+  /**
+   * Get the artifact manifest available from this source.
+   */
+  Manifest getManifest() throws IOException;
+
+  /**
+   * Get an artifact by its name.
+   */
+  void getArtifact(String name, StreamObserver<ArtifactChunk> 
responseObserver);
+}
diff --git 
a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/JobBundleFactory.java
 
b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/JobBundleFactory.java
new file mode 100644
index 00000000000..96286fa0eef
--- /dev/null
+++ 
b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/JobBundleFactory.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.fnexecution.control;
+
+import org.apache.beam.runners.core.construction.graph.ExecutableStage;
+import org.apache.beam.runners.fnexecution.state.StateRequestHandler;
+
+/**
+ * A factory that has all job-scoped information, and can be combined with 
stage-scoped information
+ * to create a {@link StageBundleFactory}.
+ *
+ * <p>Releases all job-scoped resources when closed.
+ */
+public interface JobBundleFactory extends AutoCloseable {
+  StageBundleFactory forStage(
+      ExecutableStage executableStage, StateRequestHandler 
stateRequestHandler);
+}
diff --git 
a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/OutputReceiverFactory.java
 
b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/OutputReceiverFactory.java
new file mode 100644
index 00000000000..ae818ca0726
--- /dev/null
+++ 
b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/OutputReceiverFactory.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.fnexecution.control;
+
+/**
+ * A factory that can create output receivers during an executable stage.
+ */
+public interface OutputReceiverFactory {
+  /**
+   * Get a new {@link RemoteOutputReceiver} for an output PCollection.
+   *
+   * This call should only be invoked once per PCollectionId per factory.
+   */
+  <OutputT> RemoteOutputReceiver<OutputT> create(String pCollectionId);
+}
diff --git 
a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/RemoteBundle.java
 
b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/RemoteBundle.java
new file mode 100644
index 00000000000..bc05f776fa1
--- /dev/null
+++ 
b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/RemoteBundle.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.fnexecution.control;
+
+import org.apache.beam.sdk.fn.data.FnDataReceiver;
+import org.apache.beam.sdk.util.WindowedValue;
+
+/**
+ * A bundle capable of handling input data elements for a
+ * {@link 
org.apache.beam.model.fnexecution.v1.BeamFnApi.ProcessBundleDescriptor bundle 
descriptor}
+ * by forwarding them to a remote environment for processing.
+ *
+ * <p>When a RemoteBundle is closed, it will block until bundle processing is 
finished on remote
+ * resources, and throw an exception if bundle processing has failed.
+ */
+public interface RemoteBundle<InputT> extends AutoCloseable {
+  /**
+   * Get an id used to represent this bundle.
+   */
+  String getId();
+
+  /**
+   * Get a {@link FnDataReceiver receiver} which consumes input elements, 
forwarding them to the
+   * remote environment.
+   */
+  FnDataReceiver<WindowedValue<InputT>> getInputReceiver();
+}
diff --git 
a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/RemoteOutputReceiver.java
 
b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/RemoteOutputReceiver.java
new file mode 100644
index 00000000000..a5536736a21
--- /dev/null
+++ 
b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/RemoteOutputReceiver.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.fnexecution.control;
+
+import com.google.auto.value.AutoValue;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.fn.data.FnDataReceiver;
+import org.apache.beam.sdk.fn.data.LogicalEndpoint;
+
+/**
+ * A pair of {@link Coder} and {@link FnDataReceiver} which can be registered 
to receive elements
+ * for a {@link LogicalEndpoint}.
+ */
+@AutoValue
+public abstract class RemoteOutputReceiver<T> {
+  public static <T> RemoteOutputReceiver of(Coder<T> coder, FnDataReceiver<T> 
receiver) {
+    return new AutoValue_RemoteOutputReceiver<>(coder, receiver);
+  }
+
+  public abstract Coder<T> getCoder();
+  public abstract FnDataReceiver<T> getReceiver();
+}
diff --git 
a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClient.java
 
b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClient.java
index 4f4824b119d..ae6f87678fd 100644
--- 
a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClient.java
+++ 
b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClient.java
@@ -20,7 +20,6 @@
 import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkState;
 
-import com.google.auto.value.AutoValue;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.concurrent.CompletionStage;
@@ -36,7 +35,6 @@
 import org.apache.beam.runners.fnexecution.data.RemoteInputDestination;
 import org.apache.beam.runners.fnexecution.state.StateDelegator;
 import org.apache.beam.runners.fnexecution.state.StateRequestHandler;
-import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.fn.data.CloseableFnDataReceiver;
 import org.apache.beam.sdk.fn.data.FnDataReceiver;
 import org.apache.beam.sdk.fn.data.InboundDataClient;
@@ -189,7 +187,7 @@ private BundleProcessor(
   }
 
   /** An active bundle for a particular {@link 
BeamFnApi.ProcessBundleDescriptor}. */
-  public static class ActiveBundle<InputT> implements AutoCloseable {
+  public static class ActiveBundle<InputT> implements RemoteBundle<InputT> {
     private final String bundleId;
     private final CompletionStage<BeamFnApi.ProcessBundleResponse> response;
     private final CloseableFnDataReceiver<WindowedValue<InputT>> inputReceiver;
@@ -212,7 +210,7 @@ private ActiveBundle(
     /**
      * Returns an id used to represent this bundle.
      */
-    public String getBundleId() {
+    public String getId() {
       return bundleId;
     }
 
@@ -450,17 +448,4 @@ public void abort() {
   @Override
   public void close() {}
 
-  /**
-   * A pair of {@link Coder} and {@link FnDataReceiver} which can be 
registered to receive elements
-   * for a {@link LogicalEndpoint}.
-   */
-  @AutoValue
-  public abstract static class RemoteOutputReceiver<T> {
-    public static <T> RemoteOutputReceiver of (Coder<T> coder, 
FnDataReceiver<T> receiver) {
-      return new AutoValue_SdkHarnessClient_RemoteOutputReceiver<>(coder, 
receiver);
-    }
-
-    public abstract Coder<T> getCoder();
-    public abstract FnDataReceiver<T> getReceiver();
-  }
 }
diff --git 
a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/StageBundleFactory.java
 
b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/StageBundleFactory.java
new file mode 100644
index 00000000000..812ae1cdf33
--- /dev/null
+++ 
b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/StageBundleFactory.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.fnexecution.control;
+
+import org.apache.beam.runners.fnexecution.state.StateRequestHandler;
+
+/**
+ * A bundle factory scoped to a particular
+ * {@link org.apache.beam.runners.core.construction.graph.ExecutableStage}, 
which has all of the
+ * resources it needs to provide new {@link RemoteBundle RemoteBundles}.
+ *
+ * <p>Closing a StageBundleFactory signals that the stage has completed and 
any resources bound to
+ * its lifetime can be cleaned up.
+ */
+public interface StageBundleFactory extends AutoCloseable {
+  /**
+   * Get a new {@link RemoteBundle bundle} for processing the data in an 
executable stage.
+   */
+  <InputT> RemoteBundle<InputT> getBundle(
+      OutputReceiverFactory outputReceiverFactory,
+      StateRequestHandler stateRequestHandler) throws Exception;
+}
diff --git 
a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/provisioning/JobInfo.java
 
b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/provisioning/JobInfo.java
new file mode 100644
index 00000000000..0c1f8b94181
--- /dev/null
+++ 
b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/provisioning/JobInfo.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.fnexecution.provisioning;
+
+import com.google.auto.value.AutoValue;
+import com.google.protobuf.Struct;
+
+/**
+ * A subset of {@link 
org.apache.beam.model.fnexecution.v1.ProvisionApi.ProvisionInfo} that
+ * specifies a unique job, while omitting fields that are not known to the 
runner operator.
+ */
+@AutoValue
+public abstract class JobInfo {
+  public static JobInfo create(String jobId, String jobName, Struct 
pipelineOptions) {
+    return new AutoValue_JobInfo(jobId, jobName, pipelineOptions);
+  }
+
+  public abstract String jobId();
+  public abstract String jobName();
+  public abstract Struct pipelineOptions();
+}
diff --git 
a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/RemoteExecutionTest.java
 
b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/RemoteExecutionTest.java
index 80236f1a7be..0026501137a 100644
--- 
a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/RemoteExecutionTest.java
+++ 
b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/RemoteExecutionTest.java
@@ -51,7 +51,6 @@
 import 
org.apache.beam.runners.fnexecution.control.ProcessBundleDescriptors.ExecutableProcessBundleDescriptor;
 import 
org.apache.beam.runners.fnexecution.control.SdkHarnessClient.ActiveBundle;
 import 
org.apache.beam.runners.fnexecution.control.SdkHarnessClient.BundleProcessor;
-import 
org.apache.beam.runners.fnexecution.control.SdkHarnessClient.RemoteOutputReceiver;
 import org.apache.beam.runners.fnexecution.data.GrpcDataService;
 import org.apache.beam.runners.fnexecution.data.RemoteInputDestination;
 import org.apache.beam.runners.fnexecution.logging.GrpcLoggingService;
diff --git 
a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClientTest.java
 
b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClientTest.java
index 96b3f5917dc..423b5e7995b 100644
--- 
a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClientTest.java
+++ 
b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClientTest.java
@@ -53,7 +53,6 @@
 import org.apache.beam.runners.fnexecution.InProcessSdkHarness;
 import 
org.apache.beam.runners.fnexecution.control.SdkHarnessClient.ActiveBundle;
 import 
org.apache.beam.runners.fnexecution.control.SdkHarnessClient.BundleProcessor;
-import 
org.apache.beam.runners.fnexecution.control.SdkHarnessClient.RemoteOutputReceiver;
 import org.apache.beam.runners.fnexecution.data.FnDataService;
 import org.apache.beam.runners.fnexecution.data.RemoteInputDestination;
 import org.apache.beam.runners.fnexecution.state.StateDelegator;


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
[email protected]


Issue Time Tracking
-------------------

    Worklog Id:     (was: 97661)
    Time Spent: 29h 10m  (was: 29h)

> Add abstractions to manage Environment Instance lifecycles.
> -----------------------------------------------------------
>
>                 Key: BEAM-3327
>                 URL: https://issues.apache.org/jira/browse/BEAM-3327
>             Project: Beam
>          Issue Type: New Feature
>          Components: runner-core
>            Reporter: Thomas Groh
>            Assignee: Axel Magnuson
>            Priority: Major
>              Labels: portability
>          Time Spent: 29h 10m
>  Remaining Estimate: 0h
>
> This permits remote stage execution for arbitrary environments



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to