Details
-
Bug
-
Status: Reopened
-
Major
-
Resolution: Unresolved
-
1.13.1
-
None
-
None
-
Flink Version :1.13.1
Standalone Cluster
Description
The directory where savepoint is configured in my application
String savepointDir; if (Strings.isNotBlank(savepointDir = parameterTool.get("system.savepoint.dir"))) { env.setDefaultSavepointDirectory(savepointDir); }
Where `system.savepoint.dir` is not empty.
When I submit an application to my standalone cluster, an error is prompted:
2021-07-27 16:26:34.029 [travel_spilt -> (Sink: sink_es_insert, Sink: sink_es_update) (1/1)#31] WARN org.apache.flink.runtime.taskmanager.Task - travel_spilt -> (Sink: sink_es_insert, Sink: sink_es_update) (1/1)#31 (1f33925046e04307ba6beb3959d2cad5) switched from DEPLOYING to FAILED with failure cause: java.lang.UnsupportedOperationException: The configuration is unmodifiable; its contents cannot be changed.2021-07-27 16:26:34.029 [travel_spilt -> (Sink: sink_es_insert, Sink: sink_es_update) (1/1)#31] WARN org.apache.flink.runtime.taskmanager.Task - travel_spilt -> (Sink: sink_es_insert, Sink: sink_es_update) (1/1)#31 (1f33925046e04307ba6beb3959d2cad5) switched from DEPLOYING to FAILED with failure cause: java.lang.UnsupportedOperationException: The configuration is unmodifiable; its contents cannot be changed. at org.apache.flink.configuration.UnmodifiableConfiguration.error(UnmodifiableConfiguration.java:73) at org.apache.flink.configuration.UnmodifiableConfiguration.setValueInternal(UnmodifiableConfiguration.java:63) at org.apache.flink.configuration.Configuration.set(Configuration.java:709) at org.apache.flink.runtime.state.CheckpointStorageLoader.load(CheckpointStorageLoader.java:174) at org.apache.flink.streaming.runtime.tasks.StreamTask.createCheckpointStorage(StreamTask.java:1231) at org.apache.flink.streaming.runtime.tasks.StreamTask.<init>(StreamTask.java:337) at org.apache.flink.streaming.runtime.tasks.StreamTask.<init>(StreamTask.java:308) at org.apache.flink.streaming.runtime.tasks.StreamTask.<init>(StreamTask.java:281) at org.apache.flink.streaming.runtime.tasks.StreamTask.<init>(StreamTask.java:273) at org.apache.flink.streaming.runtime.tasks.StreamTask.<init>(StreamTask.java:263) at org.apache.flink.streaming.runtime.tasks.OneInputStreamTask.<init>(OneInputStreamTask.java:65) at sun.reflect.GeneratedConstructorAccessor339.newInstance(Unknown Source) at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45) at java.lang.reflect.Constructor.newInstance(Constructor.java:423) at org.apache.flink.runtime.taskmanager.Task.loadAndInstantiateInvokable(Task.java:1524) at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:730) at org.apache.flink.runtime.taskmanager.Task.run(Task.java:566) at java.lang.Thread.run(Thread.java:748)
Every stateful operator has this error.What do I need to do? I looked at other versions and didn't find the method Unmodifiable configuration#setvalueinternal (string key, t value).
Looking forward to your reply, thank you.
Attachments
Issue Links
- duplicates
-
FLINK-23516 I try to specify savepoint dir in my app. However, the startup task reports an error, java.lang.unsupported operationexception: the configuration is unmodifiable; its contents cannot be changed.
- Closed
- is caused by
-
FLINK-19466 Implement JobManagerCheckpointStorage and FileSystemCheckpointStorage
- Closed
- is duplicated by
-
FLINK-23516 I try to specify savepoint dir in my app. However, the startup task reports an error, java.lang.unsupported operationexception: the configuration is unmodifiable; its contents cannot be changed.
- Closed