Re: [VOTE] FLIP-398: Improve Serialization Configuration And Usage In Flink

2023-12-26 Thread Zhanghao Chen
+1 (non-binding)

Best,
Zhanghao Chen

From: Yong Fang 
Sent: Wednesday, December 27, 2023 14:54
To: dev 
Subject: [VOTE] FLIP-398: Improve Serialization Configuration And Usage In Flink

Hi devs,

Thanks for all feedback about the FLIP-398: Improve Serialization
Configuration And Usage In Flink [1] which has been discussed in [2].

I'd like to start a vote for it. The vote will be open for at least 72
hours unless there is an objection or insufficient votes.

[1]
https://cwiki.apache.org/confluence/display/FLINK/FLIP-398%3A+Improve+Serialization+Configuration+And+Usage+In+Flink
[2] https://lists.apache.org/thread/m67s4qfrh660lktpq7yqf9docvvf5o9l

Best,
Fang Yong


Re: [VOTE] FLIP-398: Improve Serialization Configuration And Usage In Flink

2023-12-26 Thread Xintong Song
+1 (binding)


Best,

Xintong



On Wed, Dec 27, 2023 at 2:54 PM Yong Fang  wrote:

> Hi devs,
>
> Thanks for all feedback about the FLIP-398: Improve Serialization
> Configuration And Usage In Flink [1] which has been discussed in [2].
>
> I'd like to start a vote for it. The vote will be open for at least 72
> hours unless there is an objection or insufficient votes.
>
> [1]
>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-398%3A+Improve+Serialization+Configuration+And+Usage+In+Flink
> [2] https://lists.apache.org/thread/m67s4qfrh660lktpq7yqf9docvvf5o9l
>
> Best,
> Fang Yong
>


Re: [DISCUSS] FLIP-397: Add config options for administrator JVM options

2023-12-26 Thread Yong Fang
+1 for this, we have met jobs that need to set GC policies different from
the default ones to improve performance. Separating the default and
user-set ones can help us better manage them.

Best,
Fang Yong

On Fri, Dec 22, 2023 at 9:18 PM Benchao Li  wrote:

> +1 from my side,
>
> I also met some scenarios that I wanted to set some JVM options by
> default for all Flink jobs before, such as
> '-XX:-DontCompileHugeMethods', without it, some generated big methods
> won't be optimized in JVM C2 compiler, leading to poor performance.
>
> Zhanghao Chen  于2023年11月27日周一 20:04写道:
> >
> > Hi devs,
> >
> > I'd like to start a discussion on FLIP-397: Add config options for
> administrator JVM options [1].
> >
> > In production environments, users typically develop and operate their
> Flink jobs through a managed platform. Users may need to add JVM options to
> their Flink applications (e.g. to tune GC options). They typically use the
> env.java.opts.x series of options to do so. Platform administrators also
> have a set of JVM options to apply by default, e.g. to use JVM 17, enable
> GC logging, or apply pretuned GC options, etc. Both use cases will need to
> set the same series of options and will clobber one another. Similar issues
> have been described in SPARK-23472 [2].
> >
> > Therefore, I propose adding a set of default JVM options for
> administrator use that prepends the user-set extra JVM options.
> >
> > Looking forward to hearing from you.
> >
> > [1]
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-397%3A+Add+config+options+for+administrator+JVM+options
> > [2] https://issues.apache.org/jira/browse/SPARK-23472
> >
> > Best,
> > Zhanghao Chen
>
>
>
> --
>
> Best,
> Benchao Li
>


[VOTE] FLIP-398: Improve Serialization Configuration And Usage In Flink

2023-12-26 Thread Yong Fang
Hi devs,

Thanks for all feedback about the FLIP-398: Improve Serialization
Configuration And Usage In Flink [1] which has been discussed in [2].

I'd like to start a vote for it. The vote will be open for at least 72
hours unless there is an objection or insufficient votes.

[1]
https://cwiki.apache.org/confluence/display/FLINK/FLIP-398%3A+Improve+Serialization+Configuration+And+Usage+In+Flink
[2] https://lists.apache.org/thread/m67s4qfrh660lktpq7yqf9docvvf5o9l

Best,
Fang Yong


Re: [DISCUSS] FLIP-406: Reorganize State & Checkpointing & Recovery Configuration

2023-12-26 Thread Zakelly Lan
Hi Lijie,

Thanks for the reminder! I missed this.

Considering the name occupation, how about naming it as
`execution.checkpointing.type`?

Actually I think the current `execution.checkpointing.mode` is confusing in
some ways, maybe `execution.checkpointing.data-consistency` is better.


Best,
Zakelly


On Wed, Dec 27, 2023 at 12:59 PM Lijie Wang 
wrote:

> Hi Zakelly,
>
> >> I'm wondering if `execution.checkpointing.savepoint-dir` would be
> better.
>
> `execution.checkpointing.dir` and `execution.checkpointing.savepoint-dir`
> are also fine for me.
>
> >> So I think an enumeration option `execution.checkpointing.mode` which
> can be 'full' (default) or 'incremental' would be better
>
> I agree with using an enumeration option. But currently there is already a
> configuration option called `execution.checkpointing.mode`, which is used
> to choose EXACTLY_ONCE or AT_LEAST_ONCE. Maybe we need to use another name
> or merge these two options.
>
> Best,
> Lijie
>
> Zakelly Lan  于2023年12月27日周三 11:43写道:
>
> > Hi everyone,
> >
> > Thanks all for your comments!
> >
> > @Yanfei
> >
> > > 1. For some state backends that do not support incremental checkpoint,
> > > how does the execution.checkpointing.incrementaloption take effect? Or
> > > is it better to put incremental under state.backend.xxx.incremental?
> > >
> > I'd rather not put the option for incremental checkpoint under the
> > 'state.backend', since it is more about the checkpointing instead of
> state
> > accessing. Of course, the state backend may not necessarily do
> incremental
> > checkpoint as requested. If the state backend is not capable of taking
> > incremental cp, it is better to fallback to the full cp.
> >
> > 2. I'm a little worried that putting all configurations into
> > > `ExecutionCheckpointingOptions` will introduce some dependency
> > > problems. Some options would be used by flink-runtime module, but
> > > flink-runtime should not depend on flink-streaming-java. e.g.
> > > FLINK-28286[1].
> > > So, I prefer to move configurations to `CheckpointingOptions`, WDYT?
> > >
> >
> > Yes, that's a very good point.  Moving to
> > `CheckpointingOptions`(flink-core) makes sense.
> >
> > @Lijie
> >
> > How about
> > > state.savepoints.dir -> execution.checkpointing.savepoint.dir
> > > state.checkpoints.dir -> execution.checkpointing.checkpoint.dir
> >
> >
> > Actually, I think the `checkpointing.checkpoint` may cause some
> confusion.
> > But I'm ok if others agree.
> > I'm wondering if `execution.checkpointing.savepoint-dir` would be better.
> > WDYT?
> >
> > 2. We changed the execution.checkpointing.local-copy' to
> > > 'execution.checkpointing.local-copy.enabled'. Should we also add
> > "enabled"
> > > suffix for other boolean type configuration options ? For example,
> > > execution.checkpointing.incremental ->
> > > execution.checkpointing.incremental.enabled
> > >
> >
> > Actually, the incremental cp is something like choosing a mode for doing
> > checkpoint instead of enabling a function. So I think an enumeration
> option
> > `execution.checkpointing.mode` which can be 'full' (default) or
> > 'incremental' would be better, WDYT?
> > And @Rui Fan @Yanfei What do you think about this?
> >
> >
> > On Tue, Dec 26, 2023 at 5:15 PM Lijie Wang 
> > wrote:
> >
> > > Hi Zakelly,
> > >
> > > Thanks for driving the discussion.
> > >
> > > 1.
> > > >> But I'm not so sure since there is only one savepoint-related
> option.
> > > Maybe someone else could share some thoughts here.
> > >
> > > How about
> > > state.savepoints.dir -> execution.checkpointing.savepoint.dir
> > > state.checkpoints.dir -> execution.checkpointing.checkpoint.dir
> > >
> > > 2. We changed the execution.checkpointing.local-copy' to
> > > 'execution.checkpointing.local-copy.enabled'. Should we also add
> > "enabled"
> > > suffix for other boolean type configuration options ? For example,
> > > execution.checkpointing.incremental ->
> > > execution.checkpointing.incremental.enabled
> > >
> > > In this way, the naming style of configuration options is unified, and
> it
> > > can avoid potential similar problems (for example, we may need to add
> > more
> > > options for incremental checkpoint in the future).
> > >
> > > Best,
> > > Lijie
> > >
> > > Yanfei Lei  于2023年12月26日周二 12:05写道:
> > >
> > > > Hi Zakelly,
> > > >
> > > > Thank you for creating the FLIP and starting the discussion.
> > > >
> > > > The current arrangement of these options is indeed somewhat
> haphazard,
> > > > and the new arrangement looks much better. I have some questions
> about
> > > > the arrangement of some new configuration options:
> > > >
> > > > 1. For some state backends that do not support incremental
> checkpoint,
> > > > how does the execution.checkpointing.incrementaloption take effect?
> Or
> > > > is it better to put incremental under state.backend.xxx.incremental?
> > > >
> > > > 2. I'm a little worried that putting all configurations into
> > > > `ExecutionCheckpointingOptions` will 

Re: [VOTE] Release flink-connector-pulsar 4.1.0, release candidate #1

2023-12-26 Thread Qingsheng Ren
+1 (binding)

- Verified checksum and signature
- Checked that no binary exists in source code release
- Checked Maven staging repository contains expected JARs
- Checked release note
- Built from source locally
- Reviewed web PR

Thanks for the effort, Leonard!

Best,
Qingsheng



On Thu, Dec 21, 2023 at 11:33 PM Leonard Xu  wrote:

