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

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

                Author: ASF GitHub Bot
            Created on: 22/May/18 23:52
            Start Date: 22/May/18 23:52
    Worklog Time Spent: 10m 
      Work Description: lukecwik commented on a change in pull request #5349: 
[BEAM-3326] Remote stage evaluator
URL: https://github.com/apache/beam/pull/5349#discussion_r190083099
 
 

 ##########
 File path: 
runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/InProcessEnvironmentFactory.java
 ##########
 @@ -0,0 +1,97 @@
+/*
+ * 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.direct.portable;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import java.time.Duration;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import org.apache.beam.fn.harness.FnHarness;
+import org.apache.beam.model.pipeline.v1.Endpoints.ApiServiceDescriptor;
+import org.apache.beam.model.pipeline.v1.RunnerApi.Environment;
+import org.apache.beam.runners.fnexecution.GrpcFnServer;
+import org.apache.beam.runners.fnexecution.control.ControlClientPool;
+import 
org.apache.beam.runners.fnexecution.control.FnApiControlClientPoolService;
+import org.apache.beam.runners.fnexecution.control.InstructionRequestHandler;
+import org.apache.beam.runners.fnexecution.environment.EnvironmentFactory;
+import org.apache.beam.runners.fnexecution.environment.RemoteEnvironment;
+import org.apache.beam.runners.fnexecution.logging.GrpcLoggingService;
+import org.apache.beam.sdk.fn.stream.StreamObserverFactory;
+import org.apache.beam.sdk.fn.test.InProcessManagedChannelFactory;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+
+/**
+ * An {@link EnvironmentFactory} that communicates to a {@link FnHarness} via 
the in-process gRPC
+ * channel.
+ *
+ * <p>TODO: Move this class to the runners/java-fn-execution module, with the 
Java SDK Harness as a
+ * provided dependency.
+ */
+class InProcessEnvironmentFactory implements EnvironmentFactory {
+
+  private final GrpcFnServer<GrpcLoggingService> loggingServer;
+  private final GrpcFnServer<FnApiControlClientPoolService> controlServer;
+
+  private final ControlClientPool.Source clientSource;
+
+  InProcessEnvironmentFactory(
+      GrpcFnServer<GrpcLoggingService> loggingServer,
+      GrpcFnServer<FnApiControlClientPoolService> controlServer,
+      ControlClientPool.Source clientSource) {
+    this.loggingServer = loggingServer;
+    this.controlServer = controlServer;
+    checkArgument(
+        loggingServer.getApiServiceDescriptor() != null,
+        "Logging Server cannot have a null %s",
+        ApiServiceDescriptor.class.getSimpleName());
+    checkArgument(
+        controlServer.getApiServiceDescriptor() != null,
+        "Control Server cannot have a null %s",
+        ApiServiceDescriptor.class.getSimpleName());
+    this.clientSource = clientSource;
+  }
+
+  @Override
+  public RemoteEnvironment createEnvironment(Environment container) throws 
Exception {
+    ExecutorService executor = Executors.newSingleThreadExecutor();
+    Future<?> fnHarness =
+        executor.submit(
+            () ->
+                FnHarness.main(
 
 Review comment:
   The best scenario would be if we created a new classpath which contained the 
Java harness code + what we detect in the users classpath and ran that. This 
would have the benefit of simulating the SDK container and remove the need for 
users to remember to put provide beam-sdks-java-harness as a dependency. The 
idea would be that beam-sdks-java-harness would be a jar containing all of its 
dependencies shaded away similar to what we do for Dataflow. We would embed the 
beam-sdks-java-harness.jar as a jar inside the direct runner jar so we wouldn't 
need to fetch it from Maven and also would be able to solve the problem where 
tests would run from the provided jar.
   
   If this seems to much, a corresponding JIRA and updating this code so if the 
FnHarness.main would throw a `ClassNotFoundException`, we tell users that the 
shared process execution mode requires that the 
org.apache.beam:beam-sdks-java-harness:2.5.0 and its dependencies be part of 
the classpath.

----------------------------------------------------------------
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:
us...@infra.apache.org


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

    Worklog Id:     (was: 104871)
    Time Spent: 12h 10m  (was: 12h)

> Execute a Stage via the portability framework in the ReferenceRunner
> --------------------------------------------------------------------
>
>                 Key: BEAM-3326
>                 URL: https://issues.apache.org/jira/browse/BEAM-3326
>             Project: Beam
>          Issue Type: New Feature
>          Components: runner-core
>            Reporter: Thomas Groh
>            Assignee: Thomas Groh
>            Priority: Major
>              Labels: portability
>          Time Spent: 12h 10m
>  Remaining Estimate: 0h
>
> This is the supertask for remote execution in the Universal Local Runner 
> (BEAM-2899).
> This executes a stage remotely via portability framework APIs



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

Reply via email to