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

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

                Author: ASF GitHub Bot
            Created on: 09/Sep/19 23:21
            Start Date: 09/Sep/19 23:21
    Worklog Time Spent: 10m 
      Work Description: ibzib commented on pull request #9095: [BEAM-7600] 
borrow SDK harness management code into Spark runner
URL: https://github.com/apache/beam/pull/9095#discussion_r322494298
 
 

 ##########
 File path: 
runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkExecutableStageContextFactory.java
 ##########
 @@ -0,0 +1,69 @@
+/*
+ * 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.spark.translation;
+
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import org.apache.beam.runners.core.construction.PipelineOptionsTranslation;
+import 
org.apache.beam.runners.fnexecution.control.DefaultExecutableStageContext.MultiInstanceFactory;
+import org.apache.beam.runners.fnexecution.control.ExecutableStageContext;
+import org.apache.beam.runners.fnexecution.provisioning.JobInfo;
+import org.apache.beam.sdk.options.PortablePipelineOptions;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.MoreObjects;
+
+/**
+ * Singleton class that contains one {@link MultiInstanceFactory} per job. 
Assumes it is safe to
+ * release the backing environment asynchronously.
+ */
+public class SparkExecutableStageContextFactory implements 
ExecutableStageContext.Factory {
+
+  private static final SparkExecutableStageContextFactory instance =
+      new SparkExecutableStageContextFactory();
+  // This map should only ever have a single element, as each job will have 
its own
+  // classloader and therefore its own instance of 
SparkExecutableStageContextFactory. 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.
+  private static final ConcurrentMap<String, MultiInstanceFactory> 
jobFactories =
+      new ConcurrentHashMap<>();
+
+  private SparkExecutableStageContextFactory() {}
+
+  public static SparkExecutableStageContextFactory getInstance() {
+    return instance;
+  }
+
+  @Override
+  public ExecutableStageContext get(JobInfo jobInfo) {
+    MultiInstanceFactory state =
 
 Review comment:
   done
 
----------------------------------------------------------------
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]


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

    Worklog Id:     (was: 309356)
    Time Spent: 8h 20m  (was: 8h 10m)

> Spark portable runner: reuse SDK harness
> ----------------------------------------
>
>                 Key: BEAM-7600
>                 URL: https://issues.apache.org/jira/browse/BEAM-7600
>             Project: Beam
>          Issue Type: Improvement
>          Components: runner-spark
>            Reporter: Kyle Weaver
>            Assignee: Kyle Weaver
>            Priority: Major
>              Labels: portability-spark
>          Time Spent: 8h 20m
>  Remaining Estimate: 0h
>
> Right now, we're creating a new SDK harness every time an executable stage is 
> run [1], which is expensive. We should be able to re-use code from the Flink 
> runner to re-use the SDK harness [2].
>  
> [1] 
> [https://github.com/apache/beam/blob/c9fb261bc7666788402840bb6ce1b0ce2fd445d1/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkExecutableStageFunction.java#L135]
> [2] 
> [https://github.com/apache/beam/blob/c9fb261bc7666788402840bb6ce1b0ce2fd445d1/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDefaultExecutableStageContext.java]



--
This message was sent by Atlassian Jira
(v8.3.2#803003)

Reply via email to