> +1 (binding)
>
> - built from source code succeeded
> - verified signatures
> - verified hashsums
> - checked the contents contains jar and pom files in apache repo
> - checked Github release tag
> - checked release notes
>
> Best,
> Leonard
>
> > 2023年12月21日 下午5:03,Danny Cranmer  写道:
> >
> > Hello,
> >
> > Thanks for driving, Leonard,
> >
> > +1 (binding)
> >
> > - Verified signatures and checksums of source and binaries
> > - Maven dist contains expected files
> > - There are no binaries in the source archive
> > - Release notes look good
> > - Tag exists in Github
> > - CI passes for v4.1 [1] (could not find a tag build, but this one is
> > equivalent)
> > - Sources build locally
> > - Verified bundled dependencies are correctly reflected in NOTICE file
> >
> > Thanks,
> > Danny
> >
> > [1]
> https://github.com/apache/flink-connector-pulsar/actions/runs/7164218129
> >
> > On Wed, Dec 20, 2023 at 11:16 PM Sergey Nuyanzin 
> > wrote:
> >
> >> +1 (non-binding)
> >>
> >> - Downloaded artifacts
> >> - Maven staging artifacts look good
> >> - Verified checksum && keys
> >> - Verified LICENSE and NOTICE files
> >> - Built from source
> >>
> >> On Wed, Dec 20, 2023 at 5:13 AM tison  wrote:
> >>
> >>> Hi Leonard,
> >>>
> >>> You are a PMC member also. Perhaps you can check the candidate and
> >>> vote on what you do :D
> >>>
> >>> Best,
> >>> tison.
> >>>
> >>> Leonard Xu  于2023年12月20日周三 11:35写道:
> 
>  Bubble up, I need more votes, especially from PMC members.
> 
>  Best,
>  Leonard
> 
> > 2023年12月14日 下午11:03,Hang Ruan  写道:
> >
> > +1 (non-binding)
> >
> > - Validated checksum hash
> > - Verified signature
> > - Verified that no binaries exist in the source archive
> > - Build the source with jdk8
> > - Verified web PR
> > - Make sure flink-connector-base have the provided scope
> >
> > Best,
> > Hang
> >
> > tison  于2023年12月14日周四 11:51写道:
> >
> >> Thanks Leonard for driving this release!
> >>
> >> +1 (non-binding)
> >>
> >> * Download link valid
> >> * Maven staging artifacts look good.
> >> * Checksum and gpg matches
> >> * LICENSE and NOTICE exist
> >> * Can build from source.
> >>
> >> Best,
> >> tison.
> >>
> >> Rui Fan <1996fan...@gmail.com> 于2023年12月14日周四 09:23写道:
> >>>
> >>> Thanks Leonard for driving this release!
> >>>
> >>> +1 (non-binding)
> >>>
> >>> - Validated checksum hash
> >>> - Verified signature
> >>> - Verified that no binaries exist in the source archive
> >>> - Build the source with Maven and jdk8
> >>> - Verified licenses
> >>> - Verified web PRs, left a minor comment
> >>>
> >>> Best,
> >>> Rui
> >>>
> >>> On Wed, Dec 13, 2023 at 7:15 PM Leonard Xu 
> >>> wrote:
> 
>  Hey all,
> 
>  Please review and vote on the release candidate #1 for the version
> >> 4.1.0 of the Apache Flink Pulsar Connector as follows:
> 
>  [ ] +1, Approve the release
>  [ ] -1, Do not approve the release (please provide specific
> >>> comments)
> 
>  The complete staging area is available for your review, which
> >>> includes:
>  * JIRA release notes [1],
>  * The official Apache source release to be deployed to
> >>> dist.apache.org
> >> [2], which are signed with the key with fingerprint
>  5B2F6608732389AEB67331F5B197E1F1108998AD [3],
>  * All artifacts to be deployed to the Maven Central Repository
> >> [4],
>  * Source code tag v4.1.0-rc1 [5],
>  * Website pull request listing the new release [6].
> 
>  The vote will be open for at least 72 hours. It is adopted by
> >>> majority
> >> approval, with at least 3 PMC affirmative votes.
> 
> 
>  Best,
>  Leonard
> 
>  [1]
> >>
> >>>
> >>
> https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12315522=12353431
>  [2]
> >>
> >>>
> >>
> https://dist.apache.org/repos/dist/dev/flink/flink-connector-pulsar-4.1.0-rc1/
>  [3] https://dist.apache.org/repos/dist/release/flink/KEYS
>  [4]
> >>
> >>>
> https://repository.apache.org/content/repositories/orgapacheflink-1688/
>  [5]
> >>> https://github.com/apache/flink-connector-pulsar/tree/v4.1.0-rc1
>  [6] https://github.com/apache/flink-web/pull/703
> >>
> 
> >>>
> >>
> >>
> >> --
> >> Best regards,
> >> Sergey
> >>
>
>


[jira] [Created] (FLINK-33949) METHOD_ABSTRACT_NOW_DEFAULT should be both source compatible and binary compatible

2023-12-26 Thread Wencong Liu (Jira)
Wencong Liu created FLINK-33949:
---

 Summary: METHOD_ABSTRACT_NOW_DEFAULT should be both source 
compatible and binary compatible
 Key: FLINK-33949
 URL: https://issues.apache.org/jira/browse/FLINK-33949
 Project: Flink
  Issue Type: Bug
  Components: Test Infrastructure
Affects Versions: 1.19.0
Reporter: Wencong Liu
 Fix For: 1.19.0


Currently  I'm trying to refactor some APIs annotated by @Public in [FLIP-382: 
Unify the Provision of Diverse Metadata for Context-like APIs - Apache Flink - 
Apache Software 
Foundation|https://cwiki.apache.org/confluence/display/FLINK/FLIP-382%3A+Unify+the+Provision+of+Diverse+Metadata+for+Context-like+APIs].
 When an abstract method is changed into a default method, the japicmp maven 
plugin names this change METHOD_ABSTRACT_NOW_DEFAULT and considers it as source 
incompatible and binary incompatible.

The reason maybe that if the abstract method becomes default, the logic in the 
default method will be ignored by the previous implementations.

I create a test case in which a job is compiled with newly changed default 
method and submitted to the previous version. There is no exception thrown. 
Therefore, the METHOD_ABSTRACT_NOW_DEFAULT shouldn't be incompatible both for 
source and binary.

By the way, currently the master branch checks both source compatibility and 
binary compatibility between minor versions. According to Flink's API 
compatibility constraints, the master branch shouldn't check binary 
compatibility. There is already a [Jira|[FLINK-33009] 
tools/release/update_japicmp_configuration.sh should only enable binary 
compatibility checks in the release branch - ASF JIRA (apache.org)] to track it 
and we should fix it as soon as possible.

 

 

 



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


Re: [DISCUSS] FLIP-405: Migrate string configuration key to ConfigOption

2023-12-26 Thread Hang Ruan
Hi, Rui Fan.

Thanks for this FLIP.

I think the key of LOCAL_NUMBER_TASK_MANAGER is better as
'minicluster.number-of-taskmanagers' or 'minicluster.taskmanager-number'
instead of 'minicluster.number-taskmanager'.

Best,
Hang

Xuannan Su  于2023年12月27日周三 12:40写道:

> Hi Xintong and Rui,
>
> Thanks for the quick feedback and the suggestions.
>
> > 1. I think the default value for `TASK_MANAGER_LOG_PATH_KEY` should be
> "no
> > default".
>
> I have considered both ways of describing the default value. However,
> I found out that some of the configurations, such as `web.tmpdir`, put
> `System.getProperty()` in the default value [1]. Some are putting the
> description in the default value column[2]. So I just picked the first
> one. I am fine with either way, so long as they are consistent. WDYT?
>
> > 3. Simply saying "getting / setting value with string key is discouraged"
> > in JavaDoc of get/setString is IMHO a bit confusing. People may have the
> > question why would we keep the discouraged interfaces at all. I would
> > suggest the following:
> > ```
> > We encourage users and developers to always use ConfigOption for getting
> /
> > setting the configurations if possible, for its rich description, type,
> > default-value and other supports. The string-key-based getter / setter
> > should only be used when ConfigOption is not applicable, e.g., the key is
> > programmatically generated in runtime.
> > ```
>
> The suggested comment looks good to me. Thanks for the suggestion. I
> will update the comment in the FLIP.
>
> > 2. So I wonder if we can simply mark them as deprecated and remove in
> 2.0.
>
> After some investigation, it turns out those options of input/output
> format are only publicly exposed in the DataSet docs[2], which is
> deprecated. Thus, marking them as deprecated and removed in Flink 2.0
> looks fine to me.
>
>
> @Rui
>
> > Configuration has a `public  T get(ConfigOption option)` method.
> > Could we remove all `Xxx getXxx(ConfigOption configOption)` methods?
>
> +1 Only keep the get(ConfigOption option),
> getOptional(ConfigOption option), and set(ConfigOption option, T
> value).
>
> Best,
> Xuannan
>
> [1]
> https://nightlies.apache.org/flink/flink-docs-master/docs/deployment/config/#web-tmpdir
> [2]
> https://nightlies.apache.org/flink/flink-docs-master/docs/deployment/config/#kubernetes-container-image-ref
> [3]
> https://nightlies.apache.org/flink/flink-docs-master/docs/dev/dataset/overview/#data-sources
>
>
>
>
> On Tue, Dec 26, 2023 at 8:47 PM Xintong Song 
> wrote:
> >
> > >
> > > Configuration has a `public  T get(ConfigOption option)` method.
> > > Could we remove all `Xxx getXxx(ConfigOption configOption)`
> methods?
> >
> >
> >
> > Note: all `public void setXxx(ConfigOption key, Xxx value)` methods
> > > can be replaced with `public  Configuration set(ConfigOption
> option,
> > > T value)` as well.
> >
> >
> > +1
> >
> >
> > Best,
> >
> > Xintong
> >
> >
> >
> > On Tue, Dec 26, 2023 at 8:44 PM Xintong Song 
> wrote:
> >
> > > These features don't have a public option, but they work. I'm not sure
> > >> whether these features are used by some advanced users.
> > >> Actually, I think some of them are valuable! For example:
> > >>
> > >> - ConfigConstants.YARN_CONTAINER_START_COMMAND_TEMPLATE
> > >>   allows users to define the start command of the yarn container.
> > >> - FileInputFormat.ENUMERATE_NESTED_FILES_FLAG allows
> > >>   flink job reads all files under the directory even if it has nested
> > >> directories.
> > >>
> > >> This FLIP focuses on the refactor option, I'm afraid these features
> are
> > >> used
> > >> in some production and removing these features will affect some flink
> > >> jobs.
> > >> So I prefer to keep these features, WDTY?
> > >>
> > >
> > > First of all, I don't think we should support any knobs that users can
> > > only learn how to use from reading Flink's internal codes. From this
> > > perspective, for existing string-keyed knobs that are not mentioned in
> any
> > > public documentation, yes we can argue that they are functioning, but
> we
> > > can also argue that they are not really exposed to users. That means
> > > migrating them to ConfigOption is not a pure refactor, but would make
> > > something that used to be hidden from users now exposed to users. For
> such
> > > options, I personally would lean toward not exposing them. If we
> consider
> > > them as already exposed, then process-wise there's no problem in
> > > deprecating some infrequently-used options and removing them in a major
> > > version bump, and if they are proved needed later we can add them back
> > > anytime. On the other hand, if we consider them as not yet exposed,
> then
> > > removing them later would be a breaking change.
> > >
> > >
> > > Secondly, I don't really come up with any cases where users need to
> tune
> > > these knobs. E.g., why would we allow users to customize the yarn
> container
> > > start command while we already provide `env.java.opts`? And 

