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

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

                Author: ASF GitHub Bot
            Created on: 16/Oct/18 00:21
            Start Date: 16/Oct/18 00:21
    Worklog Time Spent: 10m 
      Work Description: tweise commented on a change in pull request #6638: 
[BEAM-5708] Cache environment in portable flink runner
URL: https://github.com/apache/beam/pull/6638#discussion_r225358322
 
 

 ##########
 File path: 
runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/ReferenceCountingFlinkExecutableStageContextFactory.java
 ##########
 @@ -104,9 +109,30 @@ private void scheduleRelease(JobInfo jobInfo) {
     WrappedContext wrapper = getCache().get(jobInfo.jobId());
     Preconditions.checkState(
         wrapper != null, "Releasing context for unknown job: " + 
jobInfo.jobId());
-    // Do not release this asynchronously, as the releasing could fail due to 
the classloader not being
-    // available anymore after the tasks have been removed from the execution 
engine.
-    release(wrapper);
+
+    PipelineOptions pipelineOptions =
+        PipelineOptionsTranslation.fromProto(jobInfo.pipelineOptions());
+    int environmentCacheTTLMillis =
+        
pipelineOptions.as(PortablePipelineOptions.class).getEnvironmentCacheMillis();
+    if (environmentCacheTTLMillis > 0) {
+      if (this.getClass().getClassLoader() != 
ExecutionEnvironment.class.getClassLoader()) {
 
 Review comment:
   It behaves as expected with embedded mode in the job server and with default 
class loading behavior on the remote cluster (i.e. there won't be caching). The 
remaining test would be the augmented class loading described on 
https://beam.apache.org/contribute/portability/#status - which should also 
work, since it is parent first. 

----------------------------------------------------------------
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: 154541)
    Time Spent: 4h  (was: 3h 50m)

> Support caching of SDKHarness environments in flink
> ---------------------------------------------------
>
>                 Key: BEAM-5708
>                 URL: https://issues.apache.org/jira/browse/BEAM-5708
>             Project: Beam
>          Issue Type: Improvement
>          Components: runner-flink
>            Reporter: Ankur Goenka
>            Assignee: Ankur Goenka
>            Priority: Major
>          Time Spent: 4h
>  Remaining Estimate: 0h
>
> Cache and reuse environment to improve performance.



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

Reply via email to