Matthias Pohl created FLINK-31093:
-------------------------------------

             Summary: NullpointerException when restoring a FlinkSQL job from a 
savepoint
                 Key: FLINK-31093
                 URL: https://issues.apache.org/jira/browse/FLINK-31093
             Project: Flink
          Issue Type: Bug
          Components: Table SQL / Runtime
    Affects Versions: 1.17.0
            Reporter: Matthias Pohl


I tried to restore a FlinkSQL job from a savepoint and ran into a 
{{NullPointerException}}:
{code}
2023-02-15 16:38:24,835 INFO  org.apache.flink.runtime.jobmaster.JobMaster      
           [] - Initializing job 'collect' (0263d02536654102f2aa903f843cacd1).
2023-02-15 16:38:24,858 INFO  
org.apache.flink.runtime.dispatcher.StandaloneDispatcher     [] - Job 
0263d02536654102f2aa903f843cacd1 reached terminal state FAILED.
org.apache.flink.runtime.client.JobInitializationException: Could not start the 
JobMaster.
        at 
org.apache.flink.runtime.jobmaster.DefaultJobMasterServiceProcess.lambda$new$0(DefaultJobMasterServiceProcess.java:97)
        at 
java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:774)
        at 
java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:750)
        at 
java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:488)
        at 
java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1609)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:750)
Caused by: java.util.concurrent.CompletionException: 
java.lang.NullPointerException
        at 
java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:273)
        at 
java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:280)
        at 
java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1606)
        ... 3 more
Caused by: java.lang.NullPointerException
        at 
org.apache.flink.api.common.ExecutionConfig.getNumberOfExecutionRetries(ExecutionConfig.java:486)
        at 
org.apache.flink.api.common.ExecutionConfig.getRestartStrategy(ExecutionConfig.java:459)
        at 
org.apache.flink.runtime.scheduler.DefaultSchedulerFactory.createInstance(DefaultSchedulerFactory.java:99)
        at 
org.apache.flink.runtime.jobmaster.DefaultSlotPoolServiceSchedulerFactory.createScheduler(DefaultSlotPoolServiceSchedulerFactory.java:119)
        at 
org.apache.flink.runtime.jobmaster.JobMaster.createScheduler(JobMaster.java:371)
        at 
org.apache.flink.runtime.jobmaster.JobMaster.<init>(JobMaster.java:348)
        at 
org.apache.flink.runtime.jobmaster.factories.DefaultJobMasterServiceFactory.internalCreateJobMasterService(DefaultJobMasterServiceFactory.java:123)
        at 
org.apache.flink.runtime.jobmaster.factories.DefaultJobMasterServiceFactory.lambda$createJobMasterService$0(DefaultJobMasterServiceFactory.java:95)
        at 
org.apache.flink.util.function.FunctionUtils.lambda$uncheckedSupplier$4(FunctionUtils.java:112)
        at 
java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1604)
        ... 3 more
{code}

The SQL job was submitted through the SQL client:
{code}
$ -- table created in Flink 1.16.1
$ CREATE TABLE MyTable (
>    a bigint,
>    b int not null,
>    c varchar,
>    d timestamp(3)
> ) with ('connector' = 'datagen', 'rows-per-second' = '1', 'fields.a.kind' = 
> 'sequence', 'fields.a.start' = '0', 'fields.a.end' = '1000000');
$ -- SELECT statement ran in Flink 1.16.1 session cluster
$ SELECT a FROM MyTable WHERE a = 1 or a = 2 or a IS NOT NULL;
{code}
The job was stopped with a savepoint from the command line:
{code}
$ ./bin/flink stop --type native --savepointPath ../1.16.1-savepoint 
6029e8e5632a9852c630b1b0e4b62477
{code}
A new 1.17-SNAPSHOT (commit: {{21158c06}}) session cluster was started and the 
following SQL code was executed from within the SQL client:
{code}
$ SET 'execution.savepoint.path' = 
'/home/mapohl/research/FLINK-31066/1.16.1-savepoint/savepoint-6029e8-ef1e50f0dd2e';
$ SELECT a FROM MyTable WHERE a = 1 or a = 2 or a IS NOT NULL;
[ERROR] Could not execute SQL statement. Reason:
java.util.concurrent.CompletionException: java.lang.NullPointerException
{code}

This caused the {{NullPointerException}} with the aforementioned stacktrace.

The error is caused by 
[ExecutionConfig:486|https://github.com/apache/flink/blob/143464d82814e342aa845f3ac976ae2854fc892f/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java#L486].
 The line can only cause a {{NullPointerException}} if the corresponding 
configuration is not set. This only happens if the {{ExecutionConfig}} is 
deserialized but the {{configuration}} field is not deserialized which leaves 
the field to be {{null}} initialized.

This field is not set to {{null}} in any other way.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to