Re: [DISCUSS] FLIP-406: Reorganize State & Checkpointing & Recovery Configuration

2023-12-26 Thread Lijie Wang
Hi Zakelly,

>> I'm wondering if `execution.checkpointing.savepoint-dir` would be better.

`execution.checkpointing.dir` and `execution.checkpointing.savepoint-dir`
are also fine for me.

>> So I think an enumeration option `execution.checkpointing.mode` which
can be 'full' (default) or 'incremental' would be better

I agree with using an enumeration option. But currently there is already a
configuration option called `execution.checkpointing.mode`, which is used
to choose EXACTLY_ONCE or AT_LEAST_ONCE. Maybe we need to use another name
or merge these two options.

Best,
Lijie

Zakelly Lan  于2023年12月27日周三 11:43写道:

> Hi everyone,
>
> Thanks all for your comments!
>
> @Yanfei
>
> > 1. For some state backends that do not support incremental checkpoint,
> > how does the execution.checkpointing.incrementaloption take effect? Or
> > is it better to put incremental under state.backend.xxx.incremental?
> >
> I'd rather not put the option for incremental checkpoint under the
> 'state.backend', since it is more about the checkpointing instead of state
> accessing. Of course, the state backend may not necessarily do incremental
> checkpoint as requested. If the state backend is not capable of taking
> incremental cp, it is better to fallback to the full cp.
>
> 2. I'm a little worried that putting all configurations into
> > `ExecutionCheckpointingOptions` will introduce some dependency
> > problems. Some options would be used by flink-runtime module, but
> > flink-runtime should not depend on flink-streaming-java. e.g.
> > FLINK-28286[1].
> > So, I prefer to move configurations to `CheckpointingOptions`, WDYT?
> >
>
> Yes, that's a very good point.  Moving to
> `CheckpointingOptions`(flink-core) makes sense.
>
> @Lijie
>
> How about
> > state.savepoints.dir -> execution.checkpointing.savepoint.dir
> > state.checkpoints.dir -> execution.checkpointing.checkpoint.dir
>
>
> Actually, I think the `checkpointing.checkpoint` may cause some confusion.
> But I'm ok if others agree.
> I'm wondering if `execution.checkpointing.savepoint-dir` would be better.
> WDYT?
>
> 2. We changed the execution.checkpointing.local-copy' to
> > 'execution.checkpointing.local-copy.enabled'. Should we also add
> "enabled"
> > suffix for other boolean type configuration options ? For example,
> > execution.checkpointing.incremental ->
> > execution.checkpointing.incremental.enabled
> >
>
> Actually, the incremental cp is something like choosing a mode for doing
> checkpoint instead of enabling a function. So I think an enumeration option
> `execution.checkpointing.mode` which can be 'full' (default) or
> 'incremental' would be better, WDYT?
> And @Rui Fan @Yanfei What do you think about this?
>
>
> On Tue, Dec 26, 2023 at 5:15 PM Lijie Wang 
> wrote:
>
> > Hi Zakelly,
> >
> > Thanks for driving the discussion.
> >
> > 1.
> > >> But I'm not so sure since there is only one savepoint-related option.
> > Maybe someone else could share some thoughts here.
> >
> > How about
> > state.savepoints.dir -> execution.checkpointing.savepoint.dir
> > state.checkpoints.dir -> execution.checkpointing.checkpoint.dir
> >
> > 2. We changed the execution.checkpointing.local-copy' to
> > 'execution.checkpointing.local-copy.enabled'. Should we also add
> "enabled"
> > suffix for other boolean type configuration options ? For example,
> > execution.checkpointing.incremental ->
> > execution.checkpointing.incremental.enabled
> >
> > In this way, the naming style of configuration options is unified, and it
> > can avoid potential similar problems (for example, we may need to add
> more
> > options for incremental checkpoint in the future).
> >
> > Best,
> > Lijie
> >
> > Yanfei Lei  于2023年12月26日周二 12:05写道:
> >
> > > Hi Zakelly,
> > >
> > > Thank you for creating the FLIP and starting the discussion.
> > >
> > > The current arrangement of these options is indeed somewhat haphazard,
> > > and the new arrangement looks much better. I have some questions about
> > > the arrangement of some new configuration options:
> > >
> > > 1. For some state backends that do not support incremental checkpoint,
> > > how does the execution.checkpointing.incrementaloption take effect? Or
> > > is it better to put incremental under state.backend.xxx.incremental?
> > >
> > > 2. I'm a little worried that putting all configurations into
> > > `ExecutionCheckpointingOptions` will introduce some dependency
> > > problems. Some options would be used by flink-runtime module, but
> > > flink-runtime should not depend on flink-streaming-java. e.g.
> > > FLINK-28286[1].
> > > So, I prefer to move configurations to `CheckpointingOptions`, WDYT?
> > >
> > > [1] https://issues.apache.org/jira/browse/FLINK-28286
> > >
> > > --
> > > Best,
> > > Yanfei
> > >
> > > Zakelly Lan  于2023年12月25日周一 21:14写道:
> > > >
> > > > Hi Rui Fan and Junrui,
> > > >
> > > > Thanks for the reminder! I agree to change the
> > > > 'execution.checkpointing.local-copy' to
> > > > 'execution.checkpointing.local-copy.enabled'.

Re: [DISCUSS] FLIP-405: Migrate string configuration key to ConfigOption

2023-12-26 Thread Xuannan Su
Hi Xintong and Rui,

Thanks for the quick feedback and the suggestions.

> 1. I think the default value for `TASK_MANAGER_LOG_PATH_KEY` should be "no
> default".

I have considered both ways of describing the default value. However,
I found out that some of the configurations, such as `web.tmpdir`, put
`System.getProperty()` in the default value [1]. Some are putting the
description in the default value column[2]. So I just picked the first
one. I am fine with either way, so long as they are consistent. WDYT?

> 3. Simply saying "getting / setting value with string key is discouraged"
> in JavaDoc of get/setString is IMHO a bit confusing. People may have the
> question why would we keep the discouraged interfaces at all. I would
> suggest the following:
> ```
> We encourage users and developers to always use ConfigOption for getting /
> setting the configurations if possible, for its rich description, type,
> default-value and other supports. The string-key-based getter / setter
> should only be used when ConfigOption is not applicable, e.g., the key is
> programmatically generated in runtime.
> ```

The suggested comment looks good to me. Thanks for the suggestion. I
will update the comment in the FLIP.

> 2. So I wonder if we can simply mark them as deprecated and remove in 2.0.

After some investigation, it turns out those options of input/output
format are only publicly exposed in the DataSet docs[2], which is
deprecated. Thus, marking them as deprecated and removed in Flink 2.0
looks fine to me.


@Rui

> Configuration has a `public  T get(ConfigOption option)` method.
> Could we remove all `Xxx getXxx(ConfigOption configOption)` methods?

+1 Only keep the get(ConfigOption option),
getOptional(ConfigOption option), and set(ConfigOption option, T
value).

Best,
Xuannan

[1] 
https://nightlies.apache.org/flink/flink-docs-master/docs/deployment/config/#web-tmpdir
[2] 
https://nightlies.apache.org/flink/flink-docs-master/docs/deployment/config/#kubernetes-container-image-ref
[3] 
https://nightlies.apache.org/flink/flink-docs-master/docs/dev/dataset/overview/#data-sources




On Tue, Dec 26, 2023 at 8:47 PM Xintong Song  wrote:
>
> >
> > Configuration has a `public  T get(ConfigOption option)` method.
> > Could we remove all `Xxx getXxx(ConfigOption configOption)` methods?
>
>
>
> Note: all `public void setXxx(ConfigOption key, Xxx value)` methods
> > can be replaced with `public  Configuration set(ConfigOption option,
> > T value)` as well.
>
>
> +1
>
>
> Best,
>
> Xintong
>
>
>
> On Tue, Dec 26, 2023 at 8:44 PM Xintong Song  wrote:
>
> > These features don't have a public option, but they work. I'm not sure
> >> whether these features are used by some advanced users.
> >> Actually, I think some of them are valuable! For example:
> >>
> >> - ConfigConstants.YARN_CONTAINER_START_COMMAND_TEMPLATE
> >>   allows users to define the start command of the yarn container.
> >> - FileInputFormat.ENUMERATE_NESTED_FILES_FLAG allows
> >>   flink job reads all files under the directory even if it has nested
> >> directories.
> >>
> >> This FLIP focuses on the refactor option, I'm afraid these features are
> >> used
> >> in some production and removing these features will affect some flink
> >> jobs.
> >> So I prefer to keep these features, WDTY?
> >>
> >
> > First of all, I don't think we should support any knobs that users can
> > only learn how to use from reading Flink's internal codes. From this
> > perspective, for existing string-keyed knobs that are not mentioned in any
> > public documentation, yes we can argue that they are functioning, but we
> > can also argue that they are not really exposed to users. That means
> > migrating them to ConfigOption is not a pure refactor, but would make
> > something that used to be hidden from users now exposed to users. For such
> > options, I personally would lean toward not exposing them. If we consider
> > them as already exposed, then process-wise there's no problem in
> > deprecating some infrequently-used options and removing them in a major
> > version bump, and if they are proved needed later we can add them back
> > anytime. On the other hand, if we consider them as not yet exposed, then
> > removing them later would be a breaking change.
> >
> >
> > Secondly, I don't really come up with any cases where users need to tune
> > these knobs. E.g., why would we allow users to customize the yarn container
> > start command while we already provide `env.java.opts`? And what would be
> > the problem if Flink just doesn't support nested files? And even worse,
> > such knobs may provide chances for users to shoot themself in the foot.
> > E.g., what if %jvmmem% is missing from a user-provided container start
> > command? Admittedly, there might be a small fraction of advanced users that
> > know how to use these knobs. However, those users usually have their own
> > custom fork of Flink, and it should not be a big problem for them to build
> > such abilities by 

Re: [DISCUSS] FLIP-397: Add config options for administrator JVM options

2023-12-26 Thread Zhanghao Chen
Thanks everyone. I'll start voting after the New Year's holiday if there's no 
further comment.

Best,
Zhanghao Chen

From: Benchao Li 
Sent: Friday, December 22, 2023 21:18
To: dev@flink.apache.org 
Subject: Re: [DISCUSS] FLIP-397: Add config options for administrator JVM 
options

+1 from my side,

