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

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_r195012204
  
    --- Diff: 
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java
 ---
    @@ -1239,6 +1286,50 @@ protected void cancelTask() throws Exception {
     
        }
     
    +   /**
    +    * A task that register a processing time service callback.
    +    */
    +   public static class TimeServiceTask extends StreamTask<String, 
AbstractStreamOperator<String>> {
    +
    +           public TimeServiceTask(Environment env) {
    +                   super(env, null);
    +           }
    +
    +           @Override
    +           protected void init() throws Exception {
    +                   getProcessingTimeService().registerTimer(1000, new 
ProcessingTimeCallback() {
    --- End diff --
    
    Please change `1000` to `0`. This parameter is not a delay in 
miliseconds/seconds from now, but specifies exact timestamp when the timer 
should fire, thus value `1000` is misleading.


> 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