You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "Dawid Wysakowicz (Jira)" <ji...@apache.org> on 2020/04/07 11:51:00 UTC
[jira] [Commented] (FLINK-16913)
ReadableConfigToConfigurationAdapter#getEnum throws
UnsupportedOperationException
[ https://issues.apache.org/jira/browse/FLINK-16913?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17077155#comment-17077155 ]
Dawid Wysakowicz commented on FLINK-16913:
------------------------------------------
Hi [~felixzheng] thank you for creating the issue. The reason for introducing the {{ReadableConfigToConfigurationAdapter}} was to support configuring state backends from flink-conf.yaml on the client side without changing interfaces such as {{StateBackendFactory}}. I wanted to implement the minimal set of getters to support all the {{StateBackendFactories}}. It turned out I missed some of the options. Moreover I agree with you that not supporting all the getters is fragile.
In master branch I removed the class and changed all the affected classes to use the {{ReadableConfig}} instead. In 1.10.1 I took a slightly different approach not to change any interfaces in a minor release. In 1.10.1 I implemented the affected getters.
> ReadableConfigToConfigurationAdapter#getEnum throws UnsupportedOperationException
> ---------------------------------------------------------------------------------
>
> Key: FLINK-16913
> URL: https://issues.apache.org/jira/browse/FLINK-16913
> Project: Flink
> Issue Type: Bug
> Components: Runtime / Configuration
> Affects Versions: 1.10.0
> Reporter: Canbin Zheng
> Assignee: Dawid Wysakowicz
> Priority: Blocker
> Labels: pull-request-available
> Fix For: 1.10.1, 1.11.0
>
> Attachments: image-2020-04-01-16-46-13-122.png
>
> Time Spent: 40m
> Remaining Estimate: 0h
>
> Steps to reproduce the issue:
> # Set flink-conf.yaml
> ** state.backend: rocksdb
> ** state.checkpoints.dir: hdfs:///flink-checkpoints
> ** state.savepoints.dir: hdfs:///flink-checkpoints
> # Start a Kubernetes session cluster
> # Submit a job to the session cluster, unfortunately a UnsupportedOperationException occurs.
> {code:java}
> The program finished with the following exception:org.apache.flink.client.program.ProgramInvocationException: The main method caused an error: The adapter does not support this method
> at org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:335)
> at org.apache.flink.client.program.PackagedProgram.invokeInteractiveModeForExecution(PackagedProgram.java:205)
> at org.apache.flink.client.ClientUtils.executeProgram(ClientUtils.java:143)
> at org.apache.flink.client.cli.CliFrontend.executeProgram(CliFrontend.java:659)
> at org.apache.flink.client.cli.CliFrontend.run(CliFrontend.java:210)
> at org.apache.flink.client.cli.CliFrontend.parseParameters(CliFrontend.java:890)
> at org.apache.flink.client.cli.CliFrontend.lambda$main$10(CliFrontend.java:963)
> at org.apache.flink.runtime.security.contexts.NoOpSecurityContext.runSecured(NoOpSecurityContext.java:30)
> at org.apache.flink.client.cli.CliFrontend.main(CliFrontend.java:963)
> Caused by: java.lang.UnsupportedOperationException: The adapter does not support this method
> at org.apache.flink.configuration.ReadableConfigToConfigurationAdapter.getEnum(ReadableConfigToConfigurationAdapter.java:258)
> at org.apache.flink.contrib.streaming.state.RocksDBStateBackend.<init>(RocksDBStateBackend.java:336)
> at org.apache.flink.contrib.streaming.state.RocksDBStateBackend.configure(RocksDBStateBackend.java:394)
> at org.apache.flink.contrib.streaming.state.RocksDBStateBackendFactory.createFromConfig(RocksDBStateBackendFactory.java:47)
> at org.apache.flink.contrib.streaming.state.RocksDBStateBackendFactory.createFromConfig(RocksDBStateBackendFactory.java:32)
> at org.apache.flink.runtime.state.StateBackendLoader.loadStateBackendFromConfig(StateBackendLoader.java:154)
> at org.apache.flink.streaming.api.environment.StreamExecutionEnvironment.loadStateBackend(StreamExecutionEnvironment.java:792)
> at org.apache.flink.streaming.api.environment.StreamExecutionEnvironment.configure(StreamExecutionEnvironment.java:761)
> at org.apache.flink.streaming.api.environment.StreamExecutionEnvironment.<init>(StreamExecutionEnvironment.java:217)
> at org.apache.flink.client.program.StreamContextEnvironment.<init>(StreamContextEnvironment.java:53)
> at org.apache.flink.client.program.StreamContextEnvironment.lambda$setAsContext$2(StreamContextEnvironment.java:103)
> at java.util.Optional.map(Optional.java:215)
> at org.apache.flink.streaming.api.environment.StreamExecutionEnvironment.getExecutionEnvironment(StreamExecutionEnvironment.java:1882)
> at org.apache.flink.streaming.examples.socket.SocketWindowWordCount.main(SocketWindowWordCount.java:62)
> at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
> at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> at java.lang.reflect.Method.invoke(Method.java:498)
> at org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:321)
> ... 8 more
> {code}
> I am wondering why we introduceĀ {{ReadableConfigToConfigurationAdapter}} to wrap theĀ {{Configuration}} but leave many of the getter methods in it to throw UnsupportedOperationException that causes potential problems.
--
This message was sent by Atlassian Jira
(v8.3.4#803005)