I also met some scenarios that I wanted to set some JVM options by
default for all Flink jobs before, such as
'-XX:-DontCompileHugeMethods', without it, some generated big methods
won't be optimized in JVM C2 compiler, leading to poor performance.

Zhanghao Chen  于2023年11月27日周一 20:04写道:
>
> Hi devs,
>
> I'd like to start a discussion on FLIP-397: Add config options for 
> administrator JVM options [1].
>
> In production environments, users typically develop and operate their Flink 
> jobs through a managed platform. Users may need to add JVM options to their 
> Flink applications (e.g. to tune GC options). They typically use the 
> env.java.opts.x series of options to do so. Platform administrators also have 
> a set of JVM options to apply by default, e.g. to use JVM 17, enable GC 
> logging, or apply pretuned GC options, etc. Both use cases will need to set 
> the same series of options and will clobber one another. Similar issues have 
> been described in SPARK-23472 [2].
>
> Therefore, I propose adding a set of default JVM options for administrator 
> use that prepends the user-set extra JVM options.
>
> Looking forward to hearing from you.
>
> [1] 
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-397%3A+Add+config+options+for+administrator+JVM+options
> [2] https://issues.apache.org/jira/browse/SPARK-23472
>
> Best,
> Zhanghao Chen



--

Best,
Benchao Li


[jira] [Created] (FLINK-33948) Cleanup usage of org.apache.flink.table.api.dataview.MapView#MapView

2023-12-26 Thread Jacky Lau (Jira)
Jacky Lau created FLINK-33948:
-

 Summary: Cleanup usage of 
org.apache.flink.table.api.dataview.MapView#MapView
 Key: FLINK-33948
 URL: https://issues.apache.org/jira/browse/FLINK-33948
 Project: Flink
  Issue Type: Sub-task
  Components: Table SQL / Planner
Affects Versions: 1.19.0
Reporter: Jacky Lau
 Fix For: 1.19.0






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


Re: [DISCUSS] FLIP-406: Reorganize State & Checkpointing & Recovery Configuration

2023-12-26 Thread Zakelly Lan
Hi everyone,

Thanks all for your comments!

@Yanfei

> 1. For some state backends that do not support incremental checkpoint,
> how does the execution.checkpointing.incrementaloption take effect? Or
> is it better to put incremental under state.backend.xxx.incremental?
>
I'd rather not put the option for incremental checkpoint under the
'state.backend', since it is more about the checkpointing instead of state
accessing. Of course, the state backend may not necessarily do incremental
checkpoint as requested. If the state backend is not capable of taking
incremental cp, it is better to fallback to the full cp.

2. I'm a little worried that putting all configurations into
> `ExecutionCheckpointingOptions` will introduce some dependency
> problems. Some options would be used by flink-runtime module, but
> flink-runtime should not depend on flink-streaming-java. e.g.
> FLINK-28286[1].
> So, I prefer to move configurations to `CheckpointingOptions`, WDYT?
>

Yes, that's a very good point.  Moving to
`CheckpointingOptions`(flink-core) makes sense.

@Lijie

How about
> state.savepoints.dir -> execution.checkpointing.savepoint.dir
> state.checkpoints.dir -> execution.checkpointing.checkpoint.dir


Actually, I think the `checkpointing.checkpoint` may cause some confusion.
But I'm ok if others agree.
I'm wondering if `execution.checkpointing.savepoint-dir` would be better.
WDYT?

2. We changed the execution.checkpointing.local-copy' to
> 'execution.checkpointing.local-copy.enabled'. Should we also add "enabled"
> suffix for other boolean type configuration options ? For example,
> execution.checkpointing.incremental ->
> execution.checkpointing.incremental.enabled
>

Actually, the incremental cp is something like choosing a mode for doing
checkpoint instead of enabling a function. So I think an enumeration option
`execution.checkpointing.mode` which can be 'full' (default) or
'incremental' would be better, WDYT?
And @Rui Fan @Yanfei What do you think about this?


On Tue, Dec 26, 2023 at 5:15 PM Lijie Wang  wrote:

> Hi Zakelly,
>
> Thanks for driving the discussion.
>
> 1.
> >> But I'm not so sure since there is only one savepoint-related option.
> Maybe someone else could share some thoughts here.
>
> How about
> state.savepoints.dir -> execution.checkpointing.savepoint.dir
> state.checkpoints.dir -> execution.checkpointing.checkpoint.dir
>
> 2. We changed the execution.checkpointing.local-copy' to
> 'execution.checkpointing.local-copy.enabled'. Should we also add "enabled"
> suffix for other boolean type configuration options ? For example,
> execution.checkpointing.incremental ->
> execution.checkpointing.incremental.enabled
>
> In this way, the naming style of configuration options is unified, and it
> can avoid potential similar problems (for example, we may need to add more
> options for incremental checkpoint in the future).
>
> Best,
> Lijie
>
> Yanfei Lei  于2023年12月26日周二 12:05写道:
>
> > Hi Zakelly,
> >
> > Thank you for creating the FLIP and starting the discussion.
> >
> > The current arrangement of these options is indeed somewhat haphazard,
> > and the new arrangement looks much better. I have some questions about
> > the arrangement of some new configuration options:
> >
> > 1. For some state backends that do not support incremental checkpoint,
> > how does the execution.checkpointing.incrementaloption take effect? Or
> > is it better to put incremental under state.backend.xxx.incremental?
> >
> > 2. I'm a little worried that putting all configurations into
> > `ExecutionCheckpointingOptions` will introduce some dependency
> > problems. Some options would be used by flink-runtime module, but
> > flink-runtime should not depend on flink-streaming-java. e.g.
> > FLINK-28286[1].
> > So, I prefer to move configurations to `CheckpointingOptions`, WDYT?
> >
> > [1] https://issues.apache.org/jira/browse/FLINK-28286
> >
> > --
> > Best,
> > Yanfei
> >
> > Zakelly Lan  于2023年12月25日周一 21:14写道:
> > >
> > > Hi Rui Fan and Junrui,
> > >
> > > Thanks for the reminder! I agree to change the
> > > 'execution.checkpointing.local-copy' to
> > > 'execution.checkpointing.local-copy.enabled'.
> > >
> > > And for other suggestions Rui proposed:
> > >
> > > 1. How about execution.checkpointing.storage.type instead
> > > > of execution.checkpointing.storage?
> > >
> > >
> > > Ah, I missed something here. Actually I suggest we could merge the
> > current
> > > 'state.checkpoints.dir' and 'state.checkpoint-storage' into one URI
> > > configuration named 'execution.checkpointing.dir'. WDYT?
> > >
> > > 3. execution.checkpointing.savepoint.dir is a little weird.
> > > >
> > >
> > > Yes, I think it is better to make 'savepoint' and 'checkpoint' the same
> > > level. But I'm not so sure since there is only one savepoint-related
> > > option. Maybe someone else could share some thoughts here.
> > >
> > > 4. How about execution.recovery.claim-mode instead of
> > > > execution.recovery.mode?
> > > >
> > >
> > >  Agreed. That's more 

[jira] [Created] (FLINK-33947) Fix bugs in DelegatingConfiguration missed the prefix mapping

2023-12-26 Thread RocMarshal (Jira)
RocMarshal created FLINK-33947:
--

 Summary: Fix bugs in DelegatingConfiguration missed the prefix 
mapping 
 Key: FLINK-33947
 URL: https://issues.apache.org/jira/browse/FLINK-33947
 Project: Flink
  Issue Type: Bug
  Components: API / Core
Reporter: RocMarshal


