[ 
https://issues.apache.org/jira/browse/FLINK-8067?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16511283#comment-16511283
 ] 

ASF GitHub Bot commented on FLINK-8067:
---------------------------------------

Github user pnowojski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/6081#discussion_r195123173
  
    --- Diff: 
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java
 ---
    @@ -806,6 +811,44 @@ public void testOperatorClosingBeforeStopRunning() 
throws Throwable {
                }
        }
     
    +   /**
    +    * Test set user code ClassLoader before calling ProcessingTimeCallback.
    +    */
    +   @Test
    +   public void testSetsUserCodeClassLoaderForTimerThreadFactory() throws 
Throwable {
    +           syncLatch = new OneShotLatch();
    +
    +           try (MockEnvironment mockEnvironment =
    +                   new MockEnvironmentBuilder()
    +                           .setUserCodeClassLoader(new 
TestUserCodeClassLoader())
    +                           .build()) {
    +                   TimeServiceTask timerServiceTask = new 
TimeServiceTask(mockEnvironment);
    +
    +                   final AtomicReference<Throwable> atomicThrowable = new 
AtomicReference<>(null);
    +
    +                   CompletableFuture<Void> invokeFuture = 
CompletableFuture.runAsync(
    +                           () -> {
    +                                   try {
    +                                           timerServiceTask.invoke();
    +                                   } catch (Exception e) {
    +                                           atomicThrowable.set(e);
    +                                   }
    +                           },
    +                           TestingUtils.defaultExecutor());
    +
    +                   // wait until the invoke is complete
    +                   invokeFuture.get();
    +
    +                   assertThat(timerServiceTask.getClassLoaders(), 
hasSize(greaterThanOrEqualTo(1)));
    +                   assertThat(timerServiceTask.getClassLoaders(), 
everyItem(instanceOf(TestUserCodeClassLoader.class)));
    +
    +                   // check if an exception occurred
    --- End diff --
    
    tiny nits: you could drop this (and the one above) comment. The code is 
pretty self explanatory on it's own.


> User code ClassLoader not set before calling ProcessingTimeCallback
> -------------------------------------------------------------------
>
>                 Key: FLINK-8067
>                 URL: https://issues.apache.org/jira/browse/FLINK-8067
>             Project: Flink
>          Issue Type: Bug
>          Components: Streaming
>    Affects Versions: 1.4.0
>            Reporter: Gary Yao
>            Assignee: vinoyang
>            Priority: Minor
>             Fix For: 1.6.0, 1.5.1
>
>
> The user code ClassLoader is not set as the context ClassLoader for the 
> thread invoking {{ProcessingTimeCallback#onProcessingTime(long timestamp)}}:
> https://github.com/apache/flink/blob/84a07a34ac22af14f2dd0319447ca5f45de6d0bb/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java#L222
> This is problematic, for example, if user code dynamically loads classes in 
> {{ProcessFunction#onTimer(long timestamp, OnTimerContext ctx, Collector<O> 
> out)}} using the current thread's context ClassLoader (also see FLINK-8005).



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

Reply via email to