mynameborat commented on a change in pull request #14554:
URL: https://github.com/apache/beam/pull/14554#discussion_r616350651



##########
File path: 
runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaExecutionContext.java
##########
@@ -81,93 +50,11 @@ void setMetricsContainer(SamzaMetricsContainer 
metricsContainer) {
     this.metricsContainer = metricsContainer;
   }
 
-  public JobBundleFactory getJobBundleFactory() {
-    return this.jobBundleFactory;
-  }
-
-  void setJobBundleFactory(JobBundleFactory jobBundleFactory) {
-    this.jobBundleFactory = jobBundleFactory;
-  }
-
   @Override
-  public void start() {
-    checkState(getJobBundleFactory() == null, "jobBundleFactory has been 
created!");
-
-    if (SamzaRunnerOverrideConfigs.isPortableMode(options)) {
-      try {
-        controlClientPool = MapControlClientPool.create();
-        dataExecutor = Executors.newCachedThreadPool();
-
-        fnControlServer =
-            GrpcFnServer.allocatePortAndCreateFor(
-                FnApiControlClientPoolService.offeringClientsToPool(
-                    controlClientPool.getSink(), () -> SAMZA_WORKER_ID),
-                ServerFactory.createWithPortSupplier(
-                    () -> 
SamzaRunnerOverrideConfigs.getFnControlPort(options)));
-        LOG.info("Started control server on port {}", 
fnControlServer.getServer().getPort());
-
-        fnDataServer =
-            GrpcFnServer.allocatePortAndCreateFor(
-                GrpcDataService.create(
-                    options, dataExecutor, 
OutboundObserverFactory.serverDirect()),
-                ServerFactory.createDefault());
-        LOG.info("Started data server on port {}", 
fnDataServer.getServer().getPort());
-
-        fnStateServer =
-            GrpcFnServer.allocatePortAndCreateFor(
-                GrpcStateService.create(), ServerFactory.createDefault());
-        LOG.info("Started state server on port {}", 
fnStateServer.getServer().getPort());
-
-        final long waitTimeoutMs =
-            SamzaRunnerOverrideConfigs.getControlClientWaitTimeoutMs(options);
-        LOG.info("Control client wait timeout config: " + waitTimeoutMs);
-
-        final InstructionRequestHandler instructionHandler =
-            controlClientPool.getSource().take(SAMZA_WORKER_ID, 
Duration.ofMillis(waitTimeoutMs));
-        final EnvironmentFactory environmentFactory =
-            (environment, workerId) ->
-                RemoteEnvironment.forHandler(environment, instructionHandler);
-        // TODO: use JobBundleFactoryBase.WrappedSdkHarnessClient.wrapping
-        jobBundleFactory =
-            SingleEnvironmentInstanceJobBundleFactory.create(
-                environmentFactory, fnDataServer, fnStateServer, idGenerator);
-        LOG.info("Started job bundle factory");
-      } catch (Exception e) {
-        throw new RuntimeException(
-            "Running samza in Beam portable mode but failed to create job 
bundle factory", e);
-      }
-
-      setJobBundleFactory(jobBundleFactory);
-    }
-  }
+  public void start() {}

Review comment:
       Seems like this isn't doing much post your changes? Can we get rid of it 
instead? 

##########
File path: 
runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaExecutableStageContextFactory.java
##########
@@ -0,0 +1,61 @@
+/*
+ * 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.samza.runtime;
+
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import 
org.apache.beam.runners.fnexecution.control.DefaultExecutableStageContext;
+import org.apache.beam.runners.fnexecution.control.ExecutableStageContext;
+import 
org.apache.beam.runners.fnexecution.control.ReferenceCountingExecutableStageContextFactory;
+import org.apache.beam.runners.fnexecution.provisioning.JobInfo;
+
+/**
+ * Singleton class that contains one {@link ExecutableStageContext.Factory} 
per job. Assumes it is
+ * safe to release the backing environment asynchronously.
+ */
+public class SamzaExecutableStageContextFactory implements 
ExecutableStageContext.Factory {
+
+  private static final SamzaExecutableStageContextFactory instance =
+      new SamzaExecutableStageContextFactory();
+  // This map should only ever have a single element, as each job will have 
its own
+  // classloader and therefore its own instance of 
FlinkExecutableStageContextFactory. This
+  // code supports multiple JobInfos in order to provide a sensible 
implementation of
+  // Factory.get(JobInfo), which in theory could be called with different 
JobInfos.

Review comment:
       fix the copy paste comment

##########
File path: 
runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaJobInvoker.java
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.samza;
+
+import java.util.UUID;
+import javax.annotation.Nullable;
+import org.apache.beam.model.pipeline.v1.RunnerApi;
+import org.apache.beam.runners.core.construction.PipelineOptionsTranslation;
+import org.apache.beam.runners.fnexecution.provisioning.JobInfo;
+import org.apache.beam.runners.jobsubmission.JobInvocation;
+import org.apache.beam.runners.jobsubmission.JobInvoker;
+import org.apache.beam.runners.jobsubmission.PortablePipelineJarCreator;
+import org.apache.beam.runners.jobsubmission.PortablePipelineRunner;
+import org.apache.beam.sdk.options.PortablePipelineOptions;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.Struct;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Strings;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.ListeningExecutorService;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({
+  "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
+})
+public class SamzaJobInvoker extends JobInvoker {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(SamzaJobInvoker.class);
+  private final SamzaJobServerDriver.SamzaServerConfiguration configuration;
+
+  public static SamzaJobInvoker create(
+      SamzaJobServerDriver.SamzaServerConfiguration configuration) {
+    return new SamzaJobInvoker(configuration);
+  }
+
+  private SamzaJobInvoker(SamzaJobServerDriver.SamzaServerConfiguration 
configuration) {
+    super("samza-runner-job-invoker-%d");
+    this.configuration = configuration;
+  }
+
+  @Override
+  protected JobInvocation invokeWithExecutor(
+      RunnerApi.Pipeline pipeline,
+      Struct options,
+      @Nullable String retrievalToken,
+      ListeningExecutorService executorService) {
+    LOG.trace("Parsing pipeline options");
+    final SamzaPortablePipelineOptions samzaOptions =
+        
PipelineOptionsTranslation.fromProto(options).as(SamzaPortablePipelineOptions.class);
+    // Options can't be translated to proto if runner class is unresolvable, 
so set it to null.
+    samzaOptions.setRunner(null);

Review comment:
       Looks like you don't need this? In the older way, we had o ignore the 
`options` passed to the executor and used the `pipelineOptions` from the 
`SamzaJobServerDriver`. Hence it was required for us to convert it to proto 
again to ensure `pipelineOptions` is the one that got used.
   
   With your change, this is no longer needed. Given, we don't mutate the 
options passed down, you should be fine with passing `options` to the `JobInfo` 
creation code.

##########
File path: 
runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/DoFnOp.java
##########
@@ -217,7 +222,9 @@ public void open(
               .stateInternalsForKey(null)
               .state(StateNamespaces.global(), StateTags.bag(bundleStateId, 
windowedValueCoder));
       final ExecutableStage executableStage = 
ExecutableStage.fromPayload(stagePayload);
-      stageBundleFactory = 
samzaExecutionContext.getJobBundleFactory().forStage(executableStage);
+      final ExecutableStageContext stageContext =
+          SamzaExecutableStageContextFactory.getInstance().get(jobInfo);
+      stageBundleFactory = stageContext.getStageBundleFactory(executableStage);

Review comment:
       According to the deprecated notes
   ```
    * @deprecated replace with a {@link DefaultJobBundleFactory} when 
appropriate if the {@link
    *     EnvironmentFactory} is a {@link
    *     
org.apache.beam.runners.fnexecution.environment.DockerEnvironmentFactory}, or 
create an
    *     {@code InProcessJobBundleFactory} and inline the creation of the 
environment if appropriate.
    */
    ```
    I don't think the environment is docker always. Does this still work for 
other environments?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to