It was resulted from 
[https://github.com/apache/flink/pull/23994#issuecomment-1869905090] 

-  Check and confirm other potential bug points

-  Fix the bugs about prefix key mapping when operating.



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


Re: [DISCUSS] FLIP-403: High Availability Services for OLAP Scenarios

2023-12-26 Thread Zhanghao Chen
Thanks for driving this effort, Yangze! The proposal overall LGTM. Other from 
the throughput enhancement in the OLAP scenario, the separation of leader 
election/discovery services and the metadata persistence services will also 
make the HA impl clearer and easier to maintain. Just a minor comment on 
naming: would it better to rename PersistentServices to PersistenceServices, as 
usually we put a noun before Services?

Best,
Zhanghao Chen

From: Yangze Guo 
Sent: Tuesday, December 19, 2023 17:33
To: dev 
Subject: [DISCUSS] FLIP-403: High Availability Services for OLAP Scenarios

Hi, there,

We would like to start a discussion thread on "FLIP-403: High
Availability Services for OLAP Scenarios"[1].

Currently, Flink's high availability service consists of two
mechanisms: leader election/retrieval services for JobManager and
persistent services for job metadata. However, these mechanisms are
set up in an "all or nothing" manner. In OLAP scenarios, we typically
only require leader election/retrieval services for JobManager
components since jobs usually do not have a restart strategy.
Additionally, the persistence of job states can negatively impact the
cluster's throughput, especially for short query jobs.

To address these issues, this FLIP proposes splitting the
HighAvailabilityServices into LeaderServices and PersistentServices,
and enable users to independently configure the high availability
strategies specifically related to jobs.

Please find more details in the FLIP wiki document [1]. Looking
forward to your feedback.

[1] 
https://cwiki.apache.org/confluence/display/FLINK/FLIP-403+High+Availability+Services+for+OLAP+Scenarios

Best,
Yangze Guo


[jira] [Created] (FLINK-33946) RocksDb sets setAvoidFlushDuringShutdown to true to speed up Task Cancel

2023-12-26 Thread Yue Ma (Jira)
Yue Ma created FLINK-33946:
--

 Summary: RocksDb sets setAvoidFlushDuringShutdown to true to speed 
up Task Cancel
 Key: FLINK-33946
 URL: https://issues.apache.org/jira/browse/FLINK-33946
 Project: Flink
  Issue Type: Improvement
  Components: Runtime / State Backends
Affects Versions: 1.19.0
Reporter: Yue Ma
 Fix For: 1.19.0


When a Job fails, the task needs to be canceled and re-deployed. 
RocksDBStatebackend will call RocksDB.close when disposing.


{code:java}
if (!shutting_down_.load(std::memory_order_acquire) &&
has_unpersisted_data_.load(std::memory_order_relaxed) &&
!mutable_db_options_.avoid_flush_during_shutdown) {
  if (immutable_db_options_.atomic_flush) {
autovector cfds;
SelectColumnFamiliesForAtomicFlush();
mutex_.Unlock();
Status s =
AtomicFlushMemTables(cfds, FlushOptions(), FlushReason::kShutDown);
s.PermitUncheckedError();  //**TODO: What to do on error?
mutex_.Lock();
  } else {
for (auto cfd : *versions_->GetColumnFamilySet()) {
  if (!cfd->IsDropped() && cfd->initialized() && !cfd->mem()->IsEmpty()) {
cfd->Ref();
mutex_.Unlock();
Status s = FlushMemTable(cfd, FlushOptions(), FlushReason::kShutDown);
s.PermitUncheckedError();  //**TODO: What to do on error?
mutex_.Lock();
cfd->UnrefAndTryDelete();
  }
}
  } {code}


By default (avoid_flush_during_shutdown=false) RocksDb requires FlushMemtable 
when Close. When the disk pressure is high or the Memtable is large, this 
process will be more time-consuming, which will cause the Task to get stuck in 
the Canceling stage and affect the speed of job Failover.
In fact, it is completely unnecessary to Flush memtable when Flink Task is 
Close, because the data can be replayed from Checkpoint. So we can set 
avoid_flush_during_shutdown to true to speed up Task Failover



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


[jira] [Created] (FLINK-33945) Cleanup usage of deprecated org.apache.flink.table.api.dataview.ListView#ListView

2023-12-26 Thread Jacky Lau (Jira)
Jacky Lau created FLINK-33945:
-

 Summary: Cleanup usage of deprecated 
org.apache.flink.table.api.dataview.ListView#ListView
 Key: FLINK-33945
 URL: https://issues.apache.org/jira/browse/FLINK-33945
 Project: Flink
  Issue Type: Sub-task
  Components: Table SQL / Planner
Affects Versions: 1.19.0
Reporter: Jacky Lau
 Fix For: 1.19.0






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


[jira] [Created] (FLINK-33944) Apache Flink: Process to restore more than one job on job manager startup from the respective savepoints

2023-12-26 Thread Vijay (Jira)
Vijay created FLINK-33944:
-

 Summary: Apache Flink: Process to restore more than one job on job 
manager startup from the respective savepoints
 Key: FLINK-33944
 URL: https://issues.apache.org/jira/browse/FLINK-33944
 Project: Flink
  Issue Type: Bug
  Components: Runtime / Checkpointing
Affects Versions: 1.18.0
Reporter: Vijay


 
We are using Flink (1.18) version for our Flink cluster. The job manager has 
been deployed in "Application mode" and we are looking for a process to restore 
multiple jobs (using their respective savepoint directories) when the job 
manager is started. Currently, we have the option to restore only one job while 
running "standalone-job.sh" using the --fromSavepoint and 
--allowNonRestoredState. However, we need a way to trigger multiple job 
executions via Java client.

Note: We are not using a Kubernetes native deployment, but we are using k8s 
standalone mode of deployment.

*Expected process:*
 # Before starting with the Flink/application image upgrade, trigger the 
savepoints for all the current running jobs.
 # Once the savepoints process completed for all jobs, will trigger the scale 
down of job manager and task manager instances.
 # Update the image version on the k8s deployment with the update application 
image.
 # After image version is updated, scale up the job manager and task manager.
 # We need a process to restore the previously running jobs from the savepoint 
dir and start all the jobs.



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


[jira] [Created] (FLINK-33943) Apache flink: Issues after configuring HA (using zookeeper setting)

2023-12-26 Thread Vijay (Jira)
Vijay created FLINK-33943:
-

 Summary: Apache flink: Issues after configuring HA (using 
zookeeper setting)
 Key: FLINK-33943
 URL: https://issues.apache.org/jira/browse/FLINK-33943
 Project: Flink
  Issue Type: Bug
  Components: Build System
Affects Versions: 1.18.0
 Environment: Flink version: 1.18

Zookeeper version: 3.7.2

Env: Custom flink docker image (with embedded application class) deployed over 
kubernetes (v1.26.11).

 
Reporter: Vijay


Hi Team,

Note: Not sure whether I have picked the right component while raising the 
issue.

Good Day. I am using Flink (1.18) version and zookeeper (3.7.2) for our flink 
cluster. Job manager has been deployed on "Application mode" and when HA is 
disabled (high-availability.type: NONE) we are able to start multiple jobs 
(using env.executeAsyn()) for a single application. But when I setup the 
Zookeeper as the HA type (high-availability.type: zookeeper), we are only 
seeing only one job is getting executed on the Flink dashboard. Following are 
the parameters setup for the Zookeeper based HA setup on the flink-conf.yaml. 
Please let us know if anyone has experienced similar issues and have any 
suggestions. Thanks in advance for your assistance.

Note: We are using a Streaming application and following are the 
flink-config.yaml configurations.
 # high-availability.storageDir: /opt/flink/data
 # high-availability.cluster-id: test
 # high-availability.zookeeper.quorum: localhost:2181
 # high-availability.type: zookeeper
 # high-availability.zookeeper.path.root: /dp/configs/flinkha



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


Re: [PROPOSAL] Contribute Flink CDC Connectors project to Apache Flink

2023-12-26 Thread Nicholas Jiang
Awesome! Big +1.

Regards,
Nicholas Jiang


On 2023/12/07 03:24:59 Leonard Xu wrote:
> Dear Flink devs,
> 
> As you may have heard, we at Alibaba (Ververica) are planning to donate CDC 
> Connectors for the Apache Flink project[1] to the Apache Flink community.
> 
> CDC Connectors for Apache Flink comprise a collection of source connectors 
> designed specifically for Apache Flink. These connectors[2] enable the 
> ingestion of changes from various databases using Change Data Capture (CDC), 
> most of these CDC connectors are powered by Debezium[3]. They support both 
> the DataStream API and the Table/SQL API, facilitating the reading of 
> database snapshots and continuous reading of transaction logs with 
> exactly-once processing, even in the event of failures.
> 
> 
> Additionally, in the latest version 3.0, we have introduced many long-awaited 
> features. Starting from CDC version 3.0, we've built a Streaming ELT 
> Framework available for streaming data integration. This framework allows 
> users to write their data synchronization logic in a simple YAML file, which 
> will automatically be translated into a Flink DataStreaming job. It 
> emphasizes optimizing the task submission process and offers advanced 
> functionalities such as whole database synchronization, merging sharded 
> tables, and schema evolution[4].
> 
> 
> I believe this initiative is a perfect match for both sides. For the Flink 
> community, it presents an opportunity to enhance Flink's competitive 
> advantage in streaming data integration, promoting the healthy growth and 
> prosperity of the Apache Flink ecosystem. For the CDC Connectors project, 
> becoming a sub-project of Apache Flink means being part of a neutral 
> open-source community, which can attract a more diverse pool of contributors.
> 
> Please note that the aforementioned points represent only some of our 
> motivations and vision for this donation. Specific future operations need to 
> be further discussed in this thread. For example, the sub-project name after 
> the donation; we hope to name it Flink-CDC aiming to streaming data 
> intergration through Apache Flink, following the naming convention of 
> Flink-ML; And this project is managed by a total of 8 maintainers, including 
> 3 Flink PMC members and 1 Flink Committer. The remaining 4 maintainers are 
> also highly active contributors to the Flink community, donating this project 
> to the Flink community implies that their permissions might be reduced. 
> Therefore, we may need to bring up this topic for further discussion within 
> the Flink PMC. Additionally, we need to discuss how to migrate existing users 
> and documents. We have a user group of nearly 10,000 people and a 
> multi-version documentation site need to migrate. We also need to plan for 
> the migration of CI/CD processes and other specifics. 
> 
> 
> While there are many intricate details that require implementation, we are 
> committed to progressing and finalizing this donation process.
> 
> 
> Despite being Flink’s most active ecological project (as evaluated by GitHub 
> metrics), it also boasts a significant user base. However, I believe it's 
> essential to commence discussions on future operations only after the 
> community reaches a consensus on whether they desire this donation.
> 
> 
> Really looking forward to hear what you think! 
> 
> 
> Best,
> Leonard (on behalf of the Flink CDC Connectors project maintainers)
> 
> [1] https://github.com/ververica/flink-cdc-connectors
> [2] 
> https://ververica.github.io/flink-cdc-connectors/master/content/overview/cdc-connectors.html
> [3] https://debezium.io
> [4] 
> https://ververica.github.io/flink-cdc-connectors/master/content/overview/cdc-pipeline.html


Re: [DISCUSS] FLIP-405: Migrate string configuration key to ConfigOption

2023-12-26 Thread Xintong Song
>
> Configuration has a `public  T get(ConfigOption option)` method.
> Could we remove all `Xxx getXxx(ConfigOption configOption)` methods?



Note: all `public void setXxx(ConfigOption key, Xxx value)` methods
> can be replaced with `public  Configuration set(ConfigOption option,
> T value)` as well.


+1


Best,

Xintong



On Tue, Dec 26, 2023 at 8:44 PM Xintong Song  wrote:

> These features don't have a public option, but they work. I'm not sure
>> whether these features are used by some advanced users.
>> Actually, I think some of them are valuable! For example:
>>
>> - ConfigConstants.YARN_CONTAINER_START_COMMAND_TEMPLATE
>>   allows users to define the start command of the yarn container.
>> - FileInputFormat.ENUMERATE_NESTED_FILES_FLAG allows
>>   flink job reads all files under the directory even if it has nested
>> directories.
>>
>> This FLIP focuses on the refactor option, I'm afraid these features are
>> used
>> in some production and removing these features will affect some flink
>> jobs.
>> So I prefer to keep these features, WDTY?
>>
>
> First of all, I don't think we should support any knobs that users can
> only learn how to use from reading Flink's internal codes. From this
> perspective, for existing string-keyed knobs that are not mentioned in any
> public documentation, yes we can argue that they are functioning, but we
> can also argue that they are not really exposed to users. That means
> migrating them to ConfigOption is not a pure refactor, but would make
> something that used to be hidden from users now exposed to users. For such
> options, I personally would lean toward not exposing them. If we consider
> them as already exposed, then process-wise there's no problem in
> deprecating some infrequently-used options and removing them in a major
> version bump, and if they are proved needed later we can add them back
> anytime. On the other hand, if we consider them as not yet exposed, then
> removing them later would be a breaking change.
>
>
> Secondly, I don't really come up with any cases where users need to tune
> these knobs. E.g., why would we allow users to customize the yarn container
> start command while we already provide `env.java.opts`? And what would be
> the problem if Flink just doesn't support nested files? And even worse,
> such knobs may provide chances for users to shoot themself in the foot.
> E.g., what if %jvmmem% is missing from a user-provided container start
> command? Admittedly, there might be a small fraction of advanced users that
> know how to use these knobs. However, those users usually have their own
> custom fork of Flink, and it should not be a big problem for them to build
> such abilities by themselves.
>
>
> Taking a step back, if we decide that some of the mentioned knobs are
> really useful, we should at least provide enough descriptions to help users
> understand when and how to use these options. E.g., the current description
> for yarn container command template is far from enough, which does not
> explain what the placeholders mean, what happens if some placeholders are
> missing or if an unknown placeholder is provided.
>
>
> Best,
>
> Xintong
>
>
>
> On Tue, Dec 26, 2023 at 7:39 PM Rui Fan <1996fan...@gmail.com> wrote:
>
>> In addition to what is written in FLIP, I found some methods of
>> Configuration
>> are not necessary. And I wanna hear more thoughts from all of you.
>>
>> Configuration has a `public  T get(ConfigOption option)` method.
>> Could we remove all `Xxx getXxx(ConfigOption configOption)` methods?
>> Such as:
>> - public int getInteger(ConfigOption configOption)
>> - public String getString(ConfigOption configOption)
>> - public long getLong(ConfigOption configOption)
>> - etc
>>
>> I prefer users and flink use `get(ConfigOption option)` instead of
>> `getXxx(ConfigOption configOption)` based on some reasons:
>>
>> 1. All callers can replace it directly without any extra effort.
>>   `T get(ConfigOption option)` method can replace all
>>   `Xxx getXxx(ConfigOption configOption)` methods directly.
>> 2. Callers can call get directly, and users or flink developers
>>   don't need to care about should they call getInteger or getString.
>> 3. Flink code is easier to maintain.
>> 4. `T get(ConfigOption option)` is designed later than
>> `Xxx getXxx(ConfigOption configOption)`, I guess if
>> `T get(ConfigOption option)` is designed first,
>>all `Xxx getXxx(ConfigOption configOption)` methods
>>   aren't needed.
>>
>> Note: all `public void setXxx(ConfigOption key, Xxx value)` methods
>> can be replaced with `public  Configuration set(ConfigOption option,
>> T value)` as well.
>>
>> If they can be marked as deprecated, only `public Xxx
>> getXxx(ConfigOption configOption, Xxx overrideDefault)`
>> is keeped after this FLIP. And the rest of getXxx or setXxx can be removed
>> in 2.0.
>>
>> Looking forward to everyone's feedback and suggestions, thank you!
>>
>> Best,
>> Rui
>>
>> On Tue, Dec 26, 2023 at 7:15 PM Rui 

Re: [DISCUSS] FLIP-405: Migrate string configuration key to ConfigOption

2023-12-26 Thread Xintong Song
>
> These features don't have a public option, but they work. I'm not sure
> whether these features are used by some advanced users.
> Actually, I think some of them are valuable! For example:
>
> - ConfigConstants.YARN_CONTAINER_START_COMMAND_TEMPLATE
>   allows users to define the start command of the yarn container.
> - FileInputFormat.ENUMERATE_NESTED_FILES_FLAG allows
>   flink job reads all files under the directory even if it has nested
> directories.
>
> This FLIP focuses on the refactor option, I'm afraid these features are
> used
> in some production and removing these features will affect some flink jobs.
> So I prefer to keep these features, WDTY?
>

First of all, I don't think we should support any knobs that users can only
learn how to use from reading Flink's internal codes. From this
perspective, for existing string-keyed knobs that are not mentioned in any
public documentation, yes we can argue that they are functioning, but we
can also argue that they are not really exposed to users. That means
migrating them to ConfigOption is not a pure refactor, but would make
something that used to be hidden from users now exposed to users. For such
options, I personally would lean toward not exposing them. If we consider
them as already exposed, then process-wise there's no problem in
deprecating some infrequently-used options and removing them in a major
version bump, and if they are proved needed later we can add them back
anytime. On the other hand, if we consider them as not yet exposed, then
removing them later would be a breaking change.


Secondly, I don't really come up with any cases where users need to tune
these knobs. E.g., why would we allow users to customize the yarn container
start command while we already provide `env.java.opts`? And what would be
the problem if Flink just doesn't support nested files? And even worse,
such knobs may provide chances for users to shoot themself in the foot.
E.g., what if %jvmmem% is missing from a user-provided container start
command? Admittedly, there might be a small fraction of advanced users that
know how to use these knobs. However, those users usually have their own
custom fork of Flink, and it should not be a big problem for them to build
such abilities by themselves.


Taking a step back, if we decide that some of the mentioned knobs are
really useful, we should at least provide enough descriptions to help users
understand when and how to use these options. E.g., the current description
for yarn container command template is far from enough, which does not
explain what the placeholders mean, what happens if some placeholders are
missing or if an unknown placeholder is provided.


Best,

Xintong



On Tue, Dec 26, 2023 at 7:39 PM Rui Fan <1996fan...@gmail.com> wrote:

> In addition to what is written in FLIP, I found some methods of
> Configuration
> are not necessary. And I wanna hear more thoughts from all of you.
>
> Configuration has a `public  T get(ConfigOption option)` method.
> Could we remove all `Xxx getXxx(ConfigOption configOption)` methods?
> Such as:
> - public int getInteger(ConfigOption configOption)
> - public String getString(ConfigOption configOption)
> - public long getLong(ConfigOption configOption)
> - etc
>
> I prefer users and flink use `get(ConfigOption option)` instead of
> `getXxx(ConfigOption configOption)` based on some reasons:
>
> 1. All callers can replace it directly without any extra effort.
>   `T get(ConfigOption option)` method can replace all
>   `Xxx getXxx(ConfigOption configOption)` methods directly.
> 2. Callers can call get directly, and users or flink developers
>   don't need to care about should they call getInteger or getString.
> 3. Flink code is easier to maintain.
> 4. `T get(ConfigOption option)` is designed later than
> `Xxx getXxx(ConfigOption configOption)`, I guess if
> `T get(ConfigOption option)` is designed first,
>all `Xxx getXxx(ConfigOption configOption)` methods
>   aren't needed.
>
> Note: all `public void setXxx(ConfigOption key, Xxx value)` methods
> can be replaced with `public  Configuration set(ConfigOption option,
> T value)` as well.
>
> If they can be marked as deprecated, only `public Xxx
> getXxx(ConfigOption configOption, Xxx overrideDefault)`
> is keeped after this FLIP. And the rest of getXxx or setXxx can be removed
> in 2.0.
>
> Looking forward to everyone's feedback and suggestions, thank you!
>
> Best,
> Rui
>
> On Tue, Dec 26, 2023 at 7:15 PM Rui Fan <1996fan...@gmail.com> wrote:
>
> > Thanks Xintong for the quick feedback and the good suggestions!
> >
> > > 1. I think the default value for `TASK_MANAGER_LOG_PATH_KEY` should be
> > "no
> > > default". We can explain in the description that if not configured,
> > > `System.getProperty("log.file")` will be used, but that is not a
> default
> > > value.
> >
> > Explain it in the description is fine for me.
> >
> > > 2. So I wonder if we can simply mark them as deprecated and remove in
> > 2.0.
> >
> > 

Re: [DISCUSS] FLIP-405: Migrate string configuration key to ConfigOption

2023-12-26 Thread Rui Fan
In addition to what is written in FLIP, I found some methods of
Configuration
are not necessary. And I wanna hear more thoughts from all of you.

Configuration has a `public  T get(ConfigOption option)` method.
Could we remove all `Xxx getXxx(ConfigOption configOption)` methods?
Such as:
- public int getInteger(ConfigOption configOption)
- public String getString(ConfigOption configOption)
- public long getLong(ConfigOption configOption)
- etc

I prefer users and flink use `get(ConfigOption option)` instead of
`getXxx(ConfigOption configOption)` based on some reasons:

1. All callers can replace it directly without any extra effort.
  `T get(ConfigOption option)` method can replace all
  `Xxx getXxx(ConfigOption configOption)` methods directly.
2. Callers can call get directly, and users or flink developers
  don't need to care about should they call getInteger or getString.
3. Flink code is easier to maintain.
4. `T get(ConfigOption option)` is designed later than
`Xxx getXxx(ConfigOption configOption)`, I guess if
`T get(ConfigOption option)` is designed first,
   all `Xxx getXxx(ConfigOption configOption)` methods
  aren't needed.

Note: all `public void setXxx(ConfigOption key, Xxx value)` methods
can be replaced with `public  Configuration set(ConfigOption option,
T value)` as well.

If they can be marked as deprecated, only `public Xxx
getXxx(ConfigOption configOption, Xxx overrideDefault)`
is keeped after this FLIP. And the rest of getXxx or setXxx can be removed
in 2.0.

Looking forward to everyone's feedback and suggestions, thank you!

Best,
Rui

On Tue, Dec 26, 2023 at 7:15 PM Rui Fan <1996fan...@gmail.com> wrote:

> Thanks Xintong for the quick feedback and the good suggestions!
>
> > 1. I think the default value for `TASK_MANAGER_LOG_PATH_KEY` should be
> "no
> > default". We can explain in the description that if not configured,
> > `System.getProperty("log.file")` will be used, but that is not a default
> > value.
>
> Explain it in the description is fine for me.
>
> > 2. So I wonder if we can simply mark them as deprecated and remove in
> 2.0.
>
> These features don't have a public option, but they work. I'm not sure
> whether these features are used by some advanced users.
> Actually, I think some of them are valuable! For example:
>
> - ConfigConstants.YARN_CONTAINER_START_COMMAND_TEMPLATE
>   allows users to define the start command of the yarn container.
> - FileInputFormat.ENUMERATE_NESTED_FILES_FLAG allows
>   flink job reads all files under the directory even if it has nested
> directories.
>
> This FLIP focuses on the refactor option, I'm afraid these features are
> used
> in some production and removing these features will affect some flink jobs.
> So I prefer to keep these features, WDTY?
>
> > 3. Simply saying "getting / setting value with string key is discouraged"
> > in JavaDoc of get/setString is IMHO a bit confusing. People may have the
> > question why would we keep the discouraged interfaces at all. I would
> > suggest the following:
> > ```
> > We encourage users and developers to always use ConfigOption for getting
> /
> > setting the configurations if possible, for its rich description, type,
> > default-value and other supports. The string-key-based getter / setter
> > should only be used when ConfigOption is not applicable, e.g., the key is
> > programmatically generated in runtime.
> > ```
>
> Suggested comment is good for me, and I'd like to hear the thought from
> Xuannan
> who wrote the original comment.
>
> Best,
> Rui
>
> On Tue, Dec 26, 2023 at 5:26 PM Xintong Song 
> wrote:
>
>> Thanks for the efforts, Rui and Xuannan.
>>
>> I think it's a good idea to migrate string-key configuration accesses to
>> ConfigOption-s in general.
>>
>> I have a few suggestions / questions regarding the FLIP.
>>
>> 1. I think the default value for `TASK_MANAGER_LOG_PATH_KEY` should be "no
>> default". We can explain in the description that if not configured,
>> `System.getProperty("log.file")` will be used, but that is not a default
>> value.
>>
>> 2. I wonder if the following string-keys can be simply removed? They are
>> neither set by Flink, nor documented anywhere (AFAIK) so that users know
>> how to set them. All of them were introduced a long time ago, require
>> significant knowledge and familiarity about Flink internals and low level
>> details in order to use, and some of them are even private. So I wonder if
>> we can simply mark them as deprecated and remove in 2.0.
>> - ConfigConstants.YARN_CONTAINER_START_COMMAND_TEMPLATE
>> - FileInputFormat.FILE_PARAMETER_KEY
>> - FileInputFormat.ENUMERATE_NESTED_FILES_FLAG
>> - FileOutputFormat.FILE_PARAMETER_KEY
>> - BinaryInputFormat.BLOCK_SIZE_PARAMETER_KEY
>> - BinaryOutputFormat.BLOCK_SIZE_PARAMETER_KEY
>>
>> 3. Simply saying "getting / setting value with string key is discouraged"
>> in JavaDoc of get/setString is IMHO a bit confusing. People may have the
>> question why would we keep the discouraged interfaces at 

Re: [DISCUSS] FLIP-405: Migrate string configuration key to ConfigOption

2023-12-26 Thread Rui Fan
Thanks Xintong for the quick feedback and the good suggestions!

> 1. I think the default value for `TASK_MANAGER_LOG_PATH_KEY` should be "no
> default". We can explain in the description that if not configured,
> `System.getProperty("log.file")` will be used, but that is not a default
> value.

Explain it in the description is fine for me.

> 2. So I wonder if we can simply mark them as deprecated and remove in 2.0.

These features don't have a public option, but they work. I'm not sure
whether these features are used by some advanced users.
Actually, I think some of them are valuable! For example:

- ConfigConstants.YARN_CONTAINER_START_COMMAND_TEMPLATE
  allows users to define the start command of the yarn container.
- FileInputFormat.ENUMERATE_NESTED_FILES_FLAG allows
  flink job reads all files under the directory even if it has nested
directories.

This FLIP focuses on the refactor option, I'm afraid these features are
used
in some production and removing these features will affect some flink jobs.
So I prefer to keep these features, WDTY?

> 3. Simply saying "getting / setting value with string key is discouraged"
> in JavaDoc of get/setString is IMHO a bit confusing. People may have the
> question why would we keep the discouraged interfaces at all. I would
> suggest the following:
> ```
> We encourage users and developers to always use ConfigOption for getting /
> setting the configurations if possible, for its rich description, type,
> default-value and other supports. The string-key-based getter / setter
> should only be used when ConfigOption is not applicable, e.g., the key is
> programmatically generated in runtime.
> ```

Suggested comment is good for me, and I'd like to hear the thought from
Xuannan
who wrote the original comment.

Best,
Rui

On Tue, Dec 26, 2023 at 5:26 PM Xintong Song  wrote:

> Thanks for the efforts, Rui and Xuannan.
>
> I think it's a good idea to migrate string-key configuration accesses to
> ConfigOption-s in general.
>
> I have a few suggestions / questions regarding the FLIP.
>
> 1. I think the default value for `TASK_MANAGER_LOG_PATH_KEY` should be "no
> default". We can explain in the description that if not configured,
> `System.getProperty("log.file")` will be used, but that is not a default
> value.
>
> 2. I wonder if the following string-keys can be simply removed? They are
> neither set by Flink, nor documented anywhere (AFAIK) so that users know
> how to set them. All of them were introduced a long time ago, require
> significant knowledge and familiarity about Flink internals and low level
> details in order to use, and some of them are even private. So I wonder if
> we can simply mark them as deprecated and remove in 2.0.
> - ConfigConstants.YARN_CONTAINER_START_COMMAND_TEMPLATE
> - FileInputFormat.FILE_PARAMETER_KEY
> - FileInputFormat.ENUMERATE_NESTED_FILES_FLAG
> - FileOutputFormat.FILE_PARAMETER_KEY
> - BinaryInputFormat.BLOCK_SIZE_PARAMETER_KEY
> - BinaryOutputFormat.BLOCK_SIZE_PARAMETER_KEY
>
> 3. Simply saying "getting / setting value with string key is discouraged"
> in JavaDoc of get/setString is IMHO a bit confusing. People may have the
> question why would we keep the discouraged interfaces at all. I would
> suggest the following:
> ```
> We encourage users and developers to always use ConfigOption for getting /
> setting the configurations if possible, for its rich description, type,
> default-value and other supports. The string-key-based getter / setter
> should only be used when ConfigOption is not applicable, e.g., the key is
> programmatically generated in runtime.
> ```
>
> WDYT?
>
> Best,
>
> Xintong
>
>
>
> On Tue, Dec 26, 2023 at 4:12 PM Rui Fan <1996fan...@gmail.com> wrote:
>
> > Hi all,
> >
> > Xuannan(cced) and I would like to start a discussion on FLIP-405:
> > FLIP-405: Migrate string configuration key to ConfigOption[1].
> >
> > As Flink progresses to 2.0, we want to enhance the user experience
> > with the existing configuration. In FLIP-77, Flink introduced
> ConfigOption
> > with DataType and strongly encourage users to utilize ConfigOption
> > instead of string keys for accessing and setting Flink configurations.
> > Presently, many string configuration keys have been deprecated and
> > replaced with ConfigOptions; however, some string configuration
> > keys are still in use.
> >
> > To ensure a better experience with the existing configuration in Flink
> > 2.0,
> > this FLIP will migrate all user-facing string configuration keys to
> > ConfigOptions.
> > Additionally, we want to modify the Configuration infrastructure to
> > promote the use of ConfigOption over string configuration keys
> > among developers and users. It's mentioned in a preview thread[2].
> >
> > Looking forward to everyone's feedback and suggestions, thank you!
> >
> > [1] https://cwiki.apache.org/confluence/x/6Yr5E
> > [2] https://lists.apache.org/thread/zzsf7glfcdjcjm1hfo1xdwc6jp37nb3m
> >
> > Best,
> > Rui
> >
>


Re: [DISCUSS] FLIP-405: Migrate string configuration key to ConfigOption

2023-12-26 Thread Xintong Song
Thanks for the efforts, Rui and Xuannan.

I think it's a good idea to migrate string-key configuration accesses to
ConfigOption-s in general.

I have a few suggestions / questions regarding the FLIP.

1. I think the default value for `TASK_MANAGER_LOG_PATH_KEY` should be "no
default". We can explain in the description that if not configured,
`System.getProperty("log.file")` will be used, but that is not a default
value.

2. I wonder if the following string-keys can be simply removed? They are
neither set by Flink, nor documented anywhere (AFAIK) so that users know
how to set them. All of them were introduced a long time ago, require
significant knowledge and familiarity about Flink internals and low level
details in order to use, and some of them are even private. So I wonder if
we can simply mark them as deprecated and remove in 2.0.
- ConfigConstants.YARN_CONTAINER_START_COMMAND_TEMPLATE
- FileInputFormat.FILE_PARAMETER_KEY
- FileInputFormat.ENUMERATE_NESTED_FILES_FLAG
- FileOutputFormat.FILE_PARAMETER_KEY
- BinaryInputFormat.BLOCK_SIZE_PARAMETER_KEY
- BinaryOutputFormat.BLOCK_SIZE_PARAMETER_KEY

3. Simply saying "getting / setting value with string key is discouraged"
in JavaDoc of get/setString is IMHO a bit confusing. People may have the
question why would we keep the discouraged interfaces at all. I would
suggest the following:
```
We encourage users and developers to always use ConfigOption for getting /
setting the configurations if possible, for its rich description, type,
default-value and other supports. The string-key-based getter / setter
should only be used when ConfigOption is not applicable, e.g., the key is
programmatically generated in runtime.
```

WDYT?

Best,

Xintong



On Tue, Dec 26, 2023 at 4:12 PM Rui Fan <1996fan...@gmail.com> wrote:

> Hi all,
>
> Xuannan(cced) and I would like to start a discussion on FLIP-405:
> FLIP-405: Migrate string configuration key to ConfigOption[1].
>
> As Flink progresses to 2.0, we want to enhance the user experience
> with the existing configuration. In FLIP-77, Flink introduced ConfigOption
> with DataType and strongly encourage users to utilize ConfigOption
> instead of string keys for accessing and setting Flink configurations.
> Presently, many string configuration keys have been deprecated and
> replaced with ConfigOptions; however, some string configuration
> keys are still in use.
>
> To ensure a better experience with the existing configuration in Flink
> 2.0,
> this FLIP will migrate all user-facing string configuration keys to
> ConfigOptions.
> Additionally, we want to modify the Configuration infrastructure to
> promote the use of ConfigOption over string configuration keys
> among developers and users. It's mentioned in a preview thread[2].
>
> Looking forward to everyone's feedback and suggestions, thank you!
>
> [1] https://cwiki.apache.org/confluence/x/6Yr5E
> [2] https://lists.apache.org/thread/zzsf7glfcdjcjm1hfo1xdwc6jp37nb3m
>
> Best,
> Rui
>


Re: [SUMMARY] Flink 1.19 Release Sync 12/12/2023

2023-12-26 Thread Lincoln Lee
Hi devs,

Happy holidays to you all!

Since most RMs are not available for today‘s meeting, and there were no new
attendees after waiting 10min at google meet, today's sync was skipped.
Here's the brief summary for 1.19 release:

- *Features & issues tracking*
  There're 5 weeks until the feature freeze date, so far we've had 27
flips/features and 4 of which have been completed. It is encouraged to
continuously updating the 1.19 wiki page[1] for contributors.

- *Blockers*
  -  [closed] FLINK-33641[2] the new JUnit5 failures. Thanks to
Jiabao, Sergey and Matthias for tracing and fixing it!

- *Sync meeting* (https://meet.google.com/vcx-arzs-trv)
  The next scheduled release time sync is *Jan 9, 2024. *We encourage
attendees to fill out the topics to be discussed at the bottom of 1.19 wiki
page[1] a day in advance, make it easier for everyone to understand the
background of the topics.

[1] https://cwiki.apache.org/confluence/display/FLINK/1.19+Release
[2] https://issues.apache.org/jira/browse/FLINK-33641

Best,
Yun, Jing, Martijn and Lincoln

Lincoln Lee  于2023年12月13日周三 20:39写道:

> Hi devs,
>
> This is the summary of the 3rd release sync of Flink 1.19:
>
> - *Features & issues tracking*
>   There're 7 weeks until the feature freeze date, so far we've had 23
> flips/features and the status looks good. It is encouraged to continuously
> updating the 1.19 wiki page[1] for contributors.
>   - FLINK-33528 Externalize Python connector code[2], volunteers on the
> related connectors are welcome!
>   - Voting for flink-shaded[3] version 1.18.0 is still open, binding votes
> are needed!
>
> - *Blockers*
>   - [closed] external connectors compilation failure cause by FLINK-25857,
> discussion thread[3]
>   - [potential blocker] the new JUnit5 fails to delete a directory on AZP
> for various table-planner tests FLINK-33641[4]
>
> - *Sync meeting* (https://meet.google.com/vcx-arzs-trv)
>   The next scheduled release time sync is *December 26, 2023*, but given
> that it falls during the Christmas holidays, we'd like to hear your
> suggestions on whether to skip this sync?
>   If we don't receive feedback by *December 25*, the sync will go on as
> usual.
>
> Lastly, we encourage attendees to fill out the topics to be discussed at
> the bottom of 1.19 wiki page[1] a day in advance, make it easier for
> everyone to understand the background of the topics.
>
> [1] https://cwiki.apache.org/confluence/display/FLINK/1.19+Release
> [2] https://issues.apache.org/jira/browse/FLINK-33528
> [3] https://lists.apache.org/thread/h6nkgth838dlh5s90sd95zd6hlsxwx57
> [4] https://issues.apache.org/jira/browse/FLINK-33641
>
> Best,
> Yun, Jing, Martijn and Lincoln
>


Re: [DISCUSS] FLIP-406: Reorganize State & Checkpointing & Recovery Configuration

2023-12-26 Thread Lijie Wang
Hi Zakelly,

Thanks for driving the discussion.

1.
>> But I'm not so sure since there is only one savepoint-related option.
Maybe someone else could share some thoughts here.

How about
state.savepoints.dir -> execution.checkpointing.savepoint.dir
state.checkpoints.dir -> execution.checkpointing.checkpoint.dir

2. We changed the execution.checkpointing.local-copy' to
'execution.checkpointing.local-copy.enabled'. Should we also add "enabled"
suffix for other boolean type configuration options ? For example,
execution.checkpointing.incremental ->
execution.checkpointing.incremental.enabled

In this way, the naming style of configuration options is unified, and it
can avoid potential similar problems (for example, we may need to add more
options for incremental checkpoint in the future).

Best,
Lijie

Yanfei Lei  于2023年12月26日周二 12:05写道:

> Hi Zakelly,
>
> Thank you for creating the FLIP and starting the discussion.
>
> The current arrangement of these options is indeed somewhat haphazard,
> and the new arrangement looks much better. I have some questions about
> the arrangement of some new configuration options:
>
> 1. For some state backends that do not support incremental checkpoint,
> how does the execution.checkpointing.incrementaloption take effect? Or
> is it better to put incremental under state.backend.xxx.incremental?
>
> 2. I'm a little worried that putting all configurations into
> `ExecutionCheckpointingOptions` will introduce some dependency
> problems. Some options would be used by flink-runtime module, but
> flink-runtime should not depend on flink-streaming-java. e.g.
> FLINK-28286[1].
> So, I prefer to move configurations to `CheckpointingOptions`, WDYT?
>
> [1] https://issues.apache.org/jira/browse/FLINK-28286
>
> --
> Best,
> Yanfei
>
> Zakelly Lan  于2023年12月25日周一 21:14写道:
> >
> > Hi Rui Fan and Junrui,
> >
> > Thanks for the reminder! I agree to change the
> > 'execution.checkpointing.local-copy' to
> > 'execution.checkpointing.local-copy.enabled'.
> >
> > And for other suggestions Rui proposed:
> >
> > 1. How about execution.checkpointing.storage.type instead
> > > of execution.checkpointing.storage?
> >
> >
> > Ah, I missed something here. Actually I suggest we could merge the
> current
> > 'state.checkpoints.dir' and 'state.checkpoint-storage' into one URI
> > configuration named 'execution.checkpointing.dir'. WDYT?
> >
> > 3. execution.checkpointing.savepoint.dir is a little weird.
> > >
> >
> > Yes, I think it is better to make 'savepoint' and 'checkpoint' the same
> > level. But I'm not so sure since there is only one savepoint-related
> > option. Maybe someone else could share some thoughts here.
> >
> > 4. How about execution.recovery.claim-mode instead of
> > > execution.recovery.mode?
> > >
> >
> >  Agreed. That's more accurate.
> >
> >
> > Many thanks for your suggestions!
> >
> > Best,
> > Zakelly
> >
> > On Mon, Dec 25, 2023 at 8:18 PM Junrui Lee  wrote:
> >
> > > Hi Zakelly,
> > >
> > > Thanks for driving this. I agree that the proposed restructuring of the
> > > configuration options is largely positive. It will make understanding
> and
> > > working with Flink configurations more intuitive.
> > >
> > > Most of the proposed changes look great. Just a heads-up, as Rui Fan
> > > mentioned, Flink currently requires that no configOption's key be the
> > > prefix of another to avoid issues when we eventually adopt a standard
> YAML
> > > parser, as detailed in FLINK-29372 (
> > > https://issues.apache.org/jira/browse/FLINK-29372). Therefore, it's
> better
> > > to change the key 'execution.checkpointing.local-copy' because it
> serves as
> > > a prefix to the key 'execution.checkpointing.local-copy.dir'.
> > >
> > > Best regards,
> > > Junrui
> > >
> > > Rui Fan <1996fan...@gmail.com> 于2023年12月25日周一 19:11写道:
> > >
> > > > Hi Zakelly,
> > > >
> > > > Thank you for driving this proposal!
> > > >
> > > > Overall good for me. I have some questions about these names.
> > > >
> > > > 1. How about execution.checkpointing.storage.type instead of
> > > > execution.checkpointing.storage?
> > > >
> > > > It's similar to state.backend.type.
> > > >
> > > > 2. How about execution.checkpointing.local-copy.enabled instead of
> > > > execution.checkpointing.local-copy?
> > > >
> > > > You added a new option: execution.checkpointing.local-copy.dir.
> > > > IIUC, one option name shouldn't be the prefix of other options.
> > > > If you add a new option execution.checkpointing.local-copy,
> > > > flink CI will fail directly.
> > > >
> > > > 3. execution.checkpointing.savepoint.dir is a little weird.
> > > >
> > > > For old options: state.savepoints.dir and state.checkpoints.dir,
> > > > the savepoint and checkpoint are the same level. It means
> > > > it's a checkpoint or savepoint.
> > > >
> > > > The new option execution.checkpointing.dir is fine for me.
> > > > However, execution.checkpointing.savepoint.dir is a little weird.
> > > > I don't know which name is better now. Let us think about it 

[jira] [Created] (FLINK-33942) DelegatingConfiguration misses the perfix for some methods

2023-12-26 Thread Rui Fan (Jira)
Rui Fan created FLINK-33942:
---

 Summary: DelegatingConfiguration misses the perfix for some methods
 Key: FLINK-33942
 URL: https://issues.apache.org/jira/browse/FLINK-33942
 Project: Flink
  Issue Type: Bug
  Components: Runtime / Configuration
Reporter: Rui Fan
Assignee: Rui Fan


* 
DelegatingConfiguration#getInteger(org.apache.flink.configuration.ConfigOption,
 int)
 * 
DelegatingConfiguration#getLong(org.apache.flink.configuration.ConfigOption,
 long)
 * 
org.apache.flink.configuration.DelegatingConfiguration#getBoolean(org.apache.flink.configuration.ConfigOption,
 boolean)
 * 
org.apache.flink.configuration.DelegatingConfiguration#getFloat(org.apache.flink.configuration.ConfigOption,
 float)
 * 
org.apache.flink.configuration.DelegatingConfiguration#getDouble(org.apache.flink.configuration.ConfigOption,
 double)



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


