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

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

                Author: ASF GitHub Bot
            Created on: 05/Dec/18 11:48
            Start Date: 05/Dec/18 11:48
    Worklog Time Spent: 10m 
      Work Description: robertwb commented on a change in pull request #7078: 
[BEAM-6094] Implement external environment for portable BeamPython.
URL: https://github.com/apache/beam/pull/7078#discussion_r239031615
 
 

 ##########
 File path: 
runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/ExternalEnvironmentFactory.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.runners.fnexecution.environment;
+
+import com.google.common.base.Preconditions;
+import java.time.Duration;
+import java.util.concurrent.TimeoutException;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi;
+import org.apache.beam.model.fnexecution.v1.BeamFnExternalWorkerGrpc;
+import org.apache.beam.model.pipeline.v1.RunnerApi;
+import org.apache.beam.model.pipeline.v1.RunnerApi.Environment;
+import org.apache.beam.runners.core.construction.BeamUrns;
+import org.apache.beam.runners.fnexecution.GrpcFnServer;
+import org.apache.beam.runners.fnexecution.artifact.ArtifactRetrievalService;
+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.logging.GrpcLoggingService;
+import 
org.apache.beam.runners.fnexecution.provisioning.StaticGrpcProvisionService;
+import org.apache.beam.sdk.fn.IdGenerator;
+import org.apache.beam.sdk.fn.channel.ManagedChannelFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An {@link EnvironmentFactory} which forks processes based on the given URL 
in the Environment.
+ * The returned {@link ProcessEnvironment} has to make sure to stop the 
processes.
+ */
+public class ExternalEnvironmentFactory implements EnvironmentFactory {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(ExternalEnvironmentFactory.class);
+
+  public static ExternalEnvironmentFactory create(
+      GrpcFnServer<FnApiControlClientPoolService> controlServiceServer,
+      GrpcFnServer<GrpcLoggingService> loggingServiceServer,
+      GrpcFnServer<ArtifactRetrievalService> retrievalServiceServer,
+      GrpcFnServer<StaticGrpcProvisionService> provisioningServiceServer,
+      ControlClientPool.Source clientSource,
+      IdGenerator idGenerator) {
+    return new ExternalEnvironmentFactory(
+        controlServiceServer,
+        loggingServiceServer,
+        retrievalServiceServer,
+        provisioningServiceServer,
+        idGenerator,
+        clientSource);
+  }
+
+  private final GrpcFnServer<FnApiControlClientPoolService> 
controlServiceServer;
+  private final GrpcFnServer<GrpcLoggingService> loggingServiceServer;
+  private final GrpcFnServer<ArtifactRetrievalService> retrievalServiceServer;
+  private final GrpcFnServer<StaticGrpcProvisionService> 
provisioningServiceServer;
+  private final IdGenerator idGenerator;
+  private final ControlClientPool.Source clientSource;
+
+  private ExternalEnvironmentFactory(
+      GrpcFnServer<FnApiControlClientPoolService> controlServiceServer,
+      GrpcFnServer<GrpcLoggingService> loggingServiceServer,
+      GrpcFnServer<ArtifactRetrievalService> retrievalServiceServer,
+      GrpcFnServer<StaticGrpcProvisionService> provisioningServiceServer,
+      IdGenerator idGenerator,
+      ControlClientPool.Source clientSource) {
+    this.controlServiceServer = controlServiceServer;
+    this.loggingServiceServer = loggingServiceServer;
+    this.retrievalServiceServer = retrievalServiceServer;
+    this.provisioningServiceServer = provisioningServiceServer;
+    this.idGenerator = idGenerator;
+    this.clientSource = clientSource;
+  }
+
+  /** Creates a new, active {@link RemoteEnvironment} backed by a forked 
process. */
+  @Override
+  public RemoteEnvironment createEnvironment(Environment environment) throws 
Exception {
+    Preconditions.checkState(
+        environment
+            .getUrn()
+            
.equals(BeamUrns.getUrn(RunnerApi.StandardEnvironments.Environments.EXTERNAL)),
+        "The passed environment does not contain an ExternalPayload.");
+    final RunnerApi.ExternalPayload externalPayload =
+        RunnerApi.ExternalPayload.parseFrom(environment.getPayload());
+    final String workerId = idGenerator.getId();
+
+    BeamFnApi.StartWorkerRequest startWorkRequest =
+        BeamFnApi.StartWorkerRequest.newBuilder()
+            .setWorkerId(workerId)
+            .setControlEndpoint(controlServiceServer.getApiServiceDescriptor())
+            .setLoggingEndpoint(loggingServiceServer.getApiServiceDescriptor())
+            
.setArtifactEndpoint(retrievalServiceServer.getApiServiceDescriptor())
+            
.setProvisionEndpoint(provisioningServiceServer.getApiServiceDescriptor())
+            .putAllParams(externalPayload.getParamsMap())
+            .build();
+
+    LOG.debug("Requesting worker ID {}", workerId);
+    BeamFnApi.StartWorkerResponse startWorkResponse =
+        BeamFnExternalWorkerGrpc.newBlockingStub(
+                
ManagedChannelFactory.createDefault().forDescriptor(externalPayload.getEndpoint()))
+            .startWorker(startWorkRequest);
+    if (!startWorkResponse.getError().isEmpty()) {
+      throw new RuntimeException(startWorkResponse.getError());
+    }
+
+    // Wait on a client from the gRPC server.
+    InstructionRequestHandler instructionHandler = null;
+    while (instructionHandler == null) {
+      try {
+        instructionHandler = clientSource.take(workerId, 
Duration.ofMinutes(2));
+      } catch (TimeoutException timeoutEx) {
+        LOG.info(
+            "Still waiting for startup of environment from {} for worker id 
{}",
+            externalPayload.getEndpoint().getUrl(),
+            workerId);
+      } catch (InterruptedException interruptEx) {
+        Thread.currentThread().interrupt();
+        throw new RuntimeException(interruptEx);
+      }
+    }
+    final InstructionRequestHandler finalInstructionHandler = 
instructionHandler;
+
+    return new RemoteEnvironment() {
+      @Override
+      public Environment getEnvironment() {
+        return environment;
+      }
+
+      @Override
+      public InstructionRequestHandler getInstructionRequestHandler() {
+        return finalInstructionHandler;
+      }
+    };
+  }
+
+  /** Provider of ProcessEnvironmentFactory. */
 
 Review comment:
   Oops. If it weren't for satisfying linkers, obvious comments like these seem 
to have negative value... Fixed.

----------------------------------------------------------------
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: 172251)

> Implement External environment for Portable Beam
> ------------------------------------------------
>
>                 Key: BEAM-6094
>                 URL: https://issues.apache.org/jira/browse/BEAM-6094
>             Project: Beam
>          Issue Type: Improvement
>          Components: beam-model
>            Reporter: Robert Bradshaw
>            Assignee: Robert Bradshaw
>            Priority: Major
>          Time Spent: 1h 50m
>  Remaining Estimate: 0h
>




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

Reply via email to