Re: [DISCUSS] FLIP-381: Deprecate configuration getters/setters that return/set complex Java objects
Hi Junrui, Thanks for updating the FLIP! It looks good and clear now. Appreciate it! Best regards, Jing On Sun, Nov 5, 2023 at 7:35 PM Junrui Lee wrote: > Hi Jing, > > Thank you for your feedback on the proposal. I appreciate your input and > understand your concerns. > > Regarding your questions, the complex Java class that will be deprecated is > org.apache.flink.api.common.restartstrategy.RestartStrategies and and its > internal classes, which include: > > - RestartStrategyConfiguration > - FallbackRestartStrategyConfiguration > - FailureRateRestartStrategyConfiguration > - ExponentialDelayRestartStrategyConfiguration > - FixedDelayRestartStrategyConfiguration > - NoRestartStrategyConfiguration > > And based on your suggestion, I have restructured the content of the FLIP > to clearly indicate which entire classes will be deprecated and which > methods and fields will be deprecated. Please take a look at the updated > version. > > Best regards, > Junrui > > Jing Ge 于2023年11月5日周日 00:31写道: > > > Hi Junrui, > > > > Thanks for bringing the proposal to our attention. It looks overall > great. > > > > I am just a little bit confused with the content of Flip with the > following > > questions: > > > > 1. How many complex java classes will be deprecated? > > 2. After reading the section title "Deprecate following classes, fields > and > > methods", I thought that e.g. > > org.apache.flink.streaming.api.environment.StreamExecutionEnvironment > will > > be deprecated. I would suggest restructure the content and point out > > clearly which entire classes will be deprecated and which methods and > > fields and not the class itself will be deprecated instead of mixing them > > up. WDYT? > > > > Best regards, > > Jing > > > > > > On Fri, Nov 3, 2023 at 8:59 AM Wencong Liu wrote: > > > > > Thanks Junrui for your effort! > > > > > > Making all configuration code paths lead to ConfigOption is a more > > > standardized > > > approach to configuring Flink applications. > > > > > > +1 for this proposal. > > > > > > Best, > > > Wencong Liu > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > At 2023-11-02 10:10:14, "Junrui Lee" wrote: > > > >Hi devs, > > > > > > > >I would like to start a discussion on FLIP-381: Deprecate > configuration > > > >getters/setters that return/set complex Java objects[1]. > > > > > > > >Currently, the job configuration in FLINK is spread out across > different > > > >components, which leads to inconsistencies and confusion. To address > > this > > > >issue, it is necessary to migrate non-ConfigOption complex Java > objects > > to > > > >use ConfigOption and adopt a single Configuration object to host all > the > > > >configuration. > > > >However, there is a significant blocker in implementing this solution. > > > >These complex Java objects in StreamExecutionEnvironment, > > > CheckpointConfig, > > > >and ExecutionConfig have already been exposed through the public API, > > > >making it challenging to modify the existing implementation. > > > > > > > >Therefore, I propose to deprecate these Java objects and their > > > >corresponding getter/setter interfaces, ultimately removing them in > > > >FLINK-2.0. > > > > > > > >Your feedback and thoughts on this proposal are highly appreciated. > > > > > > > >Best regards, > > > >Junrui Lee > > > > > > > >[1] > > > > > > > > > > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=278464992 > > > > > >
Re: [DISCUSS] FLIP-381: Deprecate configuration getters/setters that return/set complex Java objects
Hi Jing, Thank you for your feedback on the proposal. I appreciate your input and understand your concerns. Regarding your questions, the complex Java class that will be deprecated is org.apache.flink.api.common.restartstrategy.RestartStrategies and and its internal classes, which include: - RestartStrategyConfiguration - FallbackRestartStrategyConfiguration - FailureRateRestartStrategyConfiguration - ExponentialDelayRestartStrategyConfiguration - FixedDelayRestartStrategyConfiguration - NoRestartStrategyConfiguration And based on your suggestion, I have restructured the content of the FLIP to clearly indicate which entire classes will be deprecated and which methods and fields will be deprecated. Please take a look at the updated version. Best regards, Junrui Jing Ge 于2023年11月5日周日 00:31写道: > Hi Junrui, > > Thanks for bringing the proposal to our attention. It looks overall great. > > I am just a little bit confused with the content of Flip with the following > questions: > > 1. How many complex java classes will be deprecated? > 2. After reading the section title "Deprecate following classes, fields and > methods", I thought that e.g. > org.apache.flink.streaming.api.environment.StreamExecutionEnvironment will > be deprecated. I would suggest restructure the content and point out > clearly which entire classes will be deprecated and which methods and > fields and not the class itself will be deprecated instead of mixing them > up. WDYT? > > Best regards, > Jing > > > On Fri, Nov 3, 2023 at 8:59 AM Wencong Liu wrote: > > > Thanks Junrui for your effort! > > > > Making all configuration code paths lead to ConfigOption is a more > > standardized > > approach to configuring Flink applications. > > > > +1 for this proposal. > > > > Best, > > Wencong Liu > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > At 2023-11-02 10:10:14, "Junrui Lee" wrote: > > >Hi devs, > > > > > >I would like to start a discussion on FLIP-381: Deprecate configuration > > >getters/setters that return/set complex Java objects[1]. > > > > > >Currently, the job configuration in FLINK is spread out across different > > >components, which leads to inconsistencies and confusion. To address > this > > >issue, it is necessary to migrate non-ConfigOption complex Java objects > to > > >use ConfigOption and adopt a single Configuration object to host all the > > >configuration. > > >However, there is a significant blocker in implementing this solution. > > >These complex Java objects in StreamExecutionEnvironment, > > CheckpointConfig, > > >and ExecutionConfig have already been exposed through the public API, > > >making it challenging to modify the existing implementation. > > > > > >Therefore, I propose to deprecate these Java objects and their > > >corresponding getter/setter interfaces, ultimately removing them in > > >FLINK-2.0. > > > > > >Your feedback and thoughts on this proposal are highly appreciated. > > > > > >Best regards, > > >Junrui Lee > > > > > >[1] > > > > > > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=278464992 > > >
Re: [DISCUSS] FLIP-381: Deprecate configuration getters/setters that return/set complex Java objects
Hi Junrui, Thanks for bringing the proposal to our attention. It looks overall great. I am just a little bit confused with the content of Flip with the following questions: 1. How many complex java classes will be deprecated? 2. After reading the section title "Deprecate following classes, fields and methods", I thought that e.g. org.apache.flink.streaming.api.environment.StreamExecutionEnvironment will be deprecated. I would suggest restructure the content and point out clearly which entire classes will be deprecated and which methods and fields and not the class itself will be deprecated instead of mixing them up. WDYT? Best regards, Jing On Fri, Nov 3, 2023 at 8:59 AM Wencong Liu wrote: > Thanks Junrui for your effort! > > Making all configuration code paths lead to ConfigOption is a more > standardized > approach to configuring Flink applications. > > +1 for this proposal. > > Best, > Wencong Liu > > > > > > > > > > > > > > > At 2023-11-02 10:10:14, "Junrui Lee" wrote: > >Hi devs, > > > >I would like to start a discussion on FLIP-381: Deprecate configuration > >getters/setters that return/set complex Java objects[1]. > > > >Currently, the job configuration in FLINK is spread out across different > >components, which leads to inconsistencies and confusion. To address this > >issue, it is necessary to migrate non-ConfigOption complex Java objects to > >use ConfigOption and adopt a single Configuration object to host all the > >configuration. > >However, there is a significant blocker in implementing this solution. > >These complex Java objects in StreamExecutionEnvironment, > CheckpointConfig, > >and ExecutionConfig have already been exposed through the public API, > >making it challenging to modify the existing implementation. > > > >Therefore, I propose to deprecate these Java objects and their > >corresponding getter/setter interfaces, ultimately removing them in > >FLINK-2.0. > > > >Your feedback and thoughts on this proposal are highly appreciated. > > > >Best regards, > >Junrui Lee > > > >[1] > > > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=278464992 >
Re:[DISCUSS] FLIP-381: Deprecate configuration getters/setters that return/set complex Java objects
Thanks Junrui for your effort! Making all configuration code paths lead to ConfigOption is a more standardized approach to configuring Flink applications. +1 for this proposal. Best, Wencong Liu At 2023-11-02 10:10:14, "Junrui Lee" wrote: >Hi devs, > >I would like to start a discussion on FLIP-381: Deprecate configuration >getters/setters that return/set complex Java objects[1]. > >Currently, the job configuration in FLINK is spread out across different >components, which leads to inconsistencies and confusion. To address this >issue, it is necessary to migrate non-ConfigOption complex Java objects to >use ConfigOption and adopt a single Configuration object to host all the >configuration. >However, there is a significant blocker in implementing this solution. >These complex Java objects in StreamExecutionEnvironment, CheckpointConfig, >and ExecutionConfig have already been exposed through the public API, >making it challenging to modify the existing implementation. > >Therefore, I propose to deprecate these Java objects and their >corresponding getter/setter interfaces, ultimately removing them in >FLINK-2.0. > >Your feedback and thoughts on this proposal are highly appreciated. > >Best regards, >Junrui Lee > >[1] >https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=278464992
Re: [DISCUSS] FLIP-381: Deprecate configuration getters/setters that return/set complex Java objects
Hi Hang, Thank you for your feedback. Regarding your suggestion to delete the private field 'storage', I want to clarify that in this FLIP, we are only deprecating the getter and setter methods for CheckpointConfig#storage. Therefore, we cannot directly remove the 'storage' field at this moment. However, we will consider removing it altogether in FLINK-2.0. Best regards, Junrui Hang Ruan 于2023年11月3日周五 12:04写道: > Thanks Junrui for driving the proposal. > > +1 from my side. This FLIP will help to make the configuration clearer for > users. > > ps: We should also delete the private field `storage` as its getter and > setter are deleted and it is marked as `@Deprecated`. This is not written > in the FLIP. > > Best, > Hang > > Yuxin Tan 于2023年11月3日周五 11:30写道: > > > Thanks Junrui for driving the proposal. > > > > +1 for this proposal. I believe this change will enhance the usability of > > Flink configuration for both users and developers, while also ensuring > > consistency across various types of configurations. > > > > Best, > > Yuxin > > > > > > Lijie Wang 于2023年11月3日周五 10:59写道: > > > > > Thanks Junrui for driving this. > > > > > > Making configurations simple and consistent has great benefits for both > > > users and devs. +1 for the proposal. > > > > > > Best, > > > Lijie > > > > > > weijie guo 于2023年11月2日周四 16:49写道: > > > > > > > Thanks Junrui for driving this proposal! > > > > > > > > I believe this is helpful for the new Process Function API. Because > we > > > > don't need to move some related class/components from flink-core to a > > > pure > > > > API module (maybe, called flink-core-api) after this. Even though the > > > FLIP > > > > related to new API is in preparation atm, I still want to emphasize > our > > > > goal is that user application should no longer depend on these stuff. > > So > > > > I'm + 1 for this proposal. > > > > > > > > > > > > Best regards, > > > > > > > > Weijie > > > > > > > > > > > > Zhu Zhu 于2023年11月2日周四 16:00写道: > > > > > > > > > Thanks Junrui for creating the FLIP and kicking off this > discussion. > > > > > > > > > > The community has been constantly striving to unify and simplify > the > > > > > configuration layer of Flink. Some progress has already been made, > > > > > such as FLINK-29379. However, the compatibility of public > interfaces > > > > > poses an obstacle to completing the task. The release of Flink 2.0 > > > > > presents a great opportunity to accomplish this goal. > > > > > > > > > > +1 for the proposal. > > > > > > > > > > Thanks, > > > > > Zhu > > > > > > > > > > Rui Fan <1996fan...@gmail.com> 于2023年11月2日周四 10:27写道: > > > > > > > > > > > Thanks Junrui for driving this proposal! > > > > > > > > > > > > ConfigOption is easy to use for flink users, easy to manage > options > > > > > > for flink platform maintainers, and easy to maintain for flink > > > > developers > > > > > > and flink community. > > > > > > > > > > > > So big +1 for this proposal! > > > > > > > > > > > > Best, > > > > > > Rui > > > > > > > > > > > > On Thu, Nov 2, 2023 at 10:10 AM Junrui Lee > > > > wrote: > > > > > > > > > > > > > Hi devs, > > > > > > > > > > > > > > I would like to start a discussion on FLIP-381: Deprecate > > > > configuration > > > > > > > getters/setters that return/set complex Java objects[1]. > > > > > > > > > > > > > > Currently, the job configuration in FLINK is spread out across > > > > > different > > > > > > > components, which leads to inconsistencies and confusion. To > > > address > > > > > this > > > > > > > issue, it is necessary to migrate non-ConfigOption complex Java > > > > objects > > > > > > to > > > > > > > use ConfigOption and adopt a single Configuration object to > host > > > all > > > > > the > > > > > > > configuration. > > > > > > > However, there is a significant blocker in implementing this > > > > solution. > > > > > > > These complex Java objects in StreamExecutionEnvironment, > > > > > > CheckpointConfig, > > > > > > > and ExecutionConfig have already been exposed through the > public > > > API, > > > > > > > making it challenging to modify the existing implementation. > > > > > > > > > > > > > > Therefore, I propose to deprecate these Java objects and their > > > > > > > corresponding getter/setter interfaces, ultimately removing > them > > in > > > > > > > FLINK-2.0. > > > > > > > > > > > > > > Your feedback and thoughts on this proposal are highly > > appreciated. > > > > > > > > > > > > > > Best regards, > > > > > > > Junrui Lee > > > > > > > > > > > > > > [1] > > > > > > > > > > > > > > > > > > > > > > > > > > > > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=278464992 > > > > > > > > > > > > > > > > > > > > > > > > > > > >
Re: [DISCUSS] FLIP-381: Deprecate configuration getters/setters that return/set complex Java objects
Thanks Junrui for driving the proposal. +1 from my side. This FLIP will help to make the configuration clearer for users. ps: We should also delete the private field `storage` as its getter and setter are deleted and it is marked as `@Deprecated`. This is not written in the FLIP. Best, Hang Yuxin Tan 于2023年11月3日周五 11:30写道: > Thanks Junrui for driving the proposal. > > +1 for this proposal. I believe this change will enhance the usability of > Flink configuration for both users and developers, while also ensuring > consistency across various types of configurations. > > Best, > Yuxin > > > Lijie Wang 于2023年11月3日周五 10:59写道: > > > Thanks Junrui for driving this. > > > > Making configurations simple and consistent has great benefits for both > > users and devs. +1 for the proposal. > > > > Best, > > Lijie > > > > weijie guo 于2023年11月2日周四 16:49写道: > > > > > Thanks Junrui for driving this proposal! > > > > > > I believe this is helpful for the new Process Function API. Because we > > > don't need to move some related class/components from flink-core to a > > pure > > > API module (maybe, called flink-core-api) after this. Even though the > > FLIP > > > related to new API is in preparation atm, I still want to emphasize our > > > goal is that user application should no longer depend on these stuff. > So > > > I'm + 1 for this proposal. > > > > > > > > > Best regards, > > > > > > Weijie > > > > > > > > > Zhu Zhu 于2023年11月2日周四 16:00写道: > > > > > > > Thanks Junrui for creating the FLIP and kicking off this discussion. > > > > > > > > The community has been constantly striving to unify and simplify the > > > > configuration layer of Flink. Some progress has already been made, > > > > such as FLINK-29379. However, the compatibility of public interfaces > > > > poses an obstacle to completing the task. The release of Flink 2.0 > > > > presents a great opportunity to accomplish this goal. > > > > > > > > +1 for the proposal. > > > > > > > > Thanks, > > > > Zhu > > > > > > > > Rui Fan <1996fan...@gmail.com> 于2023年11月2日周四 10:27写道: > > > > > > > > > Thanks Junrui for driving this proposal! > > > > > > > > > > ConfigOption is easy to use for flink users, easy to manage options > > > > > for flink platform maintainers, and easy to maintain for flink > > > developers > > > > > and flink community. > > > > > > > > > > So big +1 for this proposal! > > > > > > > > > > Best, > > > > > Rui > > > > > > > > > > On Thu, Nov 2, 2023 at 10:10 AM Junrui Lee > > > wrote: > > > > > > > > > > > Hi devs, > > > > > > > > > > > > I would like to start a discussion on FLIP-381: Deprecate > > > configuration > > > > > > getters/setters that return/set complex Java objects[1]. > > > > > > > > > > > > Currently, the job configuration in FLINK is spread out across > > > > different > > > > > > components, which leads to inconsistencies and confusion. To > > address > > > > this > > > > > > issue, it is necessary to migrate non-ConfigOption complex Java > > > objects > > > > > to > > > > > > use ConfigOption and adopt a single Configuration object to host > > all > > > > the > > > > > > configuration. > > > > > > However, there is a significant blocker in implementing this > > > solution. > > > > > > These complex Java objects in StreamExecutionEnvironment, > > > > > CheckpointConfig, > > > > > > and ExecutionConfig have already been exposed through the public > > API, > > > > > > making it challenging to modify the existing implementation. > > > > > > > > > > > > Therefore, I propose to deprecate these Java objects and their > > > > > > corresponding getter/setter interfaces, ultimately removing them > in > > > > > > FLINK-2.0. > > > > > > > > > > > > Your feedback and thoughts on this proposal are highly > appreciated. > > > > > > > > > > > > Best regards, > > > > > > Junrui Lee > > > > > > > > > > > > [1] > > > > > > > > > > > > > > > > > > > > > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=278464992 > > > > > > > > > > > > > > > > > > > > >
Re: [DISCUSS] FLIP-381: Deprecate configuration getters/setters that return/set complex Java objects
Thanks Junrui for driving the proposal. +1 for this proposal. I believe this change will enhance the usability of Flink configuration for both users and developers, while also ensuring consistency across various types of configurations. Best, Yuxin Lijie Wang 于2023年11月3日周五 10:59写道: > Thanks Junrui for driving this. > > Making configurations simple and consistent has great benefits for both > users and devs. +1 for the proposal. > > Best, > Lijie > > weijie guo 于2023年11月2日周四 16:49写道: > > > Thanks Junrui for driving this proposal! > > > > I believe this is helpful for the new Process Function API. Because we > > don't need to move some related class/components from flink-core to a > pure > > API module (maybe, called flink-core-api) after this. Even though the > FLIP > > related to new API is in preparation atm, I still want to emphasize our > > goal is that user application should no longer depend on these stuff. So > > I'm + 1 for this proposal. > > > > > > Best regards, > > > > Weijie > > > > > > Zhu Zhu 于2023年11月2日周四 16:00写道: > > > > > Thanks Junrui for creating the FLIP and kicking off this discussion. > > > > > > The community has been constantly striving to unify and simplify the > > > configuration layer of Flink. Some progress has already been made, > > > such as FLINK-29379. However, the compatibility of public interfaces > > > poses an obstacle to completing the task. The release of Flink 2.0 > > > presents a great opportunity to accomplish this goal. > > > > > > +1 for the proposal. > > > > > > Thanks, > > > Zhu > > > > > > Rui Fan <1996fan...@gmail.com> 于2023年11月2日周四 10:27写道: > > > > > > > Thanks Junrui for driving this proposal! > > > > > > > > ConfigOption is easy to use for flink users, easy to manage options > > > > for flink platform maintainers, and easy to maintain for flink > > developers > > > > and flink community. > > > > > > > > So big +1 for this proposal! > > > > > > > > Best, > > > > Rui > > > > > > > > On Thu, Nov 2, 2023 at 10:10 AM Junrui Lee > > wrote: > > > > > > > > > Hi devs, > > > > > > > > > > I would like to start a discussion on FLIP-381: Deprecate > > configuration > > > > > getters/setters that return/set complex Java objects[1]. > > > > > > > > > > Currently, the job configuration in FLINK is spread out across > > > different > > > > > components, which leads to inconsistencies and confusion. To > address > > > this > > > > > issue, it is necessary to migrate non-ConfigOption complex Java > > objects > > > > to > > > > > use ConfigOption and adopt a single Configuration object to host > all > > > the > > > > > configuration. > > > > > However, there is a significant blocker in implementing this > > solution. > > > > > These complex Java objects in StreamExecutionEnvironment, > > > > CheckpointConfig, > > > > > and ExecutionConfig have already been exposed through the public > API, > > > > > making it challenging to modify the existing implementation. > > > > > > > > > > Therefore, I propose to deprecate these Java objects and their > > > > > corresponding getter/setter interfaces, ultimately removing them in > > > > > FLINK-2.0. > > > > > > > > > > Your feedback and thoughts on this proposal are highly appreciated. > > > > > > > > > > Best regards, > > > > > Junrui Lee > > > > > > > > > > [1] > > > > > > > > > > > > > > > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=278464992 > > > > > > > > > > > > > > >
Re: [DISCUSS] FLIP-381: Deprecate configuration getters/setters that return/set complex Java objects
Thanks Junrui for driving this. Making configurations simple and consistent has great benefits for both users and devs. +1 for the proposal. Best, Lijie weijie guo 于2023年11月2日周四 16:49写道: > Thanks Junrui for driving this proposal! > > I believe this is helpful for the new Process Function API. Because we > don't need to move some related class/components from flink-core to a pure > API module (maybe, called flink-core-api) after this. Even though the FLIP > related to new API is in preparation atm, I still want to emphasize our > goal is that user application should no longer depend on these stuff. So > I'm + 1 for this proposal. > > > Best regards, > > Weijie > > > Zhu Zhu 于2023年11月2日周四 16:00写道: > > > Thanks Junrui for creating the FLIP and kicking off this discussion. > > > > The community has been constantly striving to unify and simplify the > > configuration layer of Flink. Some progress has already been made, > > such as FLINK-29379. However, the compatibility of public interfaces > > poses an obstacle to completing the task. The release of Flink 2.0 > > presents a great opportunity to accomplish this goal. > > > > +1 for the proposal. > > > > Thanks, > > Zhu > > > > Rui Fan <1996fan...@gmail.com> 于2023年11月2日周四 10:27写道: > > > > > Thanks Junrui for driving this proposal! > > > > > > ConfigOption is easy to use for flink users, easy to manage options > > > for flink platform maintainers, and easy to maintain for flink > developers > > > and flink community. > > > > > > So big +1 for this proposal! > > > > > > Best, > > > Rui > > > > > > On Thu, Nov 2, 2023 at 10:10 AM Junrui Lee > wrote: > > > > > > > Hi devs, > > > > > > > > I would like to start a discussion on FLIP-381: Deprecate > configuration > > > > getters/setters that return/set complex Java objects[1]. > > > > > > > > Currently, the job configuration in FLINK is spread out across > > different > > > > components, which leads to inconsistencies and confusion. To address > > this > > > > issue, it is necessary to migrate non-ConfigOption complex Java > objects > > > to > > > > use ConfigOption and adopt a single Configuration object to host all > > the > > > > configuration. > > > > However, there is a significant blocker in implementing this > solution. > > > > These complex Java objects in StreamExecutionEnvironment, > > > CheckpointConfig, > > > > and ExecutionConfig have already been exposed through the public API, > > > > making it challenging to modify the existing implementation. > > > > > > > > Therefore, I propose to deprecate these Java objects and their > > > > corresponding getter/setter interfaces, ultimately removing them in > > > > FLINK-2.0. > > > > > > > > Your feedback and thoughts on this proposal are highly appreciated. > > > > > > > > Best regards, > > > > Junrui Lee > > > > > > > > [1] > > > > > > > > > > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=278464992 > > > > > > > > > >
Re: [DISCUSS] FLIP-381: Deprecate configuration getters/setters that return/set complex Java objects
Thanks Junrui for driving this proposal! I believe this is helpful for the new Process Function API. Because we don't need to move some related class/components from flink-core to a pure API module (maybe, called flink-core-api) after this. Even though the FLIP related to new API is in preparation atm, I still want to emphasize our goal is that user application should no longer depend on these stuff. So I'm + 1 for this proposal. Best regards, Weijie Zhu Zhu 于2023年11月2日周四 16:00写道: > Thanks Junrui for creating the FLIP and kicking off this discussion. > > The community has been constantly striving to unify and simplify the > configuration layer of Flink. Some progress has already been made, > such as FLINK-29379. However, the compatibility of public interfaces > poses an obstacle to completing the task. The release of Flink 2.0 > presents a great opportunity to accomplish this goal. > > +1 for the proposal. > > Thanks, > Zhu > > Rui Fan <1996fan...@gmail.com> 于2023年11月2日周四 10:27写道: > > > Thanks Junrui for driving this proposal! > > > > ConfigOption is easy to use for flink users, easy to manage options > > for flink platform maintainers, and easy to maintain for flink developers > > and flink community. > > > > So big +1 for this proposal! > > > > Best, > > Rui > > > > On Thu, Nov 2, 2023 at 10:10 AM Junrui Lee wrote: > > > > > Hi devs, > > > > > > I would like to start a discussion on FLIP-381: Deprecate configuration > > > getters/setters that return/set complex Java objects[1]. > > > > > > Currently, the job configuration in FLINK is spread out across > different > > > components, which leads to inconsistencies and confusion. To address > this > > > issue, it is necessary to migrate non-ConfigOption complex Java objects > > to > > > use ConfigOption and adopt a single Configuration object to host all > the > > > configuration. > > > However, there is a significant blocker in implementing this solution. > > > These complex Java objects in StreamExecutionEnvironment, > > CheckpointConfig, > > > and ExecutionConfig have already been exposed through the public API, > > > making it challenging to modify the existing implementation. > > > > > > Therefore, I propose to deprecate these Java objects and their > > > corresponding getter/setter interfaces, ultimately removing them in > > > FLINK-2.0. > > > > > > Your feedback and thoughts on this proposal are highly appreciated. > > > > > > Best regards, > > > Junrui Lee > > > > > > [1] > > > > > > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=278464992 > > > > > >
Re: [DISCUSS] FLIP-381: Deprecate configuration getters/setters that return/set complex Java objects
Thanks Junrui for creating the FLIP and kicking off this discussion. The community has been constantly striving to unify and simplify the configuration layer of Flink. Some progress has already been made, such as FLINK-29379. However, the compatibility of public interfaces poses an obstacle to completing the task. The release of Flink 2.0 presents a great opportunity to accomplish this goal. +1 for the proposal. Thanks, Zhu Rui Fan <1996fan...@gmail.com> 于2023年11月2日周四 10:27写道: > Thanks Junrui for driving this proposal! > > ConfigOption is easy to use for flink users, easy to manage options > for flink platform maintainers, and easy to maintain for flink developers > and flink community. > > So big +1 for this proposal! > > Best, > Rui > > On Thu, Nov 2, 2023 at 10:10 AM Junrui Lee wrote: > > > Hi devs, > > > > I would like to start a discussion on FLIP-381: Deprecate configuration > > getters/setters that return/set complex Java objects[1]. > > > > Currently, the job configuration in FLINK is spread out across different > > components, which leads to inconsistencies and confusion. To address this > > issue, it is necessary to migrate non-ConfigOption complex Java objects > to > > use ConfigOption and adopt a single Configuration object to host all the > > configuration. > > However, there is a significant blocker in implementing this solution. > > These complex Java objects in StreamExecutionEnvironment, > CheckpointConfig, > > and ExecutionConfig have already been exposed through the public API, > > making it challenging to modify the existing implementation. > > > > Therefore, I propose to deprecate these Java objects and their > > corresponding getter/setter interfaces, ultimately removing them in > > FLINK-2.0. > > > > Your feedback and thoughts on this proposal are highly appreciated. > > > > Best regards, > > Junrui Lee > > > > [1] > > > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=278464992 > > >
Re: [DISCUSS] FLIP-381: Deprecate configuration getters/setters that return/set complex Java objects
Thanks Junrui for driving this proposal! ConfigOption is easy to use for flink users, easy to manage options for flink platform maintainers, and easy to maintain for flink developers and flink community. So big +1 for this proposal! Best, Rui On Thu, Nov 2, 2023 at 10:10 AM Junrui Lee wrote: > Hi devs, > > I would like to start a discussion on FLIP-381: Deprecate configuration > getters/setters that return/set complex Java objects[1]. > > Currently, the job configuration in FLINK is spread out across different > components, which leads to inconsistencies and confusion. To address this > issue, it is necessary to migrate non-ConfigOption complex Java objects to > use ConfigOption and adopt a single Configuration object to host all the > configuration. > However, there is a significant blocker in implementing this solution. > These complex Java objects in StreamExecutionEnvironment, CheckpointConfig, > and ExecutionConfig have already been exposed through the public API, > making it challenging to modify the existing implementation. > > Therefore, I propose to deprecate these Java objects and their > corresponding getter/setter interfaces, ultimately removing them in > FLINK-2.0. > > Your feedback and thoughts on this proposal are highly appreciated. > > Best regards, > Junrui Lee > > [1] > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=278464992 >