[Discuss] FLIP-407: Improve Flink Client performance in interactive scenarios

2023-12-26 Thread xiangyu feng
Hi devs,

I'm opening this thread to discuss FLIP-407: Improve Flink Client
performance in interactive scenarios. The POC test results and design doc
can be found at: FLIP-407

.

Currently, Flink Client is mainly designed for one time interaction with
the Flink Cluster. All the resources(http connections, threads, ha
services) and instances(ClusterDescriptor, ClusterClient, RestClient) are
created and recycled for each interaction. This works well when users do
not need to interact frequently with Flink Cluster and also saves resource
usage since resources are recycled immediately after each usage.

However, in OLAP or StreamingWarehouse scenarios, users might submit
interactive jobs to a dedicated Flink Session Cluster very often. In this
case, we find that for short queries that can finish in less than 1s in
Flink Cluster will still have E2E latency greater than 2s. Hence, we
propose this FLIP to improve the Flink Client performance in this scenario.
This could also improve the user experience when using session debug mode.

The major change in this FLIP is that there will be a new introduced option
*'execution.interactive-client'*. When this option is enabled, Flink
Client will reuse all the necessary resources to improve interactive
performance, including: HA Services, HTTP connections, threads and all
kinds of instances related to a long-running Flink Cluster. The default
value of this option will be false, then Flink Client will behave as before.

