[ https://issues.apache.org/jira/browse/FLINK-7674?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16178036#comment-16178036 ]
ASF GitHub Bot commented on FLINK-7674: --------------------------------------- GitHub user bowenli86 opened a pull request: https://github.com/apache/flink/pull/4711 [FLINK-7674][Streaming Connectors] NullPointerException in ContinuousFileMonitoringFunction close ## What is the purpose of the change If the ContinuousFileMonitoringFunction is closed before run is called (because initialization fails), we get a `NullPointerException`, because checkpointLock has not been set. ```java synchronized (checkpointLock) { globalModificationTime = Long.MAX_VALUE; isRunning = false; } ``` We need to add a null check that wraps the synchronized clause ## Brief change log Add a null check that wraps the synchronized clause, as: ```java if (checkpointLock != null) { synchronized (checkpointLock) { globalModificationTime = Long.MAX_VALUE; isRunning = false; } } ``` ## Verifying this change This change is a trivial rework / code cleanup without any test coverage. ## Does this pull request potentially affect one of the following parts: none ## Documentation none You can merge this pull request into a Git repository by running: $ git pull https://github.com/bowenli86/flink FLINK-7674 Alternatively you can review and apply these changes as the patch at: https://github.com/apache/flink/pull/4711.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #4711 ---- commit 9a5cb031bbff33d4f9ba956656e93e31a179e309 Author: Bowen Li <bowenl...@gmail.com> Date: 2017-09-24T01:24:56Z FLINK-7674 NullPointerException in ContinuousFileMonitoringFunction close ---- > NullPointerException in ContinuousFileMonitoringFunction close > -------------------------------------------------------------- > > Key: FLINK-7674 > URL: https://issues.apache.org/jira/browse/FLINK-7674 > Project: Flink > Issue Type: Bug > Components: Streaming Connectors > Affects Versions: 1.4.0 > Reporter: Ufuk Celebi > Assignee: Bowen Li > Priority: Minor > > If the ContinuousFileMonitoringFunction is closed before run is called > (because initialization fails), we get a NullPointerException, because > checkpointLock has not been set. > {code} > synchronized (checkpointLock) { > globalModificationTime = Long.MAX_VALUE; > isRunning = false; > } > {code} > This results in a follow up error log: > {code} > 2017-09-23 10:25:04,096 ERROR > org.apache.flink.streaming.runtime.tasks.StreamTask - Error during > disposal of stream operator. > java.lang.NullPointerException > at > org.apache.flink.streaming.api.functions.source.ContinuousFileMonitoringFunction.close(ContinuousFileMonitoringFunction.java:337) > at > org.apache.flink.api.common.functions.util.FunctionUtils.closeFunction(FunctionUtils.java:43) > at > org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator.dispose(AbstractUdfStreamOperator.java:126) > at > org.apache.flink.streaming.runtime.tasks.StreamTask.disposeAllOperators(StreamTask.java:429) > at > org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:334) > at org.apache.flink.runtime.taskmanager.Task.run(Task.java:702) > at java.lang.Thread.run(Thread.java:748) > {code} -- This message was sent by Atlassian JIRA (v6.4.14#64029)