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

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

                Author: ASF GitHub Bot
            Created on: 15/Oct/18 16:32
            Start Date: 15/Oct/18 16:32
    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_r225233894
 
 

 ##########
 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:
   Wordcount in batch mode was a negative test (with fat jar): Since 
environment cannot be cached because the class loader isn't the same, it is 
expected to fail. The test that I did not perform was to add the jar to the 
Flink classpath (the objective was to optimize embedded execution for VR tests).

----------------------------------------------------------------
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: 154361)
    Time Spent: 3h 10m  (was: 3h)

> 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: 3h 10m
>  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