Also, this FLIP proposed a configurable RetryStrategy when fetching results
from client-side to Flink Cluster. In interactive scenarios, this can save
more than 15% of TM CPU usage without performance degradation.

Looking forward to your feedback, thanks.

Best regards,
Xiangyu


[DISCUSS] FLIP-405: Migrate string configuration key to ConfigOption

2023-12-26 Thread Rui Fan
Hi all,

Xuannan(cced) and I would like to start a discussion on FLIP-405:
FLIP-405: Migrate string configuration key to ConfigOption[1].

As Flink progresses to 2.0, we want to enhance the user experience
with the existing configuration. In FLIP-77, Flink introduced ConfigOption
with DataType and strongly encourage users to utilize ConfigOption
instead of string keys for accessing and setting Flink configurations.
Presently, many string configuration keys have been deprecated and
replaced with ConfigOptions; however, some string configuration
keys are still in use.

To ensure a better experience with the existing configuration in Flink 2.0,
this FLIP will migrate all user-facing string configuration keys to
ConfigOptions.
Additionally, we want to modify the Configuration infrastructure to
promote the use of ConfigOption over string configuration keys
among developers and users. It's mentioned in a preview thread[2].

Looking forward to everyone's feedback and suggestions, thank you!

[1] https://cwiki.apache.org/confluence/x/6Yr5E
[2] https://lists.apache.org/thread/zzsf7glfcdjcjm1hfo1xdwc6jp37nb3m

Best,
Rui