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

2024-01-10 Thread Hang Ruan
Hi, Sergey.

Thanks for the quick reply.

I try to package it in other pc with jdk8 and it succeeds. Please ignore
it. It seems like some errors in my environment.

Best,
Hang

Sergey Nuyanzin  于2024年1月11日周四 14:31写道:

> Hi Hang
>
> thanks for checking
> yes, it could be packaged with jdk8, moreover jdk8 is checked in ci
> for instance here ci for the commit tagged with v3.0.0-rc1 [1]
>
> the strange thing in the output that you've provided is
> >org.apache.flink:flink-connector-hive_2.12:jar:3.0.0: Could not find
> > artifact jdk.tools:jdk.tools:jar:1.8 at specified path /Library/Internet
> > Plug-Ins/JavaAppletPlugin.plugin/Contents/Home/../lib/tools.jar
>
> there are no such dependencies in poms,
> could it happen that there is some specific configuration on the machine
> you used for that?
> Can you please check it on another setup?
>
>
> [1] https://github.com/apache/flink-connector-hive/actions/runs/7479158667
>
>
> On Thu, Jan 11, 2024 at 4:44 AM Hang Ruan  wrote:
>
> > Hi, Sergey Nuyanzin.
> >
> > Thanks for driving this.
> >
> > I try to package the source with jdk8 and it will cause an error as
> > follows.
> >
> > [INFO]
> > 
> > [INFO] BUILD FAILURE
> > [INFO]
> > 
> > [INFO] Total time:  4.621 s
> > [INFO] Finished at: 2024-01-11T11:34:30+08:00
> > [INFO]
> > 
> > [ERROR] Failed to execute goal on project flink-connector-hive_2.12:
> Could
> > not resolve dependencies for project
> > org.apache.flink:flink-connector-hive_2.12:jar:3.0.0: Could not find
> > artifact jdk.tools:jdk.tools:jar:1.8 at specified path /Library/Internet
> > Plug-Ins/JavaAppletPlugin.plugin/Contents/Home/../lib/tools.jar -> [Help
> 1]
> > [ERROR]
> > [ERROR] To see the full stack trace of the errors, re-run Maven with the
> -e
> > switch.
> > [ERROR] Re-run Maven using the -X switch to enable full debug logging.
> > [ERROR]
> > [ERROR] For more information about the errors and possible solutions,
> > please read the following articles:
> > [ERROR] [Help 1]
> >
> >
> http://cwiki.apache.org/confluence/display/MAVEN/DependencyResolutionException
> > [ERROR]
> > [ERROR] After correcting the problems, you can resume the build with the
> > command
> > [ERROR]   mvn  -rf :flink-connector-hive_2.12
> >
> > I see that the 'Building the Apache Flink Hive Connector from Source'
> part
> > in README requires the Java 11. I am not sure whether this could be
> treated
> > as an error.
> > Does the flink-connector-hive support to be packaged with jdk8 now?
> >
> > Best,
> > Hang
> >
> > Jiabao Sun  于2024年1月11日周四 11:35写道:
> >
> > > +1 (non-binding)
> > >
> > > - Validated checksum hash
> > > - Verified signature
> > > - Verified web PR
> > > - Verified tags
> > >
> > > Best,
> > > Jiabao
> > >
> > >
> > > > 2024年1月11日 11:25,Hang Ruan  写道:
> > > >
> > > > Sorry that I make a mistake. I build the source with Maven and jdk11.
> > > >
> > > > Best,
> > > > Hang
> > > >
> > > > Hang Ruan  于2024年1月11日周四 11:13写道:
> > > >
> > > >> +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 web PR
> > > >> - Verified that the flink-connector-base is not packaged in hive
> > > connector
> > > >>
> > > >> Best,
> > > >> Hang
> > > >>
> > > >> Sergey Nuyanzin  于2024年1月11日周四 06:19写道:
> > > >>
> > > >>> Hi everyone,
> > > >>> Please review and vote on the release candidate #1 for the version
> > > 3.0.0,
> > > >>> as follows:
> > > >>> [ ] +1, Approve the release
> > > >>> [ ] -1, Do not approve the release (please provide specific
> comments)
> > > >>>
> > > >>> This version is compatible with Flink 1.18.x
> > > >>>
> > > >>> 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 F752 9FAE 2481 1A5C
> > 0DF3
> > > >>> CA74 1596 BBF0 7268 35D8 [3],
> > > >>> * all artifacts to be deployed to the Maven Central Repository [4],
> > > >>> * source code tag v3.0.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.
> > > >>>
> > > >>> Thanks,
> > > >>> Release Manager
> > > >>>
> > > >>> [1]
> https://issues.apache.org/jira/projects/FLINK/versions/12352591
> > > >>> [2]
> > > >>>
> > > >>>
> > >
> >
> https://dist.apache.org/repos/dist/dev/flink/flink-connector-hive-3.0.0-rc1
> > > >>> [3] https://dist.apache.org/repos/dist/release/flink/KEYS
> > > >>> [4]
> > > >>>
> > >
> 

Re: [DISCUSS] FLIP-414: Support Retry Mechanism in RocksDBStateDataTransfer

2024-01-10 Thread yue ma
Thanks for driving this effort, xiangyu!
The proposal overall LGTM.
I just have a small question. There are other places in Flink that interact
with external storage. Should we consider adding a general retry mechanism
to them?

xiangyu feng  于2024年1月8日周一 11:31写道:

> Hi devs,
>
> I'm opening this thread to discuss FLIP-414: Support Retry Mechanism in
> RocksDBStateDataTransfer[1].
>
> Currently, there is no retry mechanism for downloading and uploading
> RocksDB state files. Any jittering of remote filesystem might lead to a
> checkpoint failure. By supporting retry mechanism in
> `RocksDBStateDataTransfer`, we can significantly reduce the failure rate of
> checkpoint during asynchronous phrase.
>
> To make this retry mechanism configurable, we have introduced two options
> in this FLIP: `state.backend.rocksdb.checkpoint.transfer.retry.times` and `
> state.backend.rocksdb.checkpoint.transfer.retry.interval`. The default
> behavior remains to be no retry will be performed in order to be consistent
> with the original behavior.
>
> Looking forward to your feedback, thanks.
>
> [1]
>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-414%3A+Support+Retry+Mechanism+in+RocksDBStateDataTransfer
>
> Best regards,
> Xiangyu Feng
>


-- 
Best,
Yue


Re: [DISCUSS] FLIP 411: Chaining-agnostic Operator ID generation for improved state compatibility on parallelism change

2024-01-10 Thread Zhanghao Chen
Hi Yu,

I haven't thought too much about the compatibility design before. By the nature 
of the problem, it's impossible to make V3 compatible with V2, what we can do 
is to somewhat better inform users when switching the hasher, but I don't have 
any good idea so far. Do you have any suggestions on this?

Best,
Zhanghao Chen

From: Yu Chen 
Sent: Thursday, January 11, 2024 13:52
To: dev@flink.apache.org 
Cc: Piotr Nowojski ; zhanghao.c...@outlook.com 

Subject: Re: [DISCUSS] FLIP 411: Chaining-agnostic Operator ID generation for 
improved state compatibility on parallelism change

Hi Zhanghao,

Thanks for driving this, that’s really painful for us when we need to switch 
config `pipeline.operator-chaining`.

But I have a Concern, according to FLIP description, modifying `isChainable` 
related code in `StreamGraphHasherV2` will cause the generated operator id to 
be changed, which will result in the user unable to recover from the old state 
(old and new Operator IDs can't be mapped).
Therefore switching Hasher strategy (V2->V3 or V3->V2) will lead to an 
incompatibility, is there any relevant compatibility design considered?

Best,
Yu Chen

2024年1月10日 10:25,Zhanghao Chen  写道:

Hi David,

Thanks for the comments. AFAIK, unaligned checkpoints are disabled for 
pointwise connections according to [1], let's wait Piotr for confirmation. The 
issue itself is not directly related to this proposal as well. If a user 
manually specifies UIDs for each of the chained operators and has unaligned 
checkpoints enabled, we will encounter the same issue if they decide to break 
the chain on a later restart and try to recover from a retained cp.

[1] 
https://nightlies.apache.org/flink/flink-docs-release-1.18/docs/ops/state/checkpointing_under_backpressure/


Best,
Zhanghao Chen

From: David Morávek 
Sent: Wednesday, January 10, 2024 6:26
To: dev@flink.apache.org ; Piotr Nowojski 

Subject: Re: [DISCUSS] FLIP 411: Chaining-agnostic Operator ID generation for 
improved state compatibility on parallelism change

Hi Zhanghao,

Thanks for the FLIP. What you're proposing makes a lot of sense +1

Have you thought about how this works with unaligned checkpoints in case
you go from unchained to chained? I think it should be fine because this
scenario should only apply to forward/rebalance scenarios where we, as far
as I recall, force alignment anyway, so there should be no exchanges to
snapshot. It might just work, but something to double-check. Maybe @Piotr
Nowojski  could confirm it.

Best,
D.

On Wed, Jan 3, 2024 at 7:10 AM Zhanghao Chen 
wrote:

Dear Flink devs,

I'd like to start a discussion on FLIP 411: Chaining-agnostic Operator ID
generation for improved state compatibility on parallelism change [1].

Currently, when user does not explicitly set operator UIDs, the chaining
behavior will still affect state compatibility, as the generation of the
Operator ID is dependent on its chained output nodes. For example, a simple
source->sink DAG with source and sink chained together is state
incompatible with an otherwise identical DAG with source and sink unchained
(either because the parallelisms of the two ops are changed to be unequal
or chaining is disabled). This greatly limits the flexibility to perform
chain-breaking/building for performance tuning.

The dependency on chained output nodes for Operator ID generation can be
traced back to Flink 1.2. It is unclear at this point on why chained output
nodes are involved in the algorithm, but the following history background
might be related: prior to Flink 1.3, Flink runtime takes the snapshots by
the operator ID of the first vertex in a chain, so it somewhat makes sense
to include chained output nodes into the algorithm as
chain-breaking/building is expected to break state-compatibility anyway.

Given that operator-level state recovery within a chain has long been
supported since Flink 1.3, I propose to introduce StreamGraphHasherV3 that
is agnostic of the chaining behavior of operators, so that users are free
to tune the parallelism of individual operators without worrying about
state incompatibility. We can make the V3 hasher an optional choice in
Flink 1.19, and make it the default hasher in 2.0 for backwards
compatibility.

Looking forward to your suggestions on it, thanks~

[1]
https://cwiki.apache.org/confluence/display/FLINK/FLIP-411%3A+Chaining-agnostic+Operator+ID+generation+for+improved+state+compatibility+on+parallelism+change

Best,
Zhanghao Chen




[DISCUSS] FLIP-416: Deprecate and remove the RestoreMode#LEGACY

2024-01-10 Thread Zakelly Lan
Hi devs,

I'd like to start a discussion on FLIP-416: Deprecate and remove the
RestoreMode#LEGACY[1].

The FLIP-193[2] introduced two modes of state file ownership during
checkpoint restoration: RestoreMode#CLAIM and RestoreMode#NO_CLAIM. The
LEGACY mode, which was how Flink worked until 1.15, has been superseded by
NO_CLAIM as the default mode. The main drawback of LEGACY mode is that the
new job relies on artifacts from the old job without cleaning them up,
leaving users uncertain about when it is safe to delete the old checkpoint
directories. This leads to the accumulation of unnecessary checkpoint files
that are never cleaned up. Considering cluster availability and job
maintenance, it is not recommended to use LEGACY mode. Users could choose
the other two modes to get a clear semantic for the state file ownership.

This FLIP proposes to deprecate the LEGACY mode and remove it completely in
the upcoming Flink 2.0. This will make the semantic clear as well as
eliminate many bugs caused by mode transitions involving LEGACY mode (e.g.
FLINK-27114 [3]) and enhance code maintainability.

Looking forward to hearing from you!

[1] https://cwiki.apache.org/confluence/x/ookkEQ
[2] https://cwiki.apache.org/confluence/x/bIyqCw
[3] https://issues.apache.org/jira/browse/FLINK-27114

Best,
Zakelly


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

2024-01-10 Thread Yangze Guo
Thanks for the comments, Zhu and Matthias.

@Zhu Zhu

> How about disabling the checkpoint to avoid the cost? I know the cost is 
> there even if we disable the checkpoint at the moment. But I think it can be 
> fixed.
> If HA is disabled, the jobmanager needs to directly participate in all blob 
> shipping work which may result in a hot-spot.

Currently, there are several persistence services that have specific
implementations based on the HA mode:
- JobGraphStore and JobResultStore: These are related to job recovery
and can cause significant redundant I/O in OLAP scenarios, impacting
performance. It may be necessary to configure them as in-memory stores
for OLAP.
- CompletedCheckpointStore: As @Zhu Zhu mentioned, we can avoid this
overhead by disabling checkpoints. I agree to remove Checkpoint
Storage from the scope of this FLIP.
- BlobStore: Agree that disabling BlobStore can potentially lead to
hotspots in JobManagers. However, enabling it in OLAP scenarios can
also result in high external storage access overhead , e.g.
JobInformation/ShuffleDescriptor in TDD. I think this is a trade-off.
In our internal benchmark for short query (with 128 parallelism
WordCount jobs), disabling BlobStore resulted in a 100% increase in
QPS. Therefore, I lean towards disabling it. WDYT?

> FLINK-24038

As Matthias mentioned, each component still needs to write its RPC
address, so this part of the writing may be unavoidable.

@Zhu Zhu @Matthias

> I don't see why the PersistenceServices needs to have access to the 
> MaterialProvider. I feel like there shouldn't be a component that's shared 
> between the LeaderElectionService and the PersistenceServices.
> The corresponding ZooKeeper/k8s implementation would hold the client instance 
> (which is the only thing that should be shared between the 
> LeaderElectionService and the PersistenceServices implementations).

Yes, I agree that this is the goal of splitting the interfaces.
However, when I attempted to split it, I found that these two services
still have implicit temporal dependencies, such as the closure of the
client instance and the cleanup of services and job data.

Regards the refactoring of HighAvailabilityServices, I try to
summarize the following issues that need to be considered:
- Splitting LeaderServices and PersistenceServices; As Matthias
mentioned, this allows for easier testing.
- Removal of deprecated interfaces, such as getWebMonitorLeaderElectionService.
- Reviewing existing multiple close and cleanup interfaces.
- Integration of StandaloneHaServices and EmbeddedHaServices.
I think this topic might be big enough to have a separate discussion
thread. I am now inclined to focus on the discussion of HA
functionality in the OLAP scenario in FLIP-403 and exclude the
refactoring from the scope of this FLIP. This way, we can simply
return different persistence services in AbstractHaServices based on
the configuration. And I'm willing to file a new FLIP (or perhaps a
ticket would be sufficient) for the refactoring of HA. WDYT?


Best,
Yangze Guo

On Thu, Jan 11, 2024 at 12:19 AM Matthias Pohl
 wrote:
>
> Thanks for joining the discussion, everyone and sorry for picking it up
> that late. Here are a few points, I want to add to this discussion:
>
> - FLINK-24038 [1] led to a reduction of the curator/k8s client leader
> election requests by having a single leader election per JM rather than
> individual once per RPCEndpoint. We still need to have one record per
> component/RPCEndpoint (i.e. Dispatcher, RM, JobMaster instances, ...),
> though, because we need to save the address for RPC calls (Akka/Pekko) per
> component (each JobMaster has its own RPC endpoint with a dedicated port).
> That is why we cannot get rid of the individual entries/znodes per job.
>
> - An alternative for this FLIP's proposal would be to stick to the current
> HighAvailabilityServices interface. We could come up with a new
> implementation that does provide Standalone instances of what you call
> PersistentServices in this FLIP. That would reduce the efforts that come
> with refactoring the HighAvailabilityServices interface. It should be
> discussed here as an alternative and probably mentioned in the FLIP as a
> rejected alternative if the community agrees.
>
> - From a conceptual point of view, splitting the HighAvailabilityServices
> into LeaderElectionService and PersistentServices (I'm wondering whether
> something like JobHighAvailabilityServices would be more descriptive here.
> The word "persistence" is a bit ambiguous and can also be used in scenarios
> other than HA) makes sense in my opinion. One hint why separating this big
> interface HighAvailabilityServices into two smaller interfaces would make
> sense is the fact that there is a test
> implementation EmbeddedHaServicesWithLeadershipControl right now that
> provides embedded HA with helper methods to control the LeaderElection in
> ITCases. It is a workaround to get access to leader election. With two
> separate 

Re: [FLIP] Forward a flip to introduce minibatch optimization for Join

2024-01-10 Thread shuai xu
Hi Leonard,

Thanks for your help very much !

I have already started the discussion about FLIP-415: Introduce a new join 
operator to support minibatch[1]. 
And I’m looking forward to your feedback if you have some spare time to take a 
look.


[1]
https://cwiki.apache.org/confluence/display/FLINK/FLIP-415%3A+Introduce+a+new+join+operator+to+support+minibatch

Best,
Xu Shuai

> 2024年1月10日 17:47,Leonard Xu  写道:
> 
> Hey, shuai 
> 
> I’ve added wiki permission for you, looking forward your streaming join 
> optimization.
> 
> Best,
> Leonard




Re: [DISCUSS] FLIP-389: Annotate SingleThreadFetcherManager and FutureCompletingBlockingQueue as PublicEvolving

2024-01-10 Thread Hongshun Wang
Hi Qingsheng,


I agree with you that it would be clearer to have a new interface that
extracts the SplitFetcher creation and management logic from the current
SplitFetcherManager. However, extensive modifications to the interface may
influence a lot and cause compatibility issues. Perhaps we can consider
doing it later, rather than in this FLIP.


Adding a new internal method, SplitFetcherManager#getQueue(), to
SourceReaderBase seems to be a better option than exposing methods like poll
and notifyAvailable on SplitFetcherManager.


I have taken this valuable suggestion and updated the FLIP accordingly.


Thanks,

Hongshun

On Thu, Jan 11, 2024 at 2:09 PM Qingsheng Ren  wrote:

> Hi Hongshun and Becket,
>
> Sorry for being late in the discussion! I went through the entire FLIP but
> I still have some concerns about the new SplitFetcherManager.
>
> First of all I agree that we should hide the elementQueue from connector
> developers. This could simplify the interface exposed to developers so that
> they can focus on the interaction with external systems.
>
> However in the current FLIP, SplitFetcherManager exposes 4 more methods,
> poll / getAvailabilityFuture / notifyAvailable / noAvailableElement, which
> are tightly coupled with the implementation of the elementQueue. The naming
> of these methods look weird to me, like what does it mean to "poll from a
> SplitFetcherManager" / "notify a SplitFetcherManager available"? To clarify
> these methods we have to explain to developers that "well we hide a queue
> inside SplitFetcherMamager and the poll method is actually polling from the
> queue". I'm afraid these methods will implicitly expose the concept and the
> implementation of the queue to developers.
>
> I think a cleaner solution would be having a new interface that extracts
> SplitFetcher creating and managing logic from the current
> SplitFetcherManager, but having too many concepts might make the entire
> Source API even harder to understand. To make a compromise, I'm considering
> only exposing constructors of SplitFetcherManager as public APIs, and
> adding a new internal method SplitFetcherManager#getQueue() for
> SourceReaderBase (well it's a bit hacky I admit but I think exposing
> methods like poll and notifyAvailable on SplitFetcherManager is even
> worth). WDTY?
>
> Thanks,
> Qingsheng
>
> On Thu, Dec 21, 2023 at 8:36 AM Becket Qin  wrote:
>
>> Hi Hongshun,
>>
>> I think the proposal in the FLIP is basically fine. A few minor comments:
>>
>> 1. In FLIPs, we define all the user-sensible changes as public interfaces.
>> The public interface section should list all of them. So, the code blocks
>> currently in the proposed changes section should be put into the public
>> interface section instead.
>>
>> 2. It would be good to put all the changes of one class together. For
>> example, for SplitFetcherManager, we can say:
>> - Change SplitFetcherManager from Internal to PublicEvolving.
>> - Deprecate the old constructor exposing the
>> FutureCompletingBlockingQueue, and add new constructors as replacements
>> which creates the FutureCompletingBlockingQueue instance internally.
>> - Add a few new methods to expose the functionality of the internal
>> FutureCompletingBlockingQueue via the SplitFetcherManager.
>>And then follows the code block containing all the changes above.
>> Ideally, the changes should come with something like "// <-- New", so
>> that it is. easier to be found.
>>
>> 3. In the proposed changes section, it would be good to add some more
>> detailed explanation of the idea behind the public interface changes. So
>> even people new to Flink can understand better how exactly the interface
>> changes will help fulfill the motivation. For example, regarding the
>> constructor signature change, we can say the following. We can mention a
>> few things in this section:
>> - By exposing the SplitFetcherManager / SingleThreadFetcheManager, by
>> implementing addSplits() and removeSplits(), connector developers can
>> easily create their own threading models in the SourceReaderBase.
>> - Note that the SplitFetcher constructor is package private, so users
>> can only create SplitFetchers via
>> SplitFetcherManager.createSplitFetcher().
>> This ensures each SplitFetcher is always owned by the SplitFetcherManager.
>> - This FLIP essentially embedded the element queue (a
>> FutureCompletingBlockingQueue) instance into the SplitFetcherManager. This
>> hides the element queue from the connector developers and simplifies the
>> SourceReaderBase to consist of only SplitFetcherManager and RecordEmitter
>> as major components.
>>
>> In short, the public interface section answers the question of "what". We
>> should list all the user-sensible changes in the public interface section,
>> without verbose explanation. The proposed changes section answers "how",
>> where we can add more details to explain the changes listed in the public
>> interface section.
>>
>> Thanks,

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

2024-01-10 Thread Sergey Nuyanzin
Hi Hang

thanks for checking
yes, it could be packaged with jdk8, moreover jdk8 is checked in ci
for instance here ci for the commit tagged with v3.0.0-rc1 [1]

the strange thing in the output that you've provided is
>org.apache.flink:flink-connector-hive_2.12:jar:3.0.0: Could not find
> artifact jdk.tools:jdk.tools:jar:1.8 at specified path /Library/Internet
> Plug-Ins/JavaAppletPlugin.plugin/Contents/Home/../lib/tools.jar

there are no such dependencies in poms,
could it happen that there is some specific configuration on the machine
you used for that?
Can you please check it on another setup?


[1] https://github.com/apache/flink-connector-hive/actions/runs/7479158667


On Thu, Jan 11, 2024 at 4:44 AM Hang Ruan  wrote:

> Hi, Sergey Nuyanzin.
>
> Thanks for driving this.
>
> I try to package the source with jdk8 and it will cause an error as
> follows.
>
> [INFO]
> 
> [INFO] BUILD FAILURE
> [INFO]
> 
> [INFO] Total time:  4.621 s
> [INFO] Finished at: 2024-01-11T11:34:30+08:00
> [INFO]
> 
> [ERROR] Failed to execute goal on project flink-connector-hive_2.12: Could
> not resolve dependencies for project
> org.apache.flink:flink-connector-hive_2.12:jar:3.0.0: Could not find
> artifact jdk.tools:jdk.tools:jar:1.8 at specified path /Library/Internet
> Plug-Ins/JavaAppletPlugin.plugin/Contents/Home/../lib/tools.jar -> [Help 1]
> [ERROR]
> [ERROR] To see the full stack trace of the errors, re-run Maven with the -e
> switch.
> [ERROR] Re-run Maven using the -X switch to enable full debug logging.
> [ERROR]
> [ERROR] For more information about the errors and possible solutions,
> please read the following articles:
> [ERROR] [Help 1]
>
> http://cwiki.apache.org/confluence/display/MAVEN/DependencyResolutionException
> [ERROR]
> [ERROR] After correcting the problems, you can resume the build with the
> command
> [ERROR]   mvn  -rf :flink-connector-hive_2.12
>
> I see that the 'Building the Apache Flink Hive Connector from Source' part
> in README requires the Java 11. I am not sure whether this could be treated
> as an error.
> Does the flink-connector-hive support to be packaged with jdk8 now?
>
> Best,
> Hang
>
> Jiabao Sun  于2024年1月11日周四 11:35写道:
>
> > +1 (non-binding)
> >
> > - Validated checksum hash
> > - Verified signature
> > - Verified web PR
> > - Verified tags
> >
> > Best,
> > Jiabao
> >
> >
> > > 2024年1月11日 11:25,Hang Ruan  写道:
> > >
> > > Sorry that I make a mistake. I build the source with Maven and jdk11.
> > >
> > > Best,
> > > Hang
> > >
> > > Hang Ruan  于2024年1月11日周四 11:13写道:
> > >
> > >> +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 web PR
> > >> - Verified that the flink-connector-base is not packaged in hive
> > connector
> > >>
> > >> Best,
> > >> Hang
> > >>
> > >> Sergey Nuyanzin  于2024年1月11日周四 06:19写道:
> > >>
> > >>> Hi everyone,
> > >>> Please review and vote on the release candidate #1 for the version
> > 3.0.0,
> > >>> as follows:
> > >>> [ ] +1, Approve the release
> > >>> [ ] -1, Do not approve the release (please provide specific comments)
> > >>>
> > >>> This version is compatible with Flink 1.18.x
> > >>>
> > >>> 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 F752 9FAE 2481 1A5C
> 0DF3
> > >>> CA74 1596 BBF0 7268 35D8 [3],
> > >>> * all artifacts to be deployed to the Maven Central Repository [4],
> > >>> * source code tag v3.0.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.
> > >>>
> > >>> Thanks,
> > >>> Release Manager
> > >>>
> > >>> [1] https://issues.apache.org/jira/projects/FLINK/versions/12352591
> > >>> [2]
> > >>>
> > >>>
> >
> https://dist.apache.org/repos/dist/dev/flink/flink-connector-hive-3.0.0-rc1
> > >>> [3] https://dist.apache.org/repos/dist/release/flink/KEYS
> > >>> [4]
> > >>>
> > https://repository.apache.org/content/repositories/orgapacheflink-1694/
> > >>> [5]
> > >>>
> https://github.com/apache/flink-connector-hive/releases/tag/v3.0.0-rc1
> > >>> [6] https://github.com/apache/flink-web/pull/709
> > >>>
> > >>
> >
> >
>


-- 
Best regards,
Sergey


Re: [DISCUSS] FLIP-415: Introduce a new join operator to support minibatch

2024-01-10 Thread shuai xu
Hi Jane,

Thanks for your reminder!  I missed this.

I updated the FLIP with the UML of MiniBatchStreamingJoinOperator and linking 
my POC implementation as reference. 
They are placed in the part of Proposed Changes. 

Best,
Xu Shuai



> 2024年1月11日 11:18,Jane Chan  写道:
> 
> Hi shuai,
> 
> Thanks for initiating the discussion. The mini-batch join optimization is
> very helpful, particularly for optimizing outer join conditions in CDC
> sources and handling cascade joins. And +1 for the proposal.
> 
> However, I don't see any details on the proposed
> "MiniBatchStreamingJoinOperator",  would you mind elaborating more about it?
> 
> Best,
> Jane
> 
> 
> On Wed, Jan 10, 2024 at 10:56 PM Benchao Li  wrote:
> 
>> Thanks shuai for driving this, mini-batch Join is a very useful
>> optimization, +1 for the general idea.
>> 
>> Regarding the configuration
>> "table.exec.stream.join.mini-batch-enabled", I'm not sure it's really
>> necessary. The semantic of changelog emitted by the Join operator is
>> eventual consistency, so there is no much difference between original
>> Join and mini-batch Join from this aspect. Besides, introducing more
>> options would make it more complex for users, harder to understand and
>> maintain, which we should be careful about.
>> 
>> One thing about the implementation, could you make the new operator
>> share the same state definition with the original one?
>> 
>> shuai xu  于2024年1月10日周三 21:23写道:
>>> 
>>> Hi devs,
>>> 
>>> I’d like to start a discussion on FLIP-415: Introduce a new join
>> operator to support minibatch[1].
>>> 
>>> Currently, when performing cascading connections in Flink, there is a
>> pain point of record amplification. Every record join operator receives
>> would trigger join process. However, if records of +I and -D matches , they
>> could be folded to reduce two times of join process. Besides, records of
>> -U +U might output 4 records in which two records are redundant when
>> encountering outer join .
>>> 
>>> To address this issue, this FLIP introduces a new
>> MiniBatchStreamingJoinOperator to achieve batch processing which could
>> reduce number of outputting redundant messages and avoid unnecessary join
>> processes.
>>> A new option is added to control the operator to avoid influencing
>> existing 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-415%3A+Introduce+a+new+join+operator+to+support+minibatch
>>> 
>>> Best,
>>> Xu Shuai
>> 
>> 
>> 
>> --
>> 
>> Best,
>> Benchao Li
>> 



Re: Re: Re: [VOTE] Accept Flink CDC into Apache Flink

2024-01-10 Thread Hangxiang Yu
+1 (non-binding)

On Thu, Jan 11, 2024 at 11:19 AM Xuannan Su  wrote:

> +1 (non-binding)
>
> Best,
> Xuannan
>
> On Thu, Jan 11, 2024 at 10:28 AM Xuyang  wrote:
> >
> > +1 (non-binding)--
> >
> > Best!
> > Xuyang
> >
> >
> >
> >
> >
> > 在 2024-01-11 10:00:11,"Yang Wang"  写道:
> > >+1 (binding)
> > >
> > >
> > >Best,
> > >Yang
> > >
> > >On Thu, Jan 11, 2024 at 9:53 AM liu ron  wrote:
> > >
> > >> +1 non-binding
> > >>
> > >> Best
> > >> Ron
> > >>
> > >> Matthias Pohl  于2024年1月10日周三 23:05写道:
> > >>
> > >> > +1 (binding)
> > >> >
> > >> > On Wed, Jan 10, 2024 at 3:35 PM ConradJam 
> wrote:
> > >> >
> > >> > > +1 non-binding
> > >> > >
> > >> > > Dawid Wysakowicz  于2024年1月10日周三 21:06写道:
> > >> > >
> > >> > > > +1 (binding)
> > >> > > > Best,
> > >> > > > Dawid
> > >> > > >
> > >> > > > On Wed, 10 Jan 2024 at 11:54, Piotr Nowojski <
> pnowoj...@apache.org>
> > >> > > wrote:
> > >> > > >
> > >> > > > > +1 (binding)
> > >> > > > >
> > >> > > > > śr., 10 sty 2024 o 11:25 Martijn Visser <
> martijnvis...@apache.org>
> > >> > > > > napisał(a):
> > >> > > > >
> > >> > > > > > +1 (binding)
> > >> > > > > >
> > >> > > > > > On Wed, Jan 10, 2024 at 4:43 AM Xingbo Huang <
> hxbks...@gmail.com
> > >> >
> > >> > > > wrote:
> > >> > > > > > >
> > >> > > > > > > +1 (binding)
> > >> > > > > > >
> > >> > > > > > > Best,
> > >> > > > > > > Xingbo
> > >> > > > > > >
> > >> > > > > > > Dian Fu  于2024年1月10日周三 11:35写道:
> > >> > > > > > >
> > >> > > > > > > > +1 (binding)
> > >> > > > > > > >
> > >> > > > > > > > Regards,
> > >> > > > > > > > Dian
> > >> > > > > > > >
> > >> > > > > > > > On Wed, Jan 10, 2024 at 5:09 AM Sharath <
> > >> dsaishar...@gmail.com
> > >> > >
> > >> > > > > wrote:
> > >> > > > > > > > >
> > >> > > > > > > > > +1 (non-binding)
> > >> > > > > > > > >
> > >> > > > > > > > > Best,
> > >> > > > > > > > > Sharath
> > >> > > > > > > > >
> > >> > > > > > > > > On Tue, Jan 9, 2024 at 1:02 PM Venkata Sanath
> Muppalla <
> > >> > > > > > > > sanath...@gmail.com>
> > >> > > > > > > > > wrote:
> > >> > > > > > > > >
> > >> > > > > > > > > > +1 (non-binding)
> > >> > > > > > > > > >
> > >> > > > > > > > > > Thanks,
> > >> > > > > > > > > > Sanath
> > >> > > > > > > > > >
> > >> > > > > > > > > > On Tue, Jan 9, 2024 at 11:16 AM Peter Huang <
> > >> > > > > > > > huangzhenqiu0...@gmail.com>
> > >> > > > > > > > > > wrote:
> > >> > > > > > > > > >
> > >> > > > > > > > > > > +1 (non-binding)
> > >> > > > > > > > > > >
> > >> > > > > > > > > > >
> > >> > > > > > > > > > > Best Regards
> > >> > > > > > > > > > > Peter Huang
> > >> > > > > > > > > > >
> > >> > > > > > > > > > >
> > >> > > > > > > > > > > On Tue, Jan 9, 2024 at 5:26 AM Jane Chan <
> > >> > > > > qingyue@gmail.com>
> > >> > > > > > > > wrote:
> > >> > > > > > > > > > >
> > >> > > > > > > > > > > > +1 (non-binding)
> > >> > > > > > > > > > > >
> > >> > > > > > > > > > > > Best,
> > >> > > > > > > > > > > > Jane
> > >> > > > > > > > > > > >
> > >> > > > > > > > > > > > On Tue, Jan 9, 2024 at 8:41 PM Lijie Wang <
> > >> > > > > > > > wangdachui9...@gmail.com>
> > >> > > > > > > > > > > > wrote:
> > >> > > > > > > > > > > >
> > >> > > > > > > > > > > > > +1 (non-binding)
> > >> > > > > > > > > > > > >
> > >> > > > > > > > > > > > > Best,
> > >> > > > > > > > > > > > > Lijie
> > >> > > > > > > > > > > > >
> > >> > > > > > > > > > > > > Jiabao Sun 
> > >> > > > 于2024年1月9日周二
> > >> > > > > > > > 19:28写道:
> > >> > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > +1 (non-binding)
> > >> > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > Best,
> > >> > > > > > > > > > > > > > Jiabao
> > >> > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > On 2024/01/09 09:58:04 xiangyu feng wrote:
> > >> > > > > > > > > > > > > > > +1 (non-binding)
> > >> > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > Regards,
> > >> > > > > > > > > > > > > > > Xiangyu Feng
> > >> > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > Danny Cranmer 
> 于2024年1月9日周二
> > >> > > > 17:50写道:
> > >> > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > +1 (binding)
> > >> > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > Thanks,
> > >> > > > > > > > > > > > > > > > Danny
> > >> > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > On Tue, Jan 9, 2024 at 9:31 AM Feng Jin
> <
> > >> > > > > > ji...@gmail.com>
> > >> > > > > > > > > > wrote:
> > >> > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > +1 (non-binding)
> > >> > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > Best,
> > >> > > > > > > > > > > > > > > > > Feng Jin
> > >> > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > On Tue, Jan 9, 2024 at 5:29 PM Yuxin
> Tan <
> > >> > > > > > > > ta...@gmail.com>
> > >> > > > > > > > > > > > wrote:
> > >> > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > +1 (non-binding)
> > >> > > > > > > > > > > > > 

Re: [DISCUSS] FLIP-389: Annotate SingleThreadFetcherManager and FutureCompletingBlockingQueue as PublicEvolving

2024-01-10 Thread Qingsheng Ren
Hi Hongshun and Becket,

Sorry for being late in the discussion! I went through the entire FLIP but
I still have some concerns about the new SplitFetcherManager.

First of all I agree that we should hide the elementQueue from connector
developers. This could simplify the interface exposed to developers so that
they can focus on the interaction with external systems.

However in the current FLIP, SplitFetcherManager exposes 4 more methods,
poll / getAvailabilityFuture / notifyAvailable / noAvailableElement, which
are tightly coupled with the implementation of the elementQueue. The naming
of these methods look weird to me, like what does it mean to "poll from a
SplitFetcherManager" / "notify a SplitFetcherManager available"? To clarify
these methods we have to explain to developers that "well we hide a queue
inside SplitFetcherMamager and the poll method is actually polling from the
queue". I'm afraid these methods will implicitly expose the concept and the
implementation of the queue to developers.

I think a cleaner solution would be having a new interface that extracts
SplitFetcher creating and managing logic from the current
SplitFetcherManager, but having too many concepts might make the entire
Source API even harder to understand. To make a compromise, I'm considering
only exposing constructors of SplitFetcherManager as public APIs, and
adding a new internal method SplitFetcherManager#getQueue() for
SourceReaderBase (well it's a bit hacky I admit but I think exposing
methods like poll and notifyAvailable on SplitFetcherManager is even
worth). WDTY?

Thanks,
Qingsheng

On Thu, Dec 21, 2023 at 8:36 AM Becket Qin  wrote:

> Hi Hongshun,
>
> I think the proposal in the FLIP is basically fine. A few minor comments:
>
> 1. In FLIPs, we define all the user-sensible changes as public interfaces.
> The public interface section should list all of them. So, the code blocks
> currently in the proposed changes section should be put into the public
> interface section instead.
>
> 2. It would be good to put all the changes of one class together. For
> example, for SplitFetcherManager, we can say:
> - Change SplitFetcherManager from Internal to PublicEvolving.
> - Deprecate the old constructor exposing the
> FutureCompletingBlockingQueue, and add new constructors as replacements
> which creates the FutureCompletingBlockingQueue instance internally.
> - Add a few new methods to expose the functionality of the internal
> FutureCompletingBlockingQueue via the SplitFetcherManager.
>And then follows the code block containing all the changes above.
> Ideally, the changes should come with something like "// <-- New", so
> that it is. easier to be found.
>
> 3. In the proposed changes section, it would be good to add some more
> detailed explanation of the idea behind the public interface changes. So
> even people new to Flink can understand better how exactly the interface
> changes will help fulfill the motivation. For example, regarding the
> constructor signature change, we can say the following. We can mention a
> few things in this section:
> - By exposing the SplitFetcherManager / SingleThreadFetcheManager, by
> implementing addSplits() and removeSplits(), connector developers can
> easily create their own threading models in the SourceReaderBase.
> - Note that the SplitFetcher constructor is package private, so users
> can only create SplitFetchers via SplitFetcherManager.createSplitFetcher().
> This ensures each SplitFetcher is always owned by the SplitFetcherManager.
> - This FLIP essentially embedded the element queue (a
> FutureCompletingBlockingQueue) instance into the SplitFetcherManager. This
> hides the element queue from the connector developers and simplifies the
> SourceReaderBase to consist of only SplitFetcherManager and RecordEmitter
> as major components.
>
> In short, the public interface section answers the question of "what". We
> should list all the user-sensible changes in the public interface section,
> without verbose explanation. The proposed changes section answers "how",
> where we can add more details to explain the changes listed in the public
> interface section.
>
> Thanks,
>
> Jiangjie (Becket) Qin
>
>
>
> On Wed, Dec 20, 2023 at 10:07 AM Hongshun Wang 
> wrote:
>
> > Hi Becket,
> >
> >
> > It has been a long time since we last discussed. Are there any other
> > problems with this Flip from your side? I am looking forward to hearing
> > from you.
> >
> >
> > Thanks,
> > Hongshun Wang
> >
>


Re: [DISCUSS] FLIP 411: Chaining-agnostic Operator ID generation for improved state compatibility on parallelism change

2024-01-10 Thread Yu Chen
Hi Zhanghao,

Thanks for driving this, that’s really painful for us when we need to switch 
config `pipeline.operator-chaining`.

But I have a Concern, according to FLIP description, modifying `isChainable` 
related code in `StreamGraphHasherV2` will cause the generated operator id to 
be changed, which will result in the user unable to recover from the old state 
(old and new Operator IDs can't be mapped). 
Therefore switching Hasher strategy (V2->V3 or V3->V2) will lead to an 
incompatibility, is there any relevant compatibility design considered?

Best,
Yu Chen

> 2024年1月10日 10:25,Zhanghao Chen  写道:
> 
> Hi David,
> 
> Thanks for the comments. AFAIK, unaligned checkpoints are disabled for 
> pointwise connections according to [1], let's wait Piotr for confirmation. 
> The issue itself is not directly related to this proposal as well. If a user 
> manually specifies UIDs for each of the chained operators and has unaligned 
> checkpoints enabled, we will encounter the same issue if they decide to break 
> the chain on a later restart and try to recover from a retained cp.
> 
> [1] 
> https://nightlies.apache.org/flink/flink-docs-release-1.18/docs/ops/state/checkpointing_under_backpressure/
> 
> 
> Best,
> Zhanghao Chen
> 
> From: David Morávek 
> Sent: Wednesday, January 10, 2024 6:26
> To: dev@flink.apache.org ; Piotr Nowojski 
> 
> Subject: Re: [DISCUSS] FLIP 411: Chaining-agnostic Operator ID generation for 
> improved state compatibility on parallelism change
> 
> Hi Zhanghao,
> 
> Thanks for the FLIP. What you're proposing makes a lot of sense +1
> 
> Have you thought about how this works with unaligned checkpoints in case
> you go from unchained to chained? I think it should be fine because this
> scenario should only apply to forward/rebalance scenarios where we, as far
> as I recall, force alignment anyway, so there should be no exchanges to
> snapshot. It might just work, but something to double-check. Maybe @Piotr
> Nowojski  could confirm it.
> 
> Best,
> D.
> 
> On Wed, Jan 3, 2024 at 7:10 AM Zhanghao Chen 
> wrote:
> 
>> Dear Flink devs,
>> 
>> I'd like to start a discussion on FLIP 411: Chaining-agnostic Operator ID
>> generation for improved state compatibility on parallelism change [1].
>> 
>> Currently, when user does not explicitly set operator UIDs, the chaining
>> behavior will still affect state compatibility, as the generation of the
>> Operator ID is dependent on its chained output nodes. For example, a simple
>> source->sink DAG with source and sink chained together is state
>> incompatible with an otherwise identical DAG with source and sink unchained
>> (either because the parallelisms of the two ops are changed to be unequal
>> or chaining is disabled). This greatly limits the flexibility to perform
>> chain-breaking/building for performance tuning.
>> 
>> The dependency on chained output nodes for Operator ID generation can be
>> traced back to Flink 1.2. It is unclear at this point on why chained output
>> nodes are involved in the algorithm, but the following history background
>> might be related: prior to Flink 1.3, Flink runtime takes the snapshots by
>> the operator ID of the first vertex in a chain, so it somewhat makes sense
>> to include chained output nodes into the algorithm as
>> chain-breaking/building is expected to break state-compatibility anyway.
>> 
>> Given that operator-level state recovery within a chain has long been
>> supported since Flink 1.3, I propose to introduce StreamGraphHasherV3 that
>> is agnostic of the chaining behavior of operators, so that users are free
>> to tune the parallelism of individual operators without worrying about
>> state incompatibility. We can make the V3 hasher an optional choice in
>> Flink 1.19, and make it the default hasher in 2.0 for backwards
>> compatibility.
>> 
>> Looking forward to your suggestions on it, thanks~
>> 
>> [1]
>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-411%3A+Chaining-agnostic+Operator+ID+generation+for+improved+state+compatibility+on+parallelism+change
>> 
>> Best,
>> Zhanghao Chen
>> 



Re: [DISCUSS] FLIP-415: Introduce a new join operator to support minibatch

2024-01-10 Thread shuai xu
Thanks for your response, Benchao.

Here is my thought on the newly added option.
Users' current jobs are running on a version without minibatch join. If the 
existing option to enable minibatch join is utilized, then when users' jobs are 
migrated to the new version, the internal behavior of the join operation within 
the jobs will change. Although the semantic of changelog emitted by the Join 
operator is eventual consistency, the change might not be supposed for the 
downstream of the job which requires details of changelog. This newly added 
option also refers to 
'table.exec.deduplicate.mini-batch.compact-changes-enabled'. 

As for the implementation,The new operator shares the state of the original 
operator and it merely has an additional minibatch for storing records to do 
some optimization. The storage remains consistent, and there is minor 
modification to the computational logic.

Best,
Xu Shuai

> 2024年1月10日 22:56,Benchao Li  写道:
> 
> Thanks shuai for driving this, mini-batch Join is a very useful
> optimization, +1 for the general idea.
> 
> Regarding the configuration
> "table.exec.stream.join.mini-batch-enabled", I'm not sure it's really
> necessary. The semantic of changelog emitted by the Join operator is
> eventual consistency, so there is no much difference between original
> Join and mini-batch Join from this aspect. Besides, introducing more
> options would make it more complex for users, harder to understand and
> maintain, which we should be careful about.
> 
> One thing about the implementation, could you make the new operator
> share the same state definition with the original one?
> 
> shuai xu  于2024年1月10日周三 21:23写道:
>> 
>> Hi devs,
>> 
>> I’d like to start a discussion on FLIP-415: Introduce a new join operator to 
>> support minibatch[1].
>> 
>> Currently, when performing cascading connections in Flink, there is a pain 
>> point of record amplification. Every record join operator receives would 
>> trigger join process. However, if records of +I and -D matches , they could 
>> be folded to reduce two times of join process. Besides, records of  -U +U 
>> might output 4 records in which two records are redundant when encountering 
>> outer join .
>> 
>> To address this issue, this FLIP introduces a new  
>> MiniBatchStreamingJoinOperator to achieve batch processing which could 
>> reduce number of outputting redundant messages and avoid unnecessary join 
>> processes.
>> A new option is added to control the operator to avoid influencing existing 
>> 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-415%3A+Introduce+a+new+join+operator+to+support+minibatch
>> 
>> Best,
>> Xu Shuai
> 
> 
> 
> -- 
> 
> Best,
> Benchao Li



[jira] [Created] (FLINK-34059) Add documentation on how to use state TTL hint

2024-01-10 Thread Jane Chan (Jira)
Jane Chan created FLINK-34059:
-

 Summary: Add documentation on how to use state TTL hint
 Key: FLINK-34059
 URL: https://issues.apache.org/jira/browse/FLINK-34059
 Project: Flink
  Issue Type: Sub-task
  Components: Documentation, Table SQL / API
Affects Versions: 1.19.0
Reporter: Jane Chan






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


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

2024-01-10 Thread Hang Ruan
Hi, Sergey Nuyanzin.

Thanks for driving this.

I try to package the source with jdk8 and it will cause an error as follows.

[INFO]

[INFO] BUILD FAILURE
[INFO]

[INFO] Total time:  4.621 s
[INFO] Finished at: 2024-01-11T11:34:30+08:00
[INFO]

[ERROR] Failed to execute goal on project flink-connector-hive_2.12: Could
not resolve dependencies for project
org.apache.flink:flink-connector-hive_2.12:jar:3.0.0: Could not find
artifact jdk.tools:jdk.tools:jar:1.8 at specified path /Library/Internet
Plug-Ins/JavaAppletPlugin.plugin/Contents/Home/../lib/tools.jar -> [Help 1]
[ERROR]
[ERROR] To see the full stack trace of the errors, re-run Maven with the -e
switch.
[ERROR] Re-run Maven using the -X switch to enable full debug logging.
[ERROR]
[ERROR] For more information about the errors and possible solutions,
please read the following articles:
[ERROR] [Help 1]
http://cwiki.apache.org/confluence/display/MAVEN/DependencyResolutionException
[ERROR]
[ERROR] After correcting the problems, you can resume the build with the
command
[ERROR]   mvn  -rf :flink-connector-hive_2.12

I see that the 'Building the Apache Flink Hive Connector from Source' part
in README requires the Java 11. I am not sure whether this could be treated
as an error.
Does the flink-connector-hive support to be packaged with jdk8 now?

Best,
Hang

Jiabao Sun  于2024年1月11日周四 11:35写道:

> +1 (non-binding)
>
> - Validated checksum hash
> - Verified signature
> - Verified web PR
> - Verified tags
>
> Best,
> Jiabao
>
>
> > 2024年1月11日 11:25,Hang Ruan  写道:
> >
> > Sorry that I make a mistake. I build the source with Maven and jdk11.
> >
> > Best,
> > Hang
> >
> > Hang Ruan  于2024年1月11日周四 11:13写道:
> >
> >> +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 web PR
> >> - Verified that the flink-connector-base is not packaged in hive
> connector
> >>
> >> Best,
> >> Hang
> >>
> >> Sergey Nuyanzin  于2024年1月11日周四 06:19写道:
> >>
> >>> Hi everyone,
> >>> Please review and vote on the release candidate #1 for the version
> 3.0.0,
> >>> as follows:
> >>> [ ] +1, Approve the release
> >>> [ ] -1, Do not approve the release (please provide specific comments)
> >>>
> >>> This version is compatible with Flink 1.18.x
> >>>
> >>> 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 F752 9FAE 2481 1A5C 0DF3
> >>> CA74 1596 BBF0 7268 35D8 [3],
> >>> * all artifacts to be deployed to the Maven Central Repository [4],
> >>> * source code tag v3.0.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.
> >>>
> >>> Thanks,
> >>> Release Manager
> >>>
> >>> [1] https://issues.apache.org/jira/projects/FLINK/versions/12352591
> >>> [2]
> >>>
> >>>
> https://dist.apache.org/repos/dist/dev/flink/flink-connector-hive-3.0.0-rc1
> >>> [3] https://dist.apache.org/repos/dist/release/flink/KEYS
> >>> [4]
> >>>
> https://repository.apache.org/content/repositories/orgapacheflink-1694/
> >>> [5]
> >>> https://github.com/apache/flink-connector-hive/releases/tag/v3.0.0-rc1
> >>> [6] https://github.com/apache/flink-web/pull/709
> >>>
> >>
>
>


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

2024-01-10 Thread Jiabao Sun
+1 (non-binding)

- Validated checksum hash
- Verified signature
- Verified web PR
- Verified tags

Best,
Jiabao


> 2024年1月11日 11:25,Hang Ruan  写道:
> 
> Sorry that I make a mistake. I build the source with Maven and jdk11.
> 
> Best,
> Hang
> 
> Hang Ruan  于2024年1月11日周四 11:13写道:
> 
>> +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 web PR
>> - Verified that the flink-connector-base is not packaged in hive connector
>> 
>> Best,
>> Hang
>> 
>> Sergey Nuyanzin  于2024年1月11日周四 06:19写道:
>> 
>>> Hi everyone,
>>> Please review and vote on the release candidate #1 for the version 3.0.0,
>>> as follows:
>>> [ ] +1, Approve the release
>>> [ ] -1, Do not approve the release (please provide specific comments)
>>> 
>>> This version is compatible with Flink 1.18.x
>>> 
>>> 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 F752 9FAE 2481 1A5C 0DF3
>>> CA74 1596 BBF0 7268 35D8 [3],
>>> * all artifacts to be deployed to the Maven Central Repository [4],
>>> * source code tag v3.0.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.
>>> 
>>> Thanks,
>>> Release Manager
>>> 
>>> [1] https://issues.apache.org/jira/projects/FLINK/versions/12352591
>>> [2]
>>> 
>>> https://dist.apache.org/repos/dist/dev/flink/flink-connector-hive-3.0.0-rc1
>>> [3] https://dist.apache.org/repos/dist/release/flink/KEYS
>>> [4]
>>> https://repository.apache.org/content/repositories/orgapacheflink-1694/
>>> [5]
>>> https://github.com/apache/flink-connector-hive/releases/tag/v3.0.0-rc1
>>> [6] https://github.com/apache/flink-web/pull/709
>>> 
>> 



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

2024-01-10 Thread Hang Ruan
Sorry that I make a mistake. I build the source with Maven and jdk11.

Best,
Hang

Hang Ruan  于2024年1月11日周四 11:13写道:

> +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 web PR
> - Verified that the flink-connector-base is not packaged in hive connector
>
> Best,
> Hang
>
> Sergey Nuyanzin  于2024年1月11日周四 06:19写道:
>
>> Hi everyone,
>> Please review and vote on the release candidate #1 for the version 3.0.0,
>> as follows:
>> [ ] +1, Approve the release
>> [ ] -1, Do not approve the release (please provide specific comments)
>>
>> This version is compatible with Flink 1.18.x
>>
>> 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 F752 9FAE 2481 1A5C 0DF3
>>  CA74 1596 BBF0 7268 35D8 [3],
>> * all artifacts to be deployed to the Maven Central Repository [4],
>> * source code tag v3.0.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.
>>
>> Thanks,
>> Release Manager
>>
>> [1] https://issues.apache.org/jira/projects/FLINK/versions/12352591
>> [2]
>>
>> https://dist.apache.org/repos/dist/dev/flink/flink-connector-hive-3.0.0-rc1
>> [3] https://dist.apache.org/repos/dist/release/flink/KEYS
>> [4]
>> https://repository.apache.org/content/repositories/orgapacheflink-1694/
>> [5]
>> https://github.com/apache/flink-connector-hive/releases/tag/v3.0.0-rc1
>> [6] https://github.com/apache/flink-web/pull/709
>>
>


Re: [DISCUSS] FLIP-415: Introduce a new join operator to support minibatch

2024-01-10 Thread Jane Chan
Hi shuai,

Thanks for initiating the discussion. The mini-batch join optimization is
very helpful, particularly for optimizing outer join conditions in CDC
sources and handling cascade joins. And +1 for the proposal.

However, I don't see any details on the proposed
"MiniBatchStreamingJoinOperator",  would you mind elaborating more about it?

Best,
Jane


On Wed, Jan 10, 2024 at 10:56 PM Benchao Li  wrote:

> Thanks shuai for driving this, mini-batch Join is a very useful
> optimization, +1 for the general idea.
>
> Regarding the configuration
> "table.exec.stream.join.mini-batch-enabled", I'm not sure it's really
> necessary. The semantic of changelog emitted by the Join operator is
> eventual consistency, so there is no much difference between original
> Join and mini-batch Join from this aspect. Besides, introducing more
> options would make it more complex for users, harder to understand and
> maintain, which we should be careful about.
>
> One thing about the implementation, could you make the new operator
> share the same state definition with the original one?
>
> shuai xu  于2024年1月10日周三 21:23写道:
> >
> > Hi devs,
> >
> > I’d like to start a discussion on FLIP-415: Introduce a new join
> operator to support minibatch[1].
> >
> > Currently, when performing cascading connections in Flink, there is a
> pain point of record amplification. Every record join operator receives
> would trigger join process. However, if records of +I and -D matches , they
> could be folded to reduce two times of join process. Besides, records of
> -U +U might output 4 records in which two records are redundant when
> encountering outer join .
> >
> > To address this issue, this FLIP introduces a new
> MiniBatchStreamingJoinOperator to achieve batch processing which could
> reduce number of outputting redundant messages and avoid unnecessary join
> processes.
> > A new option is added to control the operator to avoid influencing
> existing 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-415%3A+Introduce+a+new+join+operator+to+support+minibatch
> >
> > Best,
> > Xu Shuai
>
>
>
> --
>
> Best,
> Benchao Li
>


Re: Re: Re: [VOTE] Accept Flink CDC into Apache Flink

2024-01-10 Thread Xuannan Su
+1 (non-binding)

Best,
Xuannan

On Thu, Jan 11, 2024 at 10:28 AM Xuyang  wrote:
>
> +1 (non-binding)--
>
> Best!
> Xuyang
>
>
>
>
>
> 在 2024-01-11 10:00:11,"Yang Wang"  写道:
> >+1 (binding)
> >
> >
> >Best,
> >Yang
> >
> >On Thu, Jan 11, 2024 at 9:53 AM liu ron  wrote:
> >
> >> +1 non-binding
> >>
> >> Best
> >> Ron
> >>
> >> Matthias Pohl  于2024年1月10日周三 23:05写道:
> >>
> >> > +1 (binding)
> >> >
> >> > On Wed, Jan 10, 2024 at 3:35 PM ConradJam  wrote:
> >> >
> >> > > +1 non-binding
> >> > >
> >> > > Dawid Wysakowicz  于2024年1月10日周三 21:06写道:
> >> > >
> >> > > > +1 (binding)
> >> > > > Best,
> >> > > > Dawid
> >> > > >
> >> > > > On Wed, 10 Jan 2024 at 11:54, Piotr Nowojski 
> >> > > wrote:
> >> > > >
> >> > > > > +1 (binding)
> >> > > > >
> >> > > > > śr., 10 sty 2024 o 11:25 Martijn Visser 
> >> > > > > napisał(a):
> >> > > > >
> >> > > > > > +1 (binding)
> >> > > > > >
> >> > > > > > On Wed, Jan 10, 2024 at 4:43 AM Xingbo Huang  >> >
> >> > > > wrote:
> >> > > > > > >
> >> > > > > > > +1 (binding)
> >> > > > > > >
> >> > > > > > > Best,
> >> > > > > > > Xingbo
> >> > > > > > >
> >> > > > > > > Dian Fu  于2024年1月10日周三 11:35写道:
> >> > > > > > >
> >> > > > > > > > +1 (binding)
> >> > > > > > > >
> >> > > > > > > > Regards,
> >> > > > > > > > Dian
> >> > > > > > > >
> >> > > > > > > > On Wed, Jan 10, 2024 at 5:09 AM Sharath <
> >> dsaishar...@gmail.com
> >> > >
> >> > > > > wrote:
> >> > > > > > > > >
> >> > > > > > > > > +1 (non-binding)
> >> > > > > > > > >
> >> > > > > > > > > Best,
> >> > > > > > > > > Sharath
> >> > > > > > > > >
> >> > > > > > > > > On Tue, Jan 9, 2024 at 1:02 PM Venkata Sanath Muppalla <
> >> > > > > > > > sanath...@gmail.com>
> >> > > > > > > > > wrote:
> >> > > > > > > > >
> >> > > > > > > > > > +1 (non-binding)
> >> > > > > > > > > >
> >> > > > > > > > > > Thanks,
> >> > > > > > > > > > Sanath
> >> > > > > > > > > >
> >> > > > > > > > > > On Tue, Jan 9, 2024 at 11:16 AM Peter Huang <
> >> > > > > > > > huangzhenqiu0...@gmail.com>
> >> > > > > > > > > > wrote:
> >> > > > > > > > > >
> >> > > > > > > > > > > +1 (non-binding)
> >> > > > > > > > > > >
> >> > > > > > > > > > >
> >> > > > > > > > > > > Best Regards
> >> > > > > > > > > > > Peter Huang
> >> > > > > > > > > > >
> >> > > > > > > > > > >
> >> > > > > > > > > > > On Tue, Jan 9, 2024 at 5:26 AM Jane Chan <
> >> > > > > qingyue@gmail.com>
> >> > > > > > > > wrote:
> >> > > > > > > > > > >
> >> > > > > > > > > > > > +1 (non-binding)
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > Best,
> >> > > > > > > > > > > > Jane
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > On Tue, Jan 9, 2024 at 8:41 PM Lijie Wang <
> >> > > > > > > > wangdachui9...@gmail.com>
> >> > > > > > > > > > > > wrote:
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > > +1 (non-binding)
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > > > Best,
> >> > > > > > > > > > > > > Lijie
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > > > Jiabao Sun 
> >> > > > 于2024年1月9日周二
> >> > > > > > > > 19:28写道:
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > > > > +1 (non-binding)
> >> > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > Best,
> >> > > > > > > > > > > > > > Jiabao
> >> > > > > > > > > > > > > >
> >> > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > On 2024/01/09 09:58:04 xiangyu feng wrote:
> >> > > > > > > > > > > > > > > +1 (non-binding)
> >> > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > Regards,
> >> > > > > > > > > > > > > > > Xiangyu Feng
> >> > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > Danny Cranmer  于2024年1月9日周二
> >> > > > 17:50写道:
> >> > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > +1 (binding)
> >> > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > Thanks,
> >> > > > > > > > > > > > > > > > Danny
> >> > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > On Tue, Jan 9, 2024 at 9:31 AM Feng Jin <
> >> > > > > > ji...@gmail.com>
> >> > > > > > > > > > wrote:
> >> > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > +1 (non-binding)
> >> > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > Best,
> >> > > > > > > > > > > > > > > > > Feng Jin
> >> > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > On Tue, Jan 9, 2024 at 5:29 PM Yuxin Tan <
> >> > > > > > > > ta...@gmail.com>
> >> > > > > > > > > > > > wrote:
> >> > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > +1 (non-binding)
> >> > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > Best,
> >> > > > > > > > > > > > > > > > > > Yuxin
> >> > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > Márton Balassi 
> >> > > 于2024年1月9日周二
> >> > > > > > 17:25写道:
> >> > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > +1 (binding)
> >> > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > On Tue, Jan 9, 

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

2024-01-10 Thread Hang Ruan
+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 web PR
- Verified that the flink-connector-base is not packaged in hive connector

Best,
Hang

Sergey Nuyanzin  于2024年1月11日周四 06:19写道:

> Hi everyone,
> Please review and vote on the release candidate #1 for the version 3.0.0,
> as follows:
> [ ] +1, Approve the release
> [ ] -1, Do not approve the release (please provide specific comments)
>
> This version is compatible with Flink 1.18.x
>
> 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 F752 9FAE 2481 1A5C 0DF3
>  CA74 1596 BBF0 7268 35D8 [3],
> * all artifacts to be deployed to the Maven Central Repository [4],
> * source code tag v3.0.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.
>
> Thanks,
> Release Manager
>
> [1] https://issues.apache.org/jira/projects/FLINK/versions/12352591
> [2]
> https://dist.apache.org/repos/dist/dev/flink/flink-connector-hive-3.0.0-rc1
> [3] https://dist.apache.org/repos/dist/release/flink/KEYS
> [4]
> https://repository.apache.org/content/repositories/orgapacheflink-1694/
> [5] https://github.com/apache/flink-connector-hive/releases/tag/v3.0.0-rc1
> [6] https://github.com/apache/flink-web/pull/709
>


Re: [DISCUSS] Proposing an LTS Release for the 1.x Line

2024-01-10 Thread Rui Fan
Thanks everyone for discussing this topic!

My question is could we make a trade-off between Flink users
and Flink maintainers?

1. From the perspective of a Flink maintainer

I strongly agree with Martin's point of view, such as:

- Allowing backporting of new features to Flink 1.x will result in users
delaying the upgrade.
- New features will also introduce new bugs, meaning that maintainers will
have to spend time on two release versions.

Considering the simplicity of maintenance, don't backport
new features to Flink 1.x is fine.

2. From the perspective of a flink user

In the first version Flink 2.x, flink will remove a lot of
deprecated api, and introduce some features.

It's a new major version, major version changes are much
greater than minor version and patch version. Big changes
may introduce more bugs, so I guess that a large number
of Flink users will not use the first version of 2.x in the
production environment. Maybe they will wait for the second
minor version of 2.x.

So, I was wondering whether we allow backport new features
from the first minor version of 2.x to 1.x?

It means, we allow backport new features of 2.0.0 to 1.21.
And 1.21.x is similar to 2.0.x, their features are same, but
2.0.x removes deprecated apis. After 2.0.0 is released,
all new features in 2.1.x and above are only available in 2.x.

Looking forward to your opinions~

Best,
Rui

On Wed, Jan 10, 2024 at 9:39 PM Martijn Visser 
wrote:

> Hi Alex,
>
> I saw that I missed replying to this topic. I do think that Xintong
> touched on an important topic when he mentioned that we should define
> what an LTS version means. From my point of view, I would state that
> an LTS version for Apache Flink means that bug fixes only will be made
> available for a longer period of time. I think that, combined with
> what you called option 1 (a clear end-of-life date) is the best
> option.
>
> Flink 2.0 will give us primarily the ability to remove a lot of
> deprecated APIs, especially with Flink's deprecation strategy. I
> expect that the majority of users will have an easy migration path
> from a Flink 1.x to a Flink 2.0, if you're currently not using a
> deprecated API and are a Java user.
>
> Allowing backporting of new features to Flink 1.x will result in users
> delaying the upgrade, but it doesn't make the upgrade any easier when
> they must upgrade. New features will also introduce new bugs, meaning
> that maintainers will have to spend time on two release versions. As
> the codebases diverge more and more, this will just become
> increasingly more complex.
>
> With that being said, I do think that it makes sense to also formalize
> the result of this discussion in a FLIP. That's just easier to point
> users towards at a later stage.
>
> Best regards,
>
> Martijn
>
> On Mon, Dec 4, 2023 at 9:55 PM Alexander Fedulov
>  wrote:
> >
> > Hi everyone,
> >
> > As we progress with the 1.19 release, which might potentially (although
> not
> > likely) be the last in the 1.x line, I'd like to revive our discussion on
> > the
> > LTS support matter. There is a general consensus that due to breaking API
> > changes in 2.0, extending bug fixes support by designating an LTS release
> > is
> > something we want to do.
> >
> > To summarize, the approaches we've considered are:
> >
> > Time-based: The last release of the 1.x line gets a clear end-of-life
> date
> > (2 years).
> > Release-based: The last release of the 1.x line gets support for 4 minor
> > releases in the 2.x line. The exact time is unknown, but we assume it to
> be
> > roughly 2 years.
> > LTS-to-LTS release: The last release of the 1.x line is supported until
> the
> > last release in the 2.x line is designated as LTS.
> >
> > We need to strike a balance between being user-friendly and nudging
> people
> > to
> > upgrade. From that perspective, option 1 is my favorite - we all know
> that
> > having a clear deadline works wonders in motivating action. At the same
> > time,
> > I appreciate that we might not want to introduce new kinds of procedures,
> > so
> > option 2 would be my second choice, provided we also formulate it like "4
> > minor releases, at least 2 years" (in case the minor release cadence
> > accelerates for some reason). I find option 3 a bit problematic since it
> > gives no incentive to upgrade, and people who do not follow Flink
> > development
> > closely might be caught by surprise when we decide to bump the major
> > version
> > again.
> >
> > I'd like to open a vote to stamp the official decision, but first, I
> would
> > like to understand if we can reach consensus on one of the options here,
> or
> > if
> > we'll need to push that to a vote by presenting multiple options.
> >
> > Looking forward to hearing your thoughts on this matter.
> >
> > P.S.: 1.x and 2.x are just examples in this case; the decision also
> > translates
> > into a procedure for future major releases.
> >
> > Best,
> > Alex
> >
> > On Thu, 27 Jul 2023 at 17:32, Jing Ge 
> wrote:
> >

[jira] [Created] (FLINK-34058) Support optional parameters for named parameters

2024-01-10 Thread Feng Jin (Jira)
Feng Jin created FLINK-34058:


 Summary: Support optional parameters for named parameters
 Key: FLINK-34058
 URL: https://issues.apache.org/jira/browse/FLINK-34058
 Project: Flink
  Issue Type: Sub-task
  Components: Table SQL / Planner
Reporter: Feng Jin
 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

2024-01-10 Thread Xuannan Su
Hi Zakelly,

I am fine with either Option 2 or Option 3. I think the naming in
Option 2 makes it clear that it is a boolean configuration. However,
most of the currently available boolean configurations do not use
"enable" as a suffix. Therefore, Option 3 looks good to me as it
follows the current practice.

Best regards,
Xuannan

On Thu, Jan 11, 2024 at 9:50 AM Hangxiang Yu  wrote:
>
> >
> > That's a very good point. I realize that the word 'recovery' means way too
> > many things. So I suggest picking a more specific word here, how about
> > 'execution.state-recovery.*' ? Checkpointing and state recovery are
> > corresponding terms and won't make ambiguity.
> >
>
> This makes the configuration clearer to me. We could focus on the
> `state-recovery` at first.
>
> I think we could create another FLIP for the deprecation of LEGACY mode.
> >
>
> LGTM, Let's create a new FLIP to do this.
>
> IIUC, there is no clear ownership of the local copy files from the previous
> > job and it's better to define one. This needs more discussion so we could
> > create another thread for this. WDYT?
> >
>
> Yeah, I have created a new ticket FLINK-34032 to track and discuss this.
>
> On Wed, Jan 10, 2024 at 6:31 PM Zakelly Lan  wrote:
>
> > Hi everyone,
> >
> > It seems we still don't have a consensus on the rules for boolean type
> > options. Let me recap the alternatives we have:
> >
> > Option 1: Use enumeration options instead if possible. But this may cause
> > some name collisions or confusion as we discussed and we should unify the
> > statement everywhere.
> > Option 2: Use boolean options and add 'enabled' as the suffix.
> > Option 3: Use boolean options and ONLY add 'enabled' when there are more
> > detailed configurations under the same prefix, to prevent one name from
> > serving as a prefix to another.
> >
> > I am inclined to Option 3, since it is more in line with current practice
> > and friendly for existing users. Also It reduces the length of
> > configuration names as much as possible.
> >
> > Looking forward to your opinions! Thanks!
> >
> >
> > Best,
> > Zakelly
> >
> > On Wed, Jan 10, 2024 at 3:30 PM Zakelly Lan  wrote:
> >
> > > Hi Hangxiang,
> > >
> > > Thanks for your suggestions!
> > >
> > > 1. Could execution.recovery also contain some other behaviors about
> > >> recovery ? e.g. restart-strategy.
> > >
> > >
> > > That's a very good point. I realize that the word 'recovery' means way
> > too
> > > many things. So I suggest picking a more specific word here, how about
> > > 'execution.state-recovery.*' ? Checkpointing and state recovery are
> > > corresponding terms and won't make ambiguity.
> > >
> > > 2. Could we also remove some legacy configuration value ? e.g. LEGACY
> > Mode
> > >> for execution.savepoint-restore-mode/execution.recovery.claim-mode.
> > >
> > >
> > > I think we could create another FLIP for the deprecation of LEGACY mode.
> > >
> > >
> > >> 3. Could the local checkpoint be cleaned
> > >> if execution.checkpointing.local-copy.enabled is true and
> > >> execution.recovery.from-local is false ? I found it's also an issue if
> > >> current local-recovery from enabled to disabled. Maybe another ticket is
> > >> needed.
> > >
> > >
> > > IIUC, there is no clear ownership of the local copy files from the
> > > previous job and it's better to define one. This needs more discussion so
> > > we could create another thread for this. WDYT?
> > >
> > >
> > > Best,
> > > Zakelly
> > >
> > > On Tue, Jan 9, 2024 at 11:23 AM Hangxiang Yu 
> > wrote:
> > >
> > >> Hi, Zakelly.
> > >> Thanks for driving this. Overall LGTM as we discussed offline.
> > >>
> > >> Some comments/suggestions just came to mind:
> > >> 1. Could execution.recovery also contain some other behaviors about
> > >> recovery ? e.g. restart-strategy.
> > >> 2. Could we also remove some legacy configuration value ? e.g. LEGACY
> > Mode
> > >> for execution.savepoint-restore-mode/execution.recovery.claim-mode.
> > >> 3. Could the local checkpoint be cleaned
> > >> if execution.checkpointing.local-copy.enabled is true and
> > >> execution.recovery.from-local is false ? I found it's also an issue if
> > >> current local-recovery from enabled to disabled. Maybe another ticket is
> > >> needed.
> > >> 4. +1 for enabling execution.checkpointing.incremental by default which
> > is
> > >> basically default configuration in our production environment.
> > >>
> > >>
> > >> On Mon, Jan 8, 2024 at 6:06 PM Zakelly Lan 
> > wrote:
> > >>
> > >> > Hi Yun,
> > >> >
> > >> > Thanks for your comments!
> > >> >
> > >> >  1.  We shall not describe the configuration with its implementation
> > for
> > >> > > 'execution.checkpointing.local-copy.*' options, for hashmap
> > >> > state-backend,
> > >> > > it would write two streams and for Rocksdb state-backend, it would
> > use
> > >> > > hard-link for backup. Thus, I think
> > >> > > 'execution.checkpointing.local-backup.*' looks better.
> > >> >
> > >> > I agreed that we'd better name the option in 

[jira] [Created] (FLINK-34057) Support named parameters for functions

2024-01-10 Thread Feng Jin (Jira)
Feng Jin created FLINK-34057:


 Summary: Support named parameters for functions
 Key: FLINK-34057
 URL: https://issues.apache.org/jira/browse/FLINK-34057
 Project: Flink
  Issue Type: Sub-task
  Components: Table SQL / Planner
Reporter: Feng Jin
 Fix For: 1.19.0






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


[jira] [Created] (FLINK-34056) Support named parameters for procedures

2024-01-10 Thread Feng Jin (Jira)
Feng Jin created FLINK-34056:


 Summary: Support named parameters for procedures
 Key: FLINK-34056
 URL: https://issues.apache.org/jira/browse/FLINK-34056
 Project: Flink
  Issue Type: Sub-task
  Components: Table SQL / Planner
Reporter: Feng Jin
 Fix For: 1.19.0






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


Re:Re: Re: [VOTE] Accept Flink CDC into Apache Flink

2024-01-10 Thread Xuyang
+1 (non-binding)--

Best!
Xuyang





在 2024-01-11 10:00:11,"Yang Wang"  写道:
>+1 (binding)
>
>
>Best,
>Yang
>
>On Thu, Jan 11, 2024 at 9:53 AM liu ron  wrote:
>
>> +1 non-binding
>>
>> Best
>> Ron
>>
>> Matthias Pohl  于2024年1月10日周三 23:05写道:
>>
>> > +1 (binding)
>> >
>> > On Wed, Jan 10, 2024 at 3:35 PM ConradJam  wrote:
>> >
>> > > +1 non-binding
>> > >
>> > > Dawid Wysakowicz  于2024年1月10日周三 21:06写道:
>> > >
>> > > > +1 (binding)
>> > > > Best,
>> > > > Dawid
>> > > >
>> > > > On Wed, 10 Jan 2024 at 11:54, Piotr Nowojski 
>> > > wrote:
>> > > >
>> > > > > +1 (binding)
>> > > > >
>> > > > > śr., 10 sty 2024 o 11:25 Martijn Visser 
>> > > > > napisał(a):
>> > > > >
>> > > > > > +1 (binding)
>> > > > > >
>> > > > > > On Wed, Jan 10, 2024 at 4:43 AM Xingbo Huang > >
>> > > > wrote:
>> > > > > > >
>> > > > > > > +1 (binding)
>> > > > > > >
>> > > > > > > Best,
>> > > > > > > Xingbo
>> > > > > > >
>> > > > > > > Dian Fu  于2024年1月10日周三 11:35写道:
>> > > > > > >
>> > > > > > > > +1 (binding)
>> > > > > > > >
>> > > > > > > > Regards,
>> > > > > > > > Dian
>> > > > > > > >
>> > > > > > > > On Wed, Jan 10, 2024 at 5:09 AM Sharath <
>> dsaishar...@gmail.com
>> > >
>> > > > > wrote:
>> > > > > > > > >
>> > > > > > > > > +1 (non-binding)
>> > > > > > > > >
>> > > > > > > > > Best,
>> > > > > > > > > Sharath
>> > > > > > > > >
>> > > > > > > > > On Tue, Jan 9, 2024 at 1:02 PM Venkata Sanath Muppalla <
>> > > > > > > > sanath...@gmail.com>
>> > > > > > > > > wrote:
>> > > > > > > > >
>> > > > > > > > > > +1 (non-binding)
>> > > > > > > > > >
>> > > > > > > > > > Thanks,
>> > > > > > > > > > Sanath
>> > > > > > > > > >
>> > > > > > > > > > On Tue, Jan 9, 2024 at 11:16 AM Peter Huang <
>> > > > > > > > huangzhenqiu0...@gmail.com>
>> > > > > > > > > > wrote:
>> > > > > > > > > >
>> > > > > > > > > > > +1 (non-binding)
>> > > > > > > > > > >
>> > > > > > > > > > >
>> > > > > > > > > > > Best Regards
>> > > > > > > > > > > Peter Huang
>> > > > > > > > > > >
>> > > > > > > > > > >
>> > > > > > > > > > > On Tue, Jan 9, 2024 at 5:26 AM Jane Chan <
>> > > > > qingyue@gmail.com>
>> > > > > > > > wrote:
>> > > > > > > > > > >
>> > > > > > > > > > > > +1 (non-binding)
>> > > > > > > > > > > >
>> > > > > > > > > > > > Best,
>> > > > > > > > > > > > Jane
>> > > > > > > > > > > >
>> > > > > > > > > > > > On Tue, Jan 9, 2024 at 8:41 PM Lijie Wang <
>> > > > > > > > wangdachui9...@gmail.com>
>> > > > > > > > > > > > wrote:
>> > > > > > > > > > > >
>> > > > > > > > > > > > > +1 (non-binding)
>> > > > > > > > > > > > >
>> > > > > > > > > > > > > Best,
>> > > > > > > > > > > > > Lijie
>> > > > > > > > > > > > >
>> > > > > > > > > > > > > Jiabao Sun 
>> > > > 于2024年1月9日周二
>> > > > > > > > 19:28写道:
>> > > > > > > > > > > > >
>> > > > > > > > > > > > > > +1 (non-binding)
>> > > > > > > > > > > > > >
>> > > > > > > > > > > > > > Best,
>> > > > > > > > > > > > > > Jiabao
>> > > > > > > > > > > > > >
>> > > > > > > > > > > > > >
>> > > > > > > > > > > > > > On 2024/01/09 09:58:04 xiangyu feng wrote:
>> > > > > > > > > > > > > > > +1 (non-binding)
>> > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > Regards,
>> > > > > > > > > > > > > > > Xiangyu Feng
>> > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > Danny Cranmer  于2024年1月9日周二
>> > > > 17:50写道:
>> > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > +1 (binding)
>> > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > Thanks,
>> > > > > > > > > > > > > > > > Danny
>> > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > On Tue, Jan 9, 2024 at 9:31 AM Feng Jin <
>> > > > > > ji...@gmail.com>
>> > > > > > > > > > wrote:
>> > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > +1 (non-binding)
>> > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > Best,
>> > > > > > > > > > > > > > > > > Feng Jin
>> > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > On Tue, Jan 9, 2024 at 5:29 PM Yuxin Tan <
>> > > > > > > > ta...@gmail.com>
>> > > > > > > > > > > > wrote:
>> > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > +1 (non-binding)
>> > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > Best,
>> > > > > > > > > > > > > > > > > > Yuxin
>> > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > Márton Balassi 
>> > > 于2024年1月9日周二
>> > > > > > 17:25写道:
>> > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > +1 (binding)
>> > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > On Tue, Jan 9, 2024 at 10:15 AM Leonard
>> > Xu
>> > > <
>> > > > > > > > > > > xb...@gmail.com>
>> > > > > > > > > > > > > > > > wrote:
>> > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > +1(binding)
>> > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > Best,
>> > > > > > > > > > > > > > > > > > > > Leonard
>> > > > > > > > > > > > > > > > > > > >
>> > > > 

[jira] [Created] (FLINK-34055) Introduce a new annotation for named parameters.

2024-01-10 Thread Feng Jin (Jira)
Feng Jin created FLINK-34055:


 Summary: Introduce a new annotation for named parameters.
 Key: FLINK-34055
 URL: https://issues.apache.org/jira/browse/FLINK-34055
 Project: Flink
  Issue Type: Sub-task
  Components: Table SQL / Planner
Reporter: Feng Jin
 Fix For: 1.19.0


Introduce a new annotation to specify the parameter name, indicate if it is 
optional, and potentially support specifying default values in the future.

Deprecate the argumentNames method in FunctionHints as it is not user-friendly 
for specifying argument names with optional configuration.

 
{code:java}
public @interface ArgumentHint {
/**
 * The name of the parameter, default is an empty string.
 */
String name() default "";
 
/**
 * Whether the parameter is optional, default is false.
 */
boolean isOptional() default false;
 
/**
 * The data type hint for the parameter.
 */
DataTypeHint type() default @DataTypeHint();
}
{code}



{code:java}
public @interface FunctionHint {
  
/**
 * Deprecated attribute for specifying the names of the arguments.
 * It is no longer recommended to use this attribute.
 */
@Deprecated
String[] argumentNames() default {""};
  
/**
 * Attribute for specifying the hints and additional information for 
function arguments.
 */
ArgumentHint[] arguments() default {};
}
{code}





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


[jira] [Created] (FLINK-34054) FLIP-387: Support named parameters for functions and call procedures

2024-01-10 Thread Feng Jin (Jira)
Feng Jin created FLINK-34054:


 Summary: FLIP-387: Support named parameters for functions and call 
procedures
 Key: FLINK-34054
 URL: https://issues.apache.org/jira/browse/FLINK-34054
 Project: Flink
  Issue Type: New Feature
  Components: Table SQL / Planner
Reporter: Feng Jin
 Fix For: 1.19.0


Umbrella issue for 
https://cwiki.apache.org/confluence/display/FLINK/FLIP-387%3A+Support+named+parameters+for+functions+and+call+procedures



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


[jira] [Created] (FLINK-34053) Support state TTL hint for group aggregate

2024-01-10 Thread Jane Chan (Jira)
Jane Chan created FLINK-34053:
-

 Summary: Support state TTL hint for group aggregate
 Key: FLINK-34053
 URL: https://issues.apache.org/jira/browse/FLINK-34053
 Project: Flink
  Issue Type: Sub-task
  Components: Table SQL / Planner
Affects Versions: 1.19.0
Reporter: Jane Chan






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


Re: Re: [VOTE] Accept Flink CDC into Apache Flink

2024-01-10 Thread Yang Wang
+1 (binding)


Best,
Yang

On Thu, Jan 11, 2024 at 9:53 AM liu ron  wrote:

> +1 non-binding
>
> Best
> Ron
>
> Matthias Pohl  于2024年1月10日周三 23:05写道:
>
> > +1 (binding)
> >
> > On Wed, Jan 10, 2024 at 3:35 PM ConradJam  wrote:
> >
> > > +1 non-binding
> > >
> > > Dawid Wysakowicz  于2024年1月10日周三 21:06写道:
> > >
> > > > +1 (binding)
> > > > Best,
> > > > Dawid
> > > >
> > > > On Wed, 10 Jan 2024 at 11:54, Piotr Nowojski 
> > > wrote:
> > > >
> > > > > +1 (binding)
> > > > >
> > > > > śr., 10 sty 2024 o 11:25 Martijn Visser 
> > > > > napisał(a):
> > > > >
> > > > > > +1 (binding)
> > > > > >
> > > > > > On Wed, Jan 10, 2024 at 4:43 AM Xingbo Huang  >
> > > > wrote:
> > > > > > >
> > > > > > > +1 (binding)
> > > > > > >
> > > > > > > Best,
> > > > > > > Xingbo
> > > > > > >
> > > > > > > Dian Fu  于2024年1月10日周三 11:35写道:
> > > > > > >
> > > > > > > > +1 (binding)
> > > > > > > >
> > > > > > > > Regards,
> > > > > > > > Dian
> > > > > > > >
> > > > > > > > On Wed, Jan 10, 2024 at 5:09 AM Sharath <
> dsaishar...@gmail.com
> > >
> > > > > wrote:
> > > > > > > > >
> > > > > > > > > +1 (non-binding)
> > > > > > > > >
> > > > > > > > > Best,
> > > > > > > > > Sharath
> > > > > > > > >
> > > > > > > > > On Tue, Jan 9, 2024 at 1:02 PM Venkata Sanath Muppalla <
> > > > > > > > sanath...@gmail.com>
> > > > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > +1 (non-binding)
> > > > > > > > > >
> > > > > > > > > > Thanks,
> > > > > > > > > > Sanath
> > > > > > > > > >
> > > > > > > > > > On Tue, Jan 9, 2024 at 11:16 AM Peter Huang <
> > > > > > > > huangzhenqiu0...@gmail.com>
> > > > > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > +1 (non-binding)
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > Best Regards
> > > > > > > > > > > Peter Huang
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > On Tue, Jan 9, 2024 at 5:26 AM Jane Chan <
> > > > > qingyue@gmail.com>
> > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > +1 (non-binding)
> > > > > > > > > > > >
> > > > > > > > > > > > Best,
> > > > > > > > > > > > Jane
> > > > > > > > > > > >
> > > > > > > > > > > > On Tue, Jan 9, 2024 at 8:41 PM Lijie Wang <
> > > > > > > > wangdachui9...@gmail.com>
> > > > > > > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > +1 (non-binding)
> > > > > > > > > > > > >
> > > > > > > > > > > > > Best,
> > > > > > > > > > > > > Lijie
> > > > > > > > > > > > >
> > > > > > > > > > > > > Jiabao Sun 
> > > > 于2024年1月9日周二
> > > > > > > > 19:28写道:
> > > > > > > > > > > > >
> > > > > > > > > > > > > > +1 (non-binding)
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > Jiabao
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On 2024/01/09 09:58:04 xiangyu feng wrote:
> > > > > > > > > > > > > > > +1 (non-binding)
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Regards,
> > > > > > > > > > > > > > > Xiangyu Feng
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Danny Cranmer  于2024年1月9日周二
> > > > 17:50写道:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > +1 (binding)
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > Danny
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > On Tue, Jan 9, 2024 at 9:31 AM Feng Jin <
> > > > > > ji...@gmail.com>
> > > > > > > > > > wrote:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > +1 (non-binding)
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > > > > Feng Jin
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > On Tue, Jan 9, 2024 at 5:29 PM Yuxin Tan <
> > > > > > > > ta...@gmail.com>
> > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > +1 (non-binding)
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > > > > > Yuxin
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Márton Balassi 
> > > 于2024年1月9日周二
> > > > > > 17:25写道:
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > +1 (binding)
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > On Tue, Jan 9, 2024 at 10:15 AM Leonard
> > Xu
> > > <
> > > > > > > > > > > xb...@gmail.com>
> > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > +1(binding)
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > > > > > > > Leonard
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > 2024年1月9日 下午5:08,Yangze Guo <
> > > > > ka...@gmail.com
> > > > > > >
> > > > > > > > 写道:
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > +1 (non-binding)
> > > > > > > > > > > 

Re: Re: [VOTE] Accept Flink CDC into Apache Flink

2024-01-10 Thread liu ron
+1 non-binding

Best
Ron

Matthias Pohl  于2024年1月10日周三 23:05写道:

> +1 (binding)
>
> On Wed, Jan 10, 2024 at 3:35 PM ConradJam  wrote:
>
> > +1 non-binding
> >
> > Dawid Wysakowicz  于2024年1月10日周三 21:06写道:
> >
> > > +1 (binding)
> > > Best,
> > > Dawid
> > >
> > > On Wed, 10 Jan 2024 at 11:54, Piotr Nowojski 
> > wrote:
> > >
> > > > +1 (binding)
> > > >
> > > > śr., 10 sty 2024 o 11:25 Martijn Visser 
> > > > napisał(a):
> > > >
> > > > > +1 (binding)
> > > > >
> > > > > On Wed, Jan 10, 2024 at 4:43 AM Xingbo Huang 
> > > wrote:
> > > > > >
> > > > > > +1 (binding)
> > > > > >
> > > > > > Best,
> > > > > > Xingbo
> > > > > >
> > > > > > Dian Fu  于2024年1月10日周三 11:35写道:
> > > > > >
> > > > > > > +1 (binding)
> > > > > > >
> > > > > > > Regards,
> > > > > > > Dian
> > > > > > >
> > > > > > > On Wed, Jan 10, 2024 at 5:09 AM Sharath  >
> > > > wrote:
> > > > > > > >
> > > > > > > > +1 (non-binding)
> > > > > > > >
> > > > > > > > Best,
> > > > > > > > Sharath
> > > > > > > >
> > > > > > > > On Tue, Jan 9, 2024 at 1:02 PM Venkata Sanath Muppalla <
> > > > > > > sanath...@gmail.com>
> > > > > > > > wrote:
> > > > > > > >
> > > > > > > > > +1 (non-binding)
> > > > > > > > >
> > > > > > > > > Thanks,
> > > > > > > > > Sanath
> > > > > > > > >
> > > > > > > > > On Tue, Jan 9, 2024 at 11:16 AM Peter Huang <
> > > > > > > huangzhenqiu0...@gmail.com>
> > > > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > +1 (non-binding)
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > Best Regards
> > > > > > > > > > Peter Huang
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > On Tue, Jan 9, 2024 at 5:26 AM Jane Chan <
> > > > qingyue@gmail.com>
> > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > +1 (non-binding)
> > > > > > > > > > >
> > > > > > > > > > > Best,
> > > > > > > > > > > Jane
> > > > > > > > > > >
> > > > > > > > > > > On Tue, Jan 9, 2024 at 8:41 PM Lijie Wang <
> > > > > > > wangdachui9...@gmail.com>
> > > > > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > +1 (non-binding)
> > > > > > > > > > > >
> > > > > > > > > > > > Best,
> > > > > > > > > > > > Lijie
> > > > > > > > > > > >
> > > > > > > > > > > > Jiabao Sun 
> > > 于2024年1月9日周二
> > > > > > > 19:28写道:
> > > > > > > > > > > >
> > > > > > > > > > > > > +1 (non-binding)
> > > > > > > > > > > > >
> > > > > > > > > > > > > Best,
> > > > > > > > > > > > > Jiabao
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > On 2024/01/09 09:58:04 xiangyu feng wrote:
> > > > > > > > > > > > > > +1 (non-binding)
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Regards,
> > > > > > > > > > > > > > Xiangyu Feng
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Danny Cranmer  于2024年1月9日周二
> > > 17:50写道:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > +1 (binding)
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > Danny
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > On Tue, Jan 9, 2024 at 9:31 AM Feng Jin <
> > > > > ji...@gmail.com>
> > > > > > > > > wrote:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > +1 (non-binding)
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > > > Feng Jin
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > On Tue, Jan 9, 2024 at 5:29 PM Yuxin Tan <
> > > > > > > ta...@gmail.com>
> > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > +1 (non-binding)
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > > > > Yuxin
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Márton Balassi 
> > 于2024年1月9日周二
> > > > > 17:25写道:
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > +1 (binding)
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > On Tue, Jan 9, 2024 at 10:15 AM Leonard
> Xu
> > <
> > > > > > > > > > xb...@gmail.com>
> > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > +1(binding)
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > > > > > > Leonard
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > 2024年1月9日 下午5:08,Yangze Guo <
> > > > ka...@gmail.com
> > > > > >
> > > > > > > 写道:
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > +1 (non-binding)
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > > > > > > > Yangze Guo
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > On Tue, Jan 9, 2024 at 5:06 PM Robert
> > > > > Metzger <
> > > > > > > > > > > > > > > rmetz...@apache.org
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > 

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

2024-01-10 Thread Hangxiang Yu
>
> That's a very good point. I realize that the word 'recovery' means way too
> many things. So I suggest picking a more specific word here, how about
> 'execution.state-recovery.*' ? Checkpointing and state recovery are
> corresponding terms and won't make ambiguity.
>

This makes the configuration clearer to me. We could focus on the
`state-recovery` at first.

I think we could create another FLIP for the deprecation of LEGACY mode.
>

LGTM, Let's create a new FLIP to do this.

IIUC, there is no clear ownership of the local copy files from the previous
> job and it's better to define one. This needs more discussion so we could
> create another thread for this. WDYT?
>

Yeah, I have created a new ticket FLINK-34032 to track and discuss this.

On Wed, Jan 10, 2024 at 6:31 PM Zakelly Lan  wrote:

> Hi everyone,
>
> It seems we still don't have a consensus on the rules for boolean type
> options. Let me recap the alternatives we have:
>
> Option 1: Use enumeration options instead if possible. But this may cause
> some name collisions or confusion as we discussed and we should unify the
> statement everywhere.
> Option 2: Use boolean options and add 'enabled' as the suffix.
> Option 3: Use boolean options and ONLY add 'enabled' when there are more
> detailed configurations under the same prefix, to prevent one name from
> serving as a prefix to another.
>
> I am inclined to Option 3, since it is more in line with current practice
> and friendly for existing users. Also It reduces the length of
> configuration names as much as possible.
>
> Looking forward to your opinions! Thanks!
>
>
> Best,
> Zakelly
>
> On Wed, Jan 10, 2024 at 3:30 PM Zakelly Lan  wrote:
>
> > Hi Hangxiang,
> >
> > Thanks for your suggestions!
> >
> > 1. Could execution.recovery also contain some other behaviors about
> >> recovery ? e.g. restart-strategy.
> >
> >
> > That's a very good point. I realize that the word 'recovery' means way
> too
> > many things. So I suggest picking a more specific word here, how about
> > 'execution.state-recovery.*' ? Checkpointing and state recovery are
> > corresponding terms and won't make ambiguity.
> >
> > 2. Could we also remove some legacy configuration value ? e.g. LEGACY
> Mode
> >> for execution.savepoint-restore-mode/execution.recovery.claim-mode.
> >
> >
> > I think we could create another FLIP for the deprecation of LEGACY mode.
> >
> >
> >> 3. Could the local checkpoint be cleaned
> >> if execution.checkpointing.local-copy.enabled is true and
> >> execution.recovery.from-local is false ? I found it's also an issue if
> >> current local-recovery from enabled to disabled. Maybe another ticket is
> >> needed.
> >
> >
> > IIUC, there is no clear ownership of the local copy files from the
> > previous job and it's better to define one. This needs more discussion so
> > we could create another thread for this. WDYT?
> >
> >
> > Best,
> > Zakelly
> >
> > On Tue, Jan 9, 2024 at 11:23 AM Hangxiang Yu 
> wrote:
> >
> >> Hi, Zakelly.
> >> Thanks for driving this. Overall LGTM as we discussed offline.
> >>
> >> Some comments/suggestions just came to mind:
> >> 1. Could execution.recovery also contain some other behaviors about
> >> recovery ? e.g. restart-strategy.
> >> 2. Could we also remove some legacy configuration value ? e.g. LEGACY
> Mode
> >> for execution.savepoint-restore-mode/execution.recovery.claim-mode.
> >> 3. Could the local checkpoint be cleaned
> >> if execution.checkpointing.local-copy.enabled is true and
> >> execution.recovery.from-local is false ? I found it's also an issue if
> >> current local-recovery from enabled to disabled. Maybe another ticket is
> >> needed.
> >> 4. +1 for enabling execution.checkpointing.incremental by default which
> is
> >> basically default configuration in our production environment.
> >>
> >>
> >> On Mon, Jan 8, 2024 at 6:06 PM Zakelly Lan 
> wrote:
> >>
> >> > Hi Yun,
> >> >
> >> > Thanks for your comments!
> >> >
> >> >  1.  We shall not describe the configuration with its implementation
> for
> >> > > 'execution.checkpointing.local-copy.*' options, for hashmap
> >> > state-backend,
> >> > > it would write two streams and for Rocksdb state-backend, it would
> use
> >> > > hard-link for backup. Thus, I think
> >> > > 'execution.checkpointing.local-backup.*' looks better.
> >> >
> >> > I agreed that we'd better name the option in user's perspective
> instead
> >> of
> >> > the implementation, thus I name it as a copy of the checkpoint in the
> >> > local disk, regardless of the way of generating it. The word 'backup'
> is
> >> > also suitable for this case, so I agree to change to
> >> > 'execution.checkpointing.local-backup.*' if no one objects.
> >> >
> >> >  2.  What does the 'execution.checkpointing.data-inline-threshold'
> >> mean? It
> >> > > seems not so easy to understand.
> >> >
> >> > The 'execution.checkpointing.data-inline-threshold' (original one as
> >> > 'state.storage.fs.memory-threshold') stands for the size threshold
> below
> >> > 

[VOTE] Release flink-connector-hive, release candidate #1

2024-01-10 Thread Sergey Nuyanzin
Hi everyone,
Please review and vote on the release candidate #1 for the version 3.0.0,
as follows:
[ ] +1, Approve the release
[ ] -1, Do not approve the release (please provide specific comments)

This version is compatible with Flink 1.18.x

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 F752 9FAE 2481 1A5C 0DF3
 CA74 1596 BBF0 7268 35D8 [3],
* all artifacts to be deployed to the Maven Central Repository [4],
* source code tag v3.0.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.

Thanks,
Release Manager

[1] https://issues.apache.org/jira/projects/FLINK/versions/12352591
[2]
https://dist.apache.org/repos/dist/dev/flink/flink-connector-hive-3.0.0-rc1
[3] https://dist.apache.org/repos/dist/release/flink/KEYS
[4] https://repository.apache.org/content/repositories/orgapacheflink-1694/
[5] https://github.com/apache/flink-connector-hive/releases/tag/v3.0.0-rc1
[6] https://github.com/apache/flink-web/pull/709


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

2024-01-10 Thread Sergey Nuyanzin
Thanks for driving this Martijn

based on the info in MANIFEST.MF of jars it seems it was built with jdk11
shouldn't we still use jdk8 for that?

On Fri, Jan 5, 2024 at 4:03 PM Martijn Visser 
wrote:

> Hi everyone,
> Please review and vote on the release candidate #1 for the version
> 3.0.1, as follows:
> [ ] +1, Approve the release
> [ ] -1, Do not approve the release (please provide specific comments)
>
> This version is compatible with Flink 1.16.x, 1.17.x and 1.18.x
>
> 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
> A5F3BCE4CBE993573EC5966A65321B8382B219AF [3],
> * all artifacts to be deployed to the Maven Central Repository [4],
> * source code tag v3.0.1-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.
>
> Thanks,
> Release Manager
>
> [1] https://issues.apache.org/jira/projects/FLINK/versions/12353603
> [2]
> https://dist.apache.org/repos/dist/dev/flink/flink-connector-hbase-3.0.1-rc1
> [3] https://dist.apache.org/repos/dist/release/flink/KEYS
> [4]
> https://repository.apache.org/content/repositories/orgapacheflink-1692/
> [5]
> https://github.com/apache/flink-connector-hbase/releases/tag/v3.0.1-rc1
> [6] https://github.com/apache/flink-web/pull/708
>


-- 
Best regards,
Sergey


Re: [DISCUSS] FLIP-329: Add operator attribute to specify support for object-reuse

2024-01-10 Thread Ken Krugler
Hi Dong,

> On Jan 4, 2024, at 10:18 PM, Dong Lin  wrote:
> 
> Hi Ken,
> 
> Sorry for the late reply. I didn't notice this email from you until now.
> 
> In this scenario you described above, I don't think operator2 will see the 
> result modified by operato1. Note that object re-use applies only to the 
> transmission of data between operators in the same operator chain. But Flink 
> won't put StreamX, operator1 and operator2 in the same operator chain when 
> both operator1 and operator2 reads the same output from StreamX.
> 
> Would this answer your question?


Actually operator2 will see the modified result.

The test case below illustrates this. It will fail when object reuse is enabled.

— Ken

package com.scaleunlimited.flinksnippets;

import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
import org.apache.flink.util.CloseableIterator;
import static org.junit.Assert.*;
import org.junit.Test;

public class ObjectReuseTest {

@Test
public void testObjectReuse() throws Exception {
StreamExecutionEnvironment env = 
StreamExecutionEnvironment.createLocalEnvironment(1);
// env.getConfig().enableObjectReuse();

DataStream stream1 = env.fromElements(
new Event("A", 1));

stream1.map((Event r) -> {
r.setValue(r.getValue() * 2);
return r;
})
.addSink(new DiscardingSink<>());

DataStream stream2 = stream1.map(r -> r);

CloseableIterator results = stream2.collectAsync();
   
env.execute();

assertTrue(results.hasNext());
Event result = results.next();
assertEquals(1, result.getValue());
assertFalse(results.hasNext());
}

public static class Event {
private String label;
private long value;

public Event() {}

public Event(String label, long value) {
this.label = label;
this.value = value;
}

public String getLabel() {
return label;
}

public void setLabel(String label) {
this.label = label;
}

public long getValue() {
return value;
}

public void setValue(long value) {
this.value = value;
}
}

}


> 
>  
> 
> On Fri, Oct 20, 2023 at 7:26 AM Ken Krugler  > wrote:
>> Hi Dong,
>> 
>> Sorry for not seeing this initially. I did have one question about the 
>> description of the issue in the FLIP:
>> 
>>> However, in cases where the upstream and downstream operators do not store 
>>> or access references to the input or output records, this deep-copy 
>>> overhead becomes unnecessary 
>> 
>> I was interested in getting clarification as to what you meant by “or access 
>> references…”, to see if it covered this situation:
>> 
>> StreamX —forward--> operator1
>> StreamX —forward--> operator2
>> 
>> If operator1 modifies the record, and object re-use is enabled, then 
>> operator2 will see the modified version, right?
>> 
>> Thanks,
>> 
>> — Ken
>> 
>>> On Jul 2, 2023, at 7:24 PM, Xuannan Su >> > wrote:
>>> 
>>> Hi all,
>>> 
>>> Dong(cc'ed) and I are opening this thread to discuss our proposal to
>>> add operator attribute to allow operator to specify support for
>>> object-reuse [1].
>>> 
>>> Currently, the default configuration for pipeline.object-reuse is set
>>> to false to avoid data corruption, which can result in suboptimal
>>> performance. We propose adding APIs that operators can utilize to
>>> inform the Flink runtime whether it is safe to reuse the emitted
>>> records. This enhancement would enable Flink to maximize its
>>> performance using the default configuration.
>>> 
>>> Please refer to the FLIP document for more details about the proposed
>>> design and implementation. We welcome any feedback and opinions on
>>> this proposal.
>>> 
>>> Best regards,
>>> 
>>> Dong and Xuannan
>>> 
>>> [1] 
>>> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=255073749
>> 
>> --
>> Ken Krugler
>> http://www.scaleunlimited.com 
>> Custom big data solutions
>> Flink & Pinot
>> 
>> 
>> 

--
Ken Krugler
http://www.scaleunlimited.com
Custom big data solutions
Flink & Pinot





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

2024-01-10 Thread Matthias Pohl
Thanks for joining the discussion, everyone and sorry for picking it up
that late. Here are a few points, I want to add to this discussion:

- FLINK-24038 [1] led to a reduction of the curator/k8s client leader
election requests by having a single leader election per JM rather than
individual once per RPCEndpoint. We still need to have one record per
component/RPCEndpoint (i.e. Dispatcher, RM, JobMaster instances, ...),
though, because we need to save the address for RPC calls (Akka/Pekko) per
component (each JobMaster has its own RPC endpoint with a dedicated port).
That is why we cannot get rid of the individual entries/znodes per job.

- An alternative for this FLIP's proposal would be to stick to the current
HighAvailabilityServices interface. We could come up with a new
implementation that does provide Standalone instances of what you call
PersistentServices in this FLIP. That would reduce the efforts that come
with refactoring the HighAvailabilityServices interface. It should be
discussed here as an alternative and probably mentioned in the FLIP as a
rejected alternative if the community agrees.

- From a conceptual point of view, splitting the HighAvailabilityServices
into LeaderElectionService and PersistentServices (I'm wondering whether
something like JobHighAvailabilityServices would be more descriptive here.
The word "persistence" is a bit ambiguous and can also be used in scenarios
other than HA) makes sense in my opinion. One hint why separating this big
interface HighAvailabilityServices into two smaller interfaces would make
sense is the fact that there is a test
implementation EmbeddedHaServicesWithLeadershipControl right now that
provides embedded HA with helper methods to control the LeaderElection in
ITCases. It is a workaround to get access to leader election. With two
separate interfaces, we could make it easier to test these things.

- I'm not too sure about the proposed class hierarchy of FLIP-403:
  - What are the semantics of the "MaterialProvider". The name doesn't give
me any hints on the interface/class purpose. There could be some
description for this component being added to the FLIP. But on another
note: I don't see why the PersistenceServices needs to have access to the
MaterialProvider. I feel like there shouldn't be a component that's shared
between the LeaderElectionService and the PersistenceServices.
  - Alternative: What about coming up with a factory interface
HighAvailabilityServicesFactory which provides two methods:
createLeaderElectionService & createPersistenceServices. The factory
wouldn't need to keep any instances (as suggested by this FLIP's
HighAvailabilityServices component. It's a plain factory component that
creates instances. The corresponding ZooKeeper/k8s implementation would
hold the client instance (which is the only thing that should be shared
between the LeaderElectionService and the PersistenceServices
implementations). The factory would live in the ClusterEntrypoint. Any
cleanup of HA data would be covered by the
LeaderElection|PersistenceServices, individually.

Looking forward to your opinions.
Best,
Matthias

On Tue, Jan 9, 2024 at 1:23 PM Zhu Zhu  wrote:

> > I would treat refactoring as a technical debt...
>
> Sorry I don't quite get the needs of the refactoring work.
>
> The refactoring work brings benefits if there are requirements to combine
> different leader election services and persistence services.
> The answer in this FLIP is to combine DefaultLeaderServices and
> EmbeddedPersistenceServices. But I'm concerned that, if the goal is to
> avoid the cost of job recovery, disable the persistence of the overall
> cluster might be an overkill. e.g. if later we want the cluster partitions
> to be recovered after JM failover?
>
> Yet I do not think of the needs of other new combinations at the moment,
> e.g. a non-HA leader election service with an HA persistence service,
> a ZK leader election service with a K8s persistence service. Maybe you
> have some good cases for it?
>
> TBH, the current class structure looks simpler to me. I'm also wondering
> whether it's possible to merge StandaloneHaServices with
> EmbeddedHaServices,
> because the latter one is a special case(all components in the same
> process)
> of the former one.
>
> > it still involves creating a znode or writing to the configmap
> for each job
>
> Is it possible to avoid the cost? My gut feeling is that these actions
> are not necessary after Flink does leader election for the overall master
> process.
>
> > such as checkpoint and blob storage except for the job graph store
>
> How about disabling the checkpoint to avoid the cost? I know the cost is
> there
> even if we disable the checkpoint at the moment. But I think it can be
> fixed.
> Checkpoint is not needed if job recovery is not needed, the concepts are
> highly related.
>
> Regarding blob storage, I'm not sure whether it's good to disable HA for
> it.
> If HA is disabled, the jobmanager needs to directly 

Re: Re: [VOTE] Accept Flink CDC into Apache Flink

2024-01-10 Thread Matthias Pohl
+1 (binding)

On Wed, Jan 10, 2024 at 3:35 PM ConradJam  wrote:

> +1 non-binding
>
> Dawid Wysakowicz  于2024年1月10日周三 21:06写道:
>
> > +1 (binding)
> > Best,
> > Dawid
> >
> > On Wed, 10 Jan 2024 at 11:54, Piotr Nowojski 
> wrote:
> >
> > > +1 (binding)
> > >
> > > śr., 10 sty 2024 o 11:25 Martijn Visser 
> > > napisał(a):
> > >
> > > > +1 (binding)
> > > >
> > > > On Wed, Jan 10, 2024 at 4:43 AM Xingbo Huang 
> > wrote:
> > > > >
> > > > > +1 (binding)
> > > > >
> > > > > Best,
> > > > > Xingbo
> > > > >
> > > > > Dian Fu  于2024年1月10日周三 11:35写道:
> > > > >
> > > > > > +1 (binding)
> > > > > >
> > > > > > Regards,
> > > > > > Dian
> > > > > >
> > > > > > On Wed, Jan 10, 2024 at 5:09 AM Sharath 
> > > wrote:
> > > > > > >
> > > > > > > +1 (non-binding)
> > > > > > >
> > > > > > > Best,
> > > > > > > Sharath
> > > > > > >
> > > > > > > On Tue, Jan 9, 2024 at 1:02 PM Venkata Sanath Muppalla <
> > > > > > sanath...@gmail.com>
> > > > > > > wrote:
> > > > > > >
> > > > > > > > +1 (non-binding)
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > > Sanath
> > > > > > > >
> > > > > > > > On Tue, Jan 9, 2024 at 11:16 AM Peter Huang <
> > > > > > huangzhenqiu0...@gmail.com>
> > > > > > > > wrote:
> > > > > > > >
> > > > > > > > > +1 (non-binding)
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > Best Regards
> > > > > > > > > Peter Huang
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On Tue, Jan 9, 2024 at 5:26 AM Jane Chan <
> > > qingyue@gmail.com>
> > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > +1 (non-binding)
> > > > > > > > > >
> > > > > > > > > > Best,
> > > > > > > > > > Jane
> > > > > > > > > >
> > > > > > > > > > On Tue, Jan 9, 2024 at 8:41 PM Lijie Wang <
> > > > > > wangdachui9...@gmail.com>
> > > > > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > +1 (non-binding)
> > > > > > > > > > >
> > > > > > > > > > > Best,
> > > > > > > > > > > Lijie
> > > > > > > > > > >
> > > > > > > > > > > Jiabao Sun 
> > 于2024年1月9日周二
> > > > > > 19:28写道:
> > > > > > > > > > >
> > > > > > > > > > > > +1 (non-binding)
> > > > > > > > > > > >
> > > > > > > > > > > > Best,
> > > > > > > > > > > > Jiabao
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > On 2024/01/09 09:58:04 xiangyu feng wrote:
> > > > > > > > > > > > > +1 (non-binding)
> > > > > > > > > > > > >
> > > > > > > > > > > > > Regards,
> > > > > > > > > > > > > Xiangyu Feng
> > > > > > > > > > > > >
> > > > > > > > > > > > > Danny Cranmer  于2024年1月9日周二
> > 17:50写道:
> > > > > > > > > > > > >
> > > > > > > > > > > > > > +1 (binding)
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > Danny
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Tue, Jan 9, 2024 at 9:31 AM Feng Jin <
> > > > ji...@gmail.com>
> > > > > > > > wrote:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > +1 (non-binding)
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > > Feng Jin
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > On Tue, Jan 9, 2024 at 5:29 PM Yuxin Tan <
> > > > > > ta...@gmail.com>
> > > > > > > > > > wrote:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > +1 (non-binding)
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > > > Yuxin
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Márton Balassi 
> 于2024年1月9日周二
> > > > 17:25写道:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > +1 (binding)
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > On Tue, Jan 9, 2024 at 10:15 AM Leonard Xu
> <
> > > > > > > > > xb...@gmail.com>
> > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > +1(binding)
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > > > > > Leonard
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > 2024年1月9日 下午5:08,Yangze Guo <
> > > ka...@gmail.com
> > > > >
> > > > > > 写道:
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > +1 (non-binding)
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > > > > > > Yangze Guo
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > On Tue, Jan 9, 2024 at 5:06 PM Robert
> > > > Metzger <
> > > > > > > > > > > > > > rmetz...@apache.org
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > > > >> +1 (binding)
> > > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > > > >> On Tue, Jan 9, 2024 at 9:54 AM Guowei
> > Ma <
> > > > > > > > > > gu...@gmail.com
> > > > > > > > > > > >
> > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > >>
> > > > > > > > > > 

Re: Flink pending record metric weired after autoscaler rescaling

2024-01-10 Thread Yang LI
Just to give more context, my setup uses Apache Flink 1.18 with the
adaptive scheduler enabled, issues happen randomly particularly
post-restart behaviors.

After each restart, the system logs indicate "Adding split(s) to reader:",
signifying the reassignment of partitions across different TaskManagers. An
anomaly arises with specific partitions, for example, partition-10. This
partition does not appear in the logs immediately post-restart. It remains
unlogged for several hours, during which no data consumption from
partition-10 occurs. Subsequently, the logs display "Discovered new
partitions:", and only then does the consumption of data from partition-10
recommence.

Could you provide any insights or hypotheses regarding the underlying cause
of this delayed recognition and processing of certain partitions?

Best regards,
Yang

On Mon, 8 Jan 2024 at 16:24, Yang LI  wrote:

> Dear Flink Community,
>
> I've encountered an issue during the testing of my Flink autoscaler. It
> appears that Flink is losing track of specific Kafka partitions, leading to
> a persistent increase in lag on these partitions. The logs indicate a
> 'kafka connector metricGroup name collision exception.' Notably, the
> consumption on these Kafka partitions returns to normal after restarting
> the Kafka broker. For context, I have enabled in-place rescaling support
> with 'jobmanager.scheduler: Adaptive.'
>
> I suspect the problem may stem from:
>
> The in-place rescaling support triggering restarts of some taskmanagers.
> This process might not be restarting the metric groups registered by the
> Kafka source connector correctly, leading to a name collision exception and
> preventing Flink from accurately reporting metrics related to Kafka
> consumption.
> A potential edge case in the metric for pending records, especially when
> different partitions exhibit varying lags. This discrepancy might be
> causing the pending record metric to malfunction.
> I would appreciate your insights on these observations.
>
> Best regards,
> Yang LI
>


Re: [DISCUSS] FLIP-415: Introduce a new join operator to support minibatch

2024-01-10 Thread Benchao Li
Thanks shuai for driving this, mini-batch Join is a very useful
optimization, +1 for the general idea.

Regarding the configuration
"table.exec.stream.join.mini-batch-enabled", I'm not sure it's really
necessary. The semantic of changelog emitted by the Join operator is
eventual consistency, so there is no much difference between original
Join and mini-batch Join from this aspect. Besides, introducing more
options would make it more complex for users, harder to understand and
maintain, which we should be careful about.

One thing about the implementation, could you make the new operator
share the same state definition with the original one?

shuai xu  于2024年1月10日周三 21:23写道:
>
> Hi devs,
>
> I’d like to start a discussion on FLIP-415: Introduce a new join operator to 
> support minibatch[1].
>
> Currently, when performing cascading connections in Flink, there is a pain 
> point of record amplification. Every record join operator receives would 
> trigger join process. However, if records of +I and -D matches , they could 
> be folded to reduce two times of join process. Besides, records of  -U +U 
> might output 4 records in which two records are redundant when encountering 
> outer join .
>
> To address this issue, this FLIP introduces a new  
> MiniBatchStreamingJoinOperator to achieve batch processing which could reduce 
> number of outputting redundant messages and avoid unnecessary join processes.
> A new option is added to control the operator to avoid influencing existing 
> 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-415%3A+Introduce+a+new+join+operator+to+support+minibatch
>
> Best,
> Xu Shuai



-- 

Best,
Benchao Li


[jira] [Created] (FLINK-34052) Missing TopSpeedWindowing and SessionWindowing JARs in Flink Maven Repository

2024-01-10 Thread Junrui Li (Jira)
Junrui Li created FLINK-34052:
-

 Summary: Missing TopSpeedWindowing and SessionWindowing JARs in 
Flink Maven Repository
 Key: FLINK-34052
 URL: https://issues.apache.org/jira/browse/FLINK-34052
 Project: Flink
  Issue Type: Bug
  Components: Build System
Affects Versions: 1.18.0
Reporter: Junrui Li


As a result of the changes implemented in FLINK-32821, the build process no 
longer produces artifacts with the names 
flink-examples-streaming-1.x-TopSpeedWindowing.jar and 
flink-examples-streaming-1.x-SessionWindowing.jar. This has led to the absence 
of these specific JAR files in the Maven repository 
(https://repo1.maven.org/maven2/org/apache/flink/flink-examples-streaming/1.18.0/).

These artifacts were previously available and may still be expected by users as 
part of their application dependencies. Their removal could potentially break 
existing build pipelines and applications that depend on these example JARs.



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


Re: Re: [VOTE] Accept Flink CDC into Apache Flink

2024-01-10 Thread ConradJam
+1 non-binding

Dawid Wysakowicz  于2024年1月10日周三 21:06写道:

> +1 (binding)
> Best,
> Dawid
>
> On Wed, 10 Jan 2024 at 11:54, Piotr Nowojski  wrote:
>
> > +1 (binding)
> >
> > śr., 10 sty 2024 o 11:25 Martijn Visser 
> > napisał(a):
> >
> > > +1 (binding)
> > >
> > > On Wed, Jan 10, 2024 at 4:43 AM Xingbo Huang 
> wrote:
> > > >
> > > > +1 (binding)
> > > >
> > > > Best,
> > > > Xingbo
> > > >
> > > > Dian Fu  于2024年1月10日周三 11:35写道:
> > > >
> > > > > +1 (binding)
> > > > >
> > > > > Regards,
> > > > > Dian
> > > > >
> > > > > On Wed, Jan 10, 2024 at 5:09 AM Sharath 
> > wrote:
> > > > > >
> > > > > > +1 (non-binding)
> > > > > >
> > > > > > Best,
> > > > > > Sharath
> > > > > >
> > > > > > On Tue, Jan 9, 2024 at 1:02 PM Venkata Sanath Muppalla <
> > > > > sanath...@gmail.com>
> > > > > > wrote:
> > > > > >
> > > > > > > +1 (non-binding)
> > > > > > >
> > > > > > > Thanks,
> > > > > > > Sanath
> > > > > > >
> > > > > > > On Tue, Jan 9, 2024 at 11:16 AM Peter Huang <
> > > > > huangzhenqiu0...@gmail.com>
> > > > > > > wrote:
> > > > > > >
> > > > > > > > +1 (non-binding)
> > > > > > > >
> > > > > > > >
> > > > > > > > Best Regards
> > > > > > > > Peter Huang
> > > > > > > >
> > > > > > > >
> > > > > > > > On Tue, Jan 9, 2024 at 5:26 AM Jane Chan <
> > qingyue@gmail.com>
> > > > > wrote:
> > > > > > > >
> > > > > > > > > +1 (non-binding)
> > > > > > > > >
> > > > > > > > > Best,
> > > > > > > > > Jane
> > > > > > > > >
> > > > > > > > > On Tue, Jan 9, 2024 at 8:41 PM Lijie Wang <
> > > > > wangdachui9...@gmail.com>
> > > > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > +1 (non-binding)
> > > > > > > > > >
> > > > > > > > > > Best,
> > > > > > > > > > Lijie
> > > > > > > > > >
> > > > > > > > > > Jiabao Sun 
> 于2024年1月9日周二
> > > > > 19:28写道:
> > > > > > > > > >
> > > > > > > > > > > +1 (non-binding)
> > > > > > > > > > >
> > > > > > > > > > > Best,
> > > > > > > > > > > Jiabao
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > On 2024/01/09 09:58:04 xiangyu feng wrote:
> > > > > > > > > > > > +1 (non-binding)
> > > > > > > > > > > >
> > > > > > > > > > > > Regards,
> > > > > > > > > > > > Xiangyu Feng
> > > > > > > > > > > >
> > > > > > > > > > > > Danny Cranmer  于2024年1月9日周二
> 17:50写道:
> > > > > > > > > > > >
> > > > > > > > > > > > > +1 (binding)
> > > > > > > > > > > > >
> > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > Danny
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Tue, Jan 9, 2024 at 9:31 AM Feng Jin <
> > > ji...@gmail.com>
> > > > > > > wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > > +1 (non-binding)
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > Feng Jin
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Tue, Jan 9, 2024 at 5:29 PM Yuxin Tan <
> > > > > ta...@gmail.com>
> > > > > > > > > wrote:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > +1 (non-binding)
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > > Yuxin
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Márton Balassi  于2024年1月9日周二
> > > 17:25写道:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > +1 (binding)
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > On Tue, Jan 9, 2024 at 10:15 AM Leonard Xu <
> > > > > > > > xb...@gmail.com>
> > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > +1(binding)
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > > > > Leonard
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > 2024年1月9日 下午5:08,Yangze Guo <
> > ka...@gmail.com
> > > >
> > > > > 写道:
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > +1 (non-binding)
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > > > > > Yangze Guo
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > On Tue, Jan 9, 2024 at 5:06 PM Robert
> > > Metzger <
> > > > > > > > > > > > > rmetz...@apache.org
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > > >> +1 (binding)
> > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > > >> On Tue, Jan 9, 2024 at 9:54 AM Guowei
> Ma <
> > > > > > > > > gu...@gmail.com
> > > > > > > > > > >
> > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > > >>> +1 (binding)
> > > > > > > > > > > > > > > > > >>> Best,
> > > > > > > > > > > > > > > > > >>> Guowei
> > > > > > > > > > > > > > > > > >>>
> > > > > > > > > > > > > > > > > >>>
> > > > > > > > > > > > > > > > > >>> On Tue, Jan 9, 2024 at 4:49 PM Rui Fan
> <
> > > > > > > > > 19...@gmail.com>
> > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > >>>
> 

[jira] [Created] (FLINK-34051) Fix equals/hashCode/toString for SavepointRestoreSettings

2024-01-10 Thread Hangxiang Yu (Jira)
Hangxiang Yu created FLINK-34051:


 Summary: Fix equals/hashCode/toString for SavepointRestoreSettings
 Key: FLINK-34051
 URL: https://issues.apache.org/jira/browse/FLINK-34051
 Project: Flink
  Issue Type: Bug
  Components: Runtime / Checkpointing
Affects Versions: 1.19.0
Reporter: Hangxiang Yu
Assignee: Hangxiang Yu


SavepointRestoreSettings#equals/hashCode/toString missed restoreMode property



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


Re: [DISCUSS] Proposing an LTS Release for the 1.x Line

2024-01-10 Thread Martijn Visser
Hi Alex,

I saw that I missed replying to this topic. I do think that Xintong
touched on an important topic when he mentioned that we should define
what an LTS version means. From my point of view, I would state that
an LTS version for Apache Flink means that bug fixes only will be made
available for a longer period of time. I think that, combined with
what you called option 1 (a clear end-of-life date) is the best
option.

Flink 2.0 will give us primarily the ability to remove a lot of
deprecated APIs, especially with Flink's deprecation strategy. I
expect that the majority of users will have an easy migration path
from a Flink 1.x to a Flink 2.0, if you're currently not using a
deprecated API and are a Java user.

Allowing backporting of new features to Flink 1.x will result in users
delaying the upgrade, but it doesn't make the upgrade any easier when
they must upgrade. New features will also introduce new bugs, meaning
that maintainers will have to spend time on two release versions. As
the codebases diverge more and more, this will just become
increasingly more complex.

With that being said, I do think that it makes sense to also formalize
the result of this discussion in a FLIP. That's just easier to point
users towards at a later stage.

Best regards,

Martijn

On Mon, Dec 4, 2023 at 9:55 PM Alexander Fedulov
 wrote:
>
> Hi everyone,
>
> As we progress with the 1.19 release, which might potentially (although not
> likely) be the last in the 1.x line, I'd like to revive our discussion on
> the
> LTS support matter. There is a general consensus that due to breaking API
> changes in 2.0, extending bug fixes support by designating an LTS release
> is
> something we want to do.
>
> To summarize, the approaches we've considered are:
>
> Time-based: The last release of the 1.x line gets a clear end-of-life date
> (2 years).
> Release-based: The last release of the 1.x line gets support for 4 minor
> releases in the 2.x line. The exact time is unknown, but we assume it to be
> roughly 2 years.
> LTS-to-LTS release: The last release of the 1.x line is supported until the
> last release in the 2.x line is designated as LTS.
>
> We need to strike a balance between being user-friendly and nudging people
> to
> upgrade. From that perspective, option 1 is my favorite - we all know that
> having a clear deadline works wonders in motivating action. At the same
> time,
> I appreciate that we might not want to introduce new kinds of procedures,
> so
> option 2 would be my second choice, provided we also formulate it like "4
> minor releases, at least 2 years" (in case the minor release cadence
> accelerates for some reason). I find option 3 a bit problematic since it
> gives no incentive to upgrade, and people who do not follow Flink
> development
> closely might be caught by surprise when we decide to bump the major
> version
> again.
>
> I'd like to open a vote to stamp the official decision, but first, I would
> like to understand if we can reach consensus on one of the options here, or
> if
> we'll need to push that to a vote by presenting multiple options.
>
> Looking forward to hearing your thoughts on this matter.
>
> P.S.: 1.x and 2.x are just examples in this case; the decision also
> translates
> into a procedure for future major releases.
>
> Best,
> Alex
>
> On Thu, 27 Jul 2023 at 17:32, Jing Ge  wrote:
>
> > Hi Konstantin,
> >
> > What you said makes sense. Dropping modules is an efficient option to
> > accelerate Flink evolution with acceptable function regressions. We should
> > do it constantly and strategically. On the other hand, we should point out
> > the core modules that must be backward compatible when a new major version
> > is released.
> >
> > Best regards,
> > Jing
> >
> > On Wed, Jul 26, 2023 at 10:52 PM Matthias Pohl
> >  wrote:
> >
> > > >
> > > > @Mathias, I am not quite sure about the 3 versions description. Are you
> > > > concerned that 1.x and 2.x LTS releases could overlap, if 3.0 comes
> > > early?
> > >
> > > Yes. Maybe, that's only a theoretical scenario. It wouldn't work if we go
> > > with your suggestion to use "proper time" rather than release cycles to
> > > define the length of a support period (which sounds reasonable). My
> > concern
> > > was that we get into a situation where we need to support four versions
> > of
> > > Flink.
> > >
> > > On Wed, Jul 26, 2023 at 4:21 PM Alexander Fedulov <
> > > alexander.fedu...@gmail.com> wrote:
> > >
> > > > The question is if we want to tie the release cycle of 2.x to how much
> > > time
> > > > we give our users to migrate. And "time" is a critical word here. I can
> > > see
> > > > us
> > > > potentially wanting to iterate on the 2.x line more rapidly, because of
> > > all
> > > > of the
> > > > major changes, until the cycles get settled to a typical cadence again.
> > > >
> > > > This means that user's won't know how much time they would have to
> > > actually
> > > > migrate off of 1.x. And I can see this knowledge being 

[jira] [Created] (FLINK-34050) Rocksdb state has space amplification after rescaling with DeleteRange

2024-01-10 Thread Jinzhong Li (Jira)
Jinzhong Li created FLINK-34050:
---

 Summary: Rocksdb state has space amplification after rescaling 
with DeleteRange
 Key: FLINK-34050
 URL: https://issues.apache.org/jira/browse/FLINK-34050
 Project: Flink
  Issue Type: Bug
  Components: Runtime / State Backends
Reporter: Jinzhong Li
 Attachments: image-2024-01-10-21-23-48-134.png, 
image-2024-01-10-21-24-10-983.png, image-2024-01-10-21-28-24-312.png

FLINK-21321 use deleteRange to speed up rocksdb rescaling, however it will 
cause space amplification in some case.

We can reproduce this problem using wordCount job:

1) before rescaling, state operator in wordCount job has 2 parallelism and 4G+ 
full checkpoint size;

!image-2024-01-10-21-24-10-983.png|width=266,height=130!

2) then restart job with 4 parallelism (for state operator),  the full 
checkpoint size of new job will be 8G+ ;

3) after many successful checkpoints, the full checkpoint size is still 8G+;

!image-2024-01-10-21-28-24-312.png|width=454,height=111!

 

The root cause of this issue is that the deleted keyGroupRange does not overlap 
with current DB keyGroupRange, so new data written into rocksdb after rescaling 
almost never do LSM compaction with the deleted data (belonging to other 
keyGroupRange.)

 

And the space amplification may affect Rocksdb read performance and disk space 
usage after rescaling. It looks like a regression due to the introduction of 
deleteRange for rescaling optimization.

 

To slove this problem, I think maybe we can invoke Rocksdb.deleteFilesInRanges 
after deleteRange?
{code:java}
public static void clipDBWithKeyGroupRange() {
  //...
  List ranges = new ArrayList<>();
  //...
  deleteRange(db, columnFamilyHandles, beginKeyGroupBytes, endKeyGroupBytes);
  ranges.add(beginKeyGroupBytes);
  ranges.add(endKeyGroupBytes);
  //

  for (ColumnFamilyHandle columnFamilyHandle : columnFamilyHandles) {
 db.deleteFilesInRanges(columnFamilyHandle, ranges, false);
  }
}


{code}
 



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


[DISCUSS] FLIP-415: Introduce a new join operator to support minibatch

2024-01-10 Thread shuai xu
Hi devs,

I’d like to start a discussion on FLIP-415: Introduce a new join operator to 
support minibatch[1].

Currently, when performing cascading connections in Flink, there is a pain 
point of record amplification. Every record join operator receives would 
trigger join process. However, if records of +I and -D matches , they could be 
folded to reduce two times of join process. Besides, records of  -U +U might 
output 4 records in which two records are redundant when encountering outer 
join . 

To address this issue, this FLIP introduces a new  
MiniBatchStreamingJoinOperator to achieve batch processing which could reduce 
number of outputting redundant messages and avoid unnecessary join processes. 
A new option is added to control the operator to avoid influencing existing 
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-415%3A+Introduce+a+new+join+operator+to+support+minibatch

Best,
Xu Shuai


Re: Re: [VOTE] Accept Flink CDC into Apache Flink

2024-01-10 Thread Dawid Wysakowicz
+1 (binding)
Best,
Dawid

On Wed, 10 Jan 2024 at 11:54, Piotr Nowojski  wrote:

> +1 (binding)
>
> śr., 10 sty 2024 o 11:25 Martijn Visser 
> napisał(a):
>
> > +1 (binding)
> >
> > On Wed, Jan 10, 2024 at 4:43 AM Xingbo Huang  wrote:
> > >
> > > +1 (binding)
> > >
> > > Best,
> > > Xingbo
> > >
> > > Dian Fu  于2024年1月10日周三 11:35写道:
> > >
> > > > +1 (binding)
> > > >
> > > > Regards,
> > > > Dian
> > > >
> > > > On Wed, Jan 10, 2024 at 5:09 AM Sharath 
> wrote:
> > > > >
> > > > > +1 (non-binding)
> > > > >
> > > > > Best,
> > > > > Sharath
> > > > >
> > > > > On Tue, Jan 9, 2024 at 1:02 PM Venkata Sanath Muppalla <
> > > > sanath...@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > +1 (non-binding)
> > > > > >
> > > > > > Thanks,
> > > > > > Sanath
> > > > > >
> > > > > > On Tue, Jan 9, 2024 at 11:16 AM Peter Huang <
> > > > huangzhenqiu0...@gmail.com>
> > > > > > wrote:
> > > > > >
> > > > > > > +1 (non-binding)
> > > > > > >
> > > > > > >
> > > > > > > Best Regards
> > > > > > > Peter Huang
> > > > > > >
> > > > > > >
> > > > > > > On Tue, Jan 9, 2024 at 5:26 AM Jane Chan <
> qingyue@gmail.com>
> > > > wrote:
> > > > > > >
> > > > > > > > +1 (non-binding)
> > > > > > > >
> > > > > > > > Best,
> > > > > > > > Jane
> > > > > > > >
> > > > > > > > On Tue, Jan 9, 2024 at 8:41 PM Lijie Wang <
> > > > wangdachui9...@gmail.com>
> > > > > > > > wrote:
> > > > > > > >
> > > > > > > > > +1 (non-binding)
> > > > > > > > >
> > > > > > > > > Best,
> > > > > > > > > Lijie
> > > > > > > > >
> > > > > > > > > Jiabao Sun  于2024年1月9日周二
> > > > 19:28写道:
> > > > > > > > >
> > > > > > > > > > +1 (non-binding)
> > > > > > > > > >
> > > > > > > > > > Best,
> > > > > > > > > > Jiabao
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > On 2024/01/09 09:58:04 xiangyu feng wrote:
> > > > > > > > > > > +1 (non-binding)
> > > > > > > > > > >
> > > > > > > > > > > Regards,
> > > > > > > > > > > Xiangyu Feng
> > > > > > > > > > >
> > > > > > > > > > > Danny Cranmer  于2024年1月9日周二 17:50写道:
> > > > > > > > > > >
> > > > > > > > > > > > +1 (binding)
> > > > > > > > > > > >
> > > > > > > > > > > > Thanks,
> > > > > > > > > > > > Danny
> > > > > > > > > > > >
> > > > > > > > > > > > On Tue, Jan 9, 2024 at 9:31 AM Feng Jin <
> > ji...@gmail.com>
> > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > +1 (non-binding)
> > > > > > > > > > > > >
> > > > > > > > > > > > > Best,
> > > > > > > > > > > > > Feng Jin
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Tue, Jan 9, 2024 at 5:29 PM Yuxin Tan <
> > > > ta...@gmail.com>
> > > > > > > > wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > > +1 (non-binding)
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > Yuxin
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Márton Balassi  于2024年1月9日周二
> > 17:25写道:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > +1 (binding)
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > On Tue, Jan 9, 2024 at 10:15 AM Leonard Xu <
> > > > > > > xb...@gmail.com>
> > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > +1(binding)
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > > > Leonard
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > 2024年1月9日 下午5:08,Yangze Guo <
> ka...@gmail.com
> > >
> > > > 写道:
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > +1 (non-binding)
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > > > > Yangze Guo
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > On Tue, Jan 9, 2024 at 5:06 PM Robert
> > Metzger <
> > > > > > > > > > > > rmetz...@apache.org
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > >> +1 (binding)
> > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > >> On Tue, Jan 9, 2024 at 9:54 AM Guowei Ma <
> > > > > > > > gu...@gmail.com
> > > > > > > > > >
> > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > >>> +1 (binding)
> > > > > > > > > > > > > > > > >>> Best,
> > > > > > > > > > > > > > > > >>> Guowei
> > > > > > > > > > > > > > > > >>>
> > > > > > > > > > > > > > > > >>>
> > > > > > > > > > > > > > > > >>> On Tue, Jan 9, 2024 at 4:49 PM Rui Fan <
> > > > > > > > 19...@gmail.com>
> > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > >>>
> > > > > > > > > > > > > > > >  +1 (non-binding)
> > > > > > > > > > > > > > > > 
> > > > > > > > > > > > > > > >  Best,
> > > > > > > > > > > > > > > >  Rui
> > > > > > > > > > > > > > > > 
> > > > > > > > > > > > > > > >  On Tue, Jan 9, 2024 at 4:41 PM Hang
> Ruan <
> > > > > > > > > > > > > ruanhang1...@gmail.com>
> > > > > > > > > > > > > > > > wrote:
> > > > 

Re: [VOTE] Release 1.18.1, release candidate #2

2024-01-10 Thread Sergey Nuyanzin
+1 (non-binding)

-verified checksums
-verified signatures
-checked release tag
-verified that there is no binary in source
-built from sources

On Wed, Jan 10, 2024 at 12:30 PM Leonard Xu  wrote:

> +1 (binding)
>
> - verified signatures
> - verified hashsums
> - checked Github release tag
> - started SQL Client, used MySQL CDC connector to capture data change from
> database , the result is expected
> - reviewed the web PR
> - reviewed the release notes PR
>
> Best,
> Leonard
>
> > 2024年1月10日 上午10:48,Qingsheng Ren  写道:
> >
> > +1 (binding)
> >
> > - Built from source
> > - Verified checksum and signature
> > - Verified that no binary exist in source
> > - Started a standalone cluster and submit Kafka consuming and producing
> job
> > with SQL client
> > - Reviewed web PR
> >
> > Thanks for driving this, Jing!
> >
> > Best,
> > Qingsheng
> >
> > On Mon, Jan 8, 2024 at 8:02 PM Jark Wu  wrote:
> >
> >> Thanks Jing for driving this.
> >>
> >> +1 (binding)
> >>
> >> - Build and compile the source code locally: *OK*
> >> - Verified signatures and hashes: *OK*
> >> - Checked no missing artifacts in the staging area: *OK*
> >> - Reviewed the website release PR: *OK*
> >> - Went through the quick start: *OK*
> >>  * Started a cluster and ran the examples
> >>  * Verified web ui and log output, nothing unexpected
> >>
> >> Best,
> >> Jark
> >>
> >> On Thu, 28 Dec 2023 at 20:59, Yun Tang  wrote:
> >>
> >>> Thanks Jing for driving this release.
> >>>
> >>> +1 (non-binding)
> >>>
> >>>
> >>>  *
> >>> Download artifacts and verify the signatures.
> >>>  *
> >>> Verified the web PR
> >>>  *
> >>> Verified the number of Python packages is 11
> >>>  *
> >>> Started a local cluster and verified FLIP-291 to see the rescale
> results.
> >>>  *
> >>> Verified the jar packages were built with JDK8
> >>>
> >>> Best
> >>> Yun Tang
> >>>
> >>>
> >>> 
> >>> From: Rui Fan <1996fan...@gmail.com>
> >>> Sent: Thursday, December 28, 2023 10:54
> >>> To: dev@flink.apache.org 
> >>> Subject: Re: [VOTE] Release 1.18.1, release candidate #2
> >>>
> >>> Thanks Jing for driving this release!
> >>>
> >>> +1(non-binding)
> >>>
> >>> - Downloaded artifacts
> >>> - Verified signatures and sha512
> >>> - The source archives do not contain any binaries
> >>> - Verified web PR
> >>> - Build the source with Maven 3 and java8 (Checked the license as well)
> >>> - bin/start-cluster.sh with java8, it works fine and no any unexpected
> >> LOG-
> >>> Ran demo, it's fine:  bin/flink
> >>> runexamples/streaming/StateMachineExample.jar
> >>>
> >>> Best,
> >>> Rui
> >>>
> >>> On Wed, Dec 27, 2023 at 8:45 PM Martijn Visser <
> martijnvis...@apache.org
> >>>
> >>> wrote:
> >>>
>  Hi Jing,
> 
>  Thanks for driving this.
> 
>  +1 (binding)
> 
>  - Validated hashes
>  - Verified signature
>  - Verified that no binaries exist in the source archive
>  - Build the source with Maven via mvn clean install
>  -Pcheck-convergence -Dflink.version=1.18.1
>  - Verified licenses
>  - Verified web PR
>  - Started a cluster and the Flink SQL client, successfully read and
>  wrote with the Kafka connector to Confluent Cloud with AVRO and Schema
>  Registry enabled
>  - Started a cluster and submitted a job that checkpoints to GCS
> without
>  problems
> 
>  Best regards,
> 
>  Martijn
> 
>  On Thu, Dec 21, 2023 at 4:55 AM gongzhongqiang
>   wrote:
> >
> > Thanks Jing Ge for driving this release.
> >
> > +1 (non-binding), I have checked:
> > [✓] The checksums and signatures are validated
> > [✓] The tag checked is fine
> > [✓] Built from source is passed
> > [✓] The flink-web PR is reviewed and checked
> >
> >
> > Best,
> > Zhongqiang Gong
> 
> >>>
> >>
>
>

-- 
Best regards,
Sergey


Re: [VOTE] Release 1.18.1, release candidate #2

2024-01-10 Thread Leonard Xu
+1 (binding)

- verified signatures
- verified hashsums 
- checked Github release tag 
- started SQL Client, used MySQL CDC connector to capture data change from 
database , the result is expected
- reviewed the web PR
- reviewed the release notes PR

Best,
Leonard

> 2024年1月10日 上午10:48,Qingsheng Ren  写道:
> 
> +1 (binding)
> 
> - Built from source
> - Verified checksum and signature
> - Verified that no binary exist in source
> - Started a standalone cluster and submit Kafka consuming and producing job
> with SQL client
> - Reviewed web PR
> 
> Thanks for driving this, Jing!
> 
> Best,
> Qingsheng
> 
> On Mon, Jan 8, 2024 at 8:02 PM Jark Wu  wrote:
> 
>> Thanks Jing for driving this.
>> 
>> +1 (binding)
>> 
>> - Build and compile the source code locally: *OK*
>> - Verified signatures and hashes: *OK*
>> - Checked no missing artifacts in the staging area: *OK*
>> - Reviewed the website release PR: *OK*
>> - Went through the quick start: *OK*
>>  * Started a cluster and ran the examples
>>  * Verified web ui and log output, nothing unexpected
>> 
>> Best,
>> Jark
>> 
>> On Thu, 28 Dec 2023 at 20:59, Yun Tang  wrote:
>> 
>>> Thanks Jing for driving this release.
>>> 
>>> +1 (non-binding)
>>> 
>>> 
>>>  *
>>> Download artifacts and verify the signatures.
>>>  *
>>> Verified the web PR
>>>  *
>>> Verified the number of Python packages is 11
>>>  *
>>> Started a local cluster and verified FLIP-291 to see the rescale results.
>>>  *
>>> Verified the jar packages were built with JDK8
>>> 
>>> Best
>>> Yun Tang
>>> 
>>> 
>>> 
>>> From: Rui Fan <1996fan...@gmail.com>
>>> Sent: Thursday, December 28, 2023 10:54
>>> To: dev@flink.apache.org 
>>> Subject: Re: [VOTE] Release 1.18.1, release candidate #2
>>> 
>>> Thanks Jing for driving this release!
>>> 
>>> +1(non-binding)
>>> 
>>> - Downloaded artifacts
>>> - Verified signatures and sha512
>>> - The source archives do not contain any binaries
>>> - Verified web PR
>>> - Build the source with Maven 3 and java8 (Checked the license as well)
>>> - bin/start-cluster.sh with java8, it works fine and no any unexpected
>> LOG-
>>> Ran demo, it's fine:  bin/flink
>>> runexamples/streaming/StateMachineExample.jar
>>> 
>>> Best,
>>> Rui
>>> 
>>> On Wed, Dec 27, 2023 at 8:45 PM Martijn Visser >> 
>>> wrote:
>>> 
 Hi Jing,
 
 Thanks for driving this.
 
 +1 (binding)
 
 - Validated hashes
 - Verified signature
 - Verified that no binaries exist in the source archive
 - Build the source with Maven via mvn clean install
 -Pcheck-convergence -Dflink.version=1.18.1
 - Verified licenses
 - Verified web PR
 - Started a cluster and the Flink SQL client, successfully read and
 wrote with the Kafka connector to Confluent Cloud with AVRO and Schema
 Registry enabled
 - Started a cluster and submitted a job that checkpoints to GCS without
 problems
 
 Best regards,
 
 Martijn
 
 On Thu, Dec 21, 2023 at 4:55 AM gongzhongqiang
  wrote:
> 
> Thanks Jing Ge for driving this release.
> 
> +1 (non-binding), I have checked:
> [✓] The checksums and signatures are validated
> [✓] The tag checked is fine
> [✓] Built from source is passed
> [✓] The flink-web PR is reviewed and checked
> 
> 
> Best,
> Zhongqiang Gong
 
>>> 
>> 



Re: Re: [VOTE] Accept Flink CDC into Apache Flink

2024-01-10 Thread Piotr Nowojski
+1 (binding)

śr., 10 sty 2024 o 11:25 Martijn Visser 
napisał(a):

> +1 (binding)
>
> On Wed, Jan 10, 2024 at 4:43 AM Xingbo Huang  wrote:
> >
> > +1 (binding)
> >
> > Best,
> > Xingbo
> >
> > Dian Fu  于2024年1月10日周三 11:35写道:
> >
> > > +1 (binding)
> > >
> > > Regards,
> > > Dian
> > >
> > > On Wed, Jan 10, 2024 at 5:09 AM Sharath  wrote:
> > > >
> > > > +1 (non-binding)
> > > >
> > > > Best,
> > > > Sharath
> > > >
> > > > On Tue, Jan 9, 2024 at 1:02 PM Venkata Sanath Muppalla <
> > > sanath...@gmail.com>
> > > > wrote:
> > > >
> > > > > +1 (non-binding)
> > > > >
> > > > > Thanks,
> > > > > Sanath
> > > > >
> > > > > On Tue, Jan 9, 2024 at 11:16 AM Peter Huang <
> > > huangzhenqiu0...@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > +1 (non-binding)
> > > > > >
> > > > > >
> > > > > > Best Regards
> > > > > > Peter Huang
> > > > > >
> > > > > >
> > > > > > On Tue, Jan 9, 2024 at 5:26 AM Jane Chan 
> > > wrote:
> > > > > >
> > > > > > > +1 (non-binding)
> > > > > > >
> > > > > > > Best,
> > > > > > > Jane
> > > > > > >
> > > > > > > On Tue, Jan 9, 2024 at 8:41 PM Lijie Wang <
> > > wangdachui9...@gmail.com>
> > > > > > > wrote:
> > > > > > >
> > > > > > > > +1 (non-binding)
> > > > > > > >
> > > > > > > > Best,
> > > > > > > > Lijie
> > > > > > > >
> > > > > > > > Jiabao Sun  于2024年1月9日周二
> > > 19:28写道:
> > > > > > > >
> > > > > > > > > +1 (non-binding)
> > > > > > > > >
> > > > > > > > > Best,
> > > > > > > > > Jiabao
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On 2024/01/09 09:58:04 xiangyu feng wrote:
> > > > > > > > > > +1 (non-binding)
> > > > > > > > > >
> > > > > > > > > > Regards,
> > > > > > > > > > Xiangyu Feng
> > > > > > > > > >
> > > > > > > > > > Danny Cranmer  于2024年1月9日周二 17:50写道:
> > > > > > > > > >
> > > > > > > > > > > +1 (binding)
> > > > > > > > > > >
> > > > > > > > > > > Thanks,
> > > > > > > > > > > Danny
> > > > > > > > > > >
> > > > > > > > > > > On Tue, Jan 9, 2024 at 9:31 AM Feng Jin <
> ji...@gmail.com>
> > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > +1 (non-binding)
> > > > > > > > > > > >
> > > > > > > > > > > > Best,
> > > > > > > > > > > > Feng Jin
> > > > > > > > > > > >
> > > > > > > > > > > > On Tue, Jan 9, 2024 at 5:29 PM Yuxin Tan <
> > > ta...@gmail.com>
> > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > +1 (non-binding)
> > > > > > > > > > > > >
> > > > > > > > > > > > > Best,
> > > > > > > > > > > > > Yuxin
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > Márton Balassi  于2024年1月9日周二
> 17:25写道:
> > > > > > > > > > > > >
> > > > > > > > > > > > > > +1 (binding)
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Tue, Jan 9, 2024 at 10:15 AM Leonard Xu <
> > > > > > xb...@gmail.com>
> > > > > > > > > > > wrote:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > +1(binding)
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > > Leonard
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 2024年1月9日 下午5:08,Yangze Guo  >
> > > 写道:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > +1 (non-binding)
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > > > Yangze Guo
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > On Tue, Jan 9, 2024 at 5:06 PM Robert
> Metzger <
> > > > > > > > > > > rmetz...@apache.org
> > > > > > > > > > > > >
> > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > >> +1 (binding)
> > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > >> On Tue, Jan 9, 2024 at 9:54 AM Guowei Ma <
> > > > > > > gu...@gmail.com
> > > > > > > > >
> > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > >>> +1 (binding)
> > > > > > > > > > > > > > > >>> Best,
> > > > > > > > > > > > > > > >>> Guowei
> > > > > > > > > > > > > > > >>>
> > > > > > > > > > > > > > > >>>
> > > > > > > > > > > > > > > >>> On Tue, Jan 9, 2024 at 4:49 PM Rui Fan <
> > > > > > > 19...@gmail.com>
> > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > >>>
> > > > > > > > > > > > > > >  +1 (non-binding)
> > > > > > > > > > > > > > > 
> > > > > > > > > > > > > > >  Best,
> > > > > > > > > > > > > > >  Rui
> > > > > > > > > > > > > > > 
> > > > > > > > > > > > > > >  On Tue, Jan 9, 2024 at 4:41 PM Hang Ruan <
> > > > > > > > > > > > ruanhang1...@gmail.com>
> > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > 
> > > > > > > > > > > > > > > > +1 (non-binding)
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > > > Hang
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > gongzhongqiang 
> > > 于2024年1月9日周二
> > > > > > > > > > > > 16:25写道:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >> +1 non-binding
> > > > > > > > > > > 

Re: Re: [VOTE] Accept Flink CDC into Apache Flink

2024-01-10 Thread Maximilian Michels
+1 (binding)

On Wed, Jan 10, 2024 at 11:22 AM Martijn Visser
 wrote:
>
> +1 (binding)
>
> On Wed, Jan 10, 2024 at 4:43 AM Xingbo Huang  wrote:
> >
> > +1 (binding)
> >
> > Best,
> > Xingbo
> >
> > Dian Fu  于2024年1月10日周三 11:35写道:
> >
> > > +1 (binding)
> > >
> > > Regards,
> > > Dian
> > >
> > > On Wed, Jan 10, 2024 at 5:09 AM Sharath  wrote:
> > > >
> > > > +1 (non-binding)
> > > >
> > > > Best,
> > > > Sharath
> > > >
> > > > On Tue, Jan 9, 2024 at 1:02 PM Venkata Sanath Muppalla <
> > > sanath...@gmail.com>
> > > > wrote:
> > > >
> > > > > +1 (non-binding)
> > > > >
> > > > > Thanks,
> > > > > Sanath
> > > > >
> > > > > On Tue, Jan 9, 2024 at 11:16 AM Peter Huang <
> > > huangzhenqiu0...@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > +1 (non-binding)
> > > > > >
> > > > > >
> > > > > > Best Regards
> > > > > > Peter Huang
> > > > > >
> > > > > >
> > > > > > On Tue, Jan 9, 2024 at 5:26 AM Jane Chan 
> > > wrote:
> > > > > >
> > > > > > > +1 (non-binding)
> > > > > > >
> > > > > > > Best,
> > > > > > > Jane
> > > > > > >
> > > > > > > On Tue, Jan 9, 2024 at 8:41 PM Lijie Wang <
> > > wangdachui9...@gmail.com>
> > > > > > > wrote:
> > > > > > >
> > > > > > > > +1 (non-binding)
> > > > > > > >
> > > > > > > > Best,
> > > > > > > > Lijie
> > > > > > > >
> > > > > > > > Jiabao Sun  于2024年1月9日周二
> > > 19:28写道:
> > > > > > > >
> > > > > > > > > +1 (non-binding)
> > > > > > > > >
> > > > > > > > > Best,
> > > > > > > > > Jiabao
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On 2024/01/09 09:58:04 xiangyu feng wrote:
> > > > > > > > > > +1 (non-binding)
> > > > > > > > > >
> > > > > > > > > > Regards,
> > > > > > > > > > Xiangyu Feng
> > > > > > > > > >
> > > > > > > > > > Danny Cranmer  于2024年1月9日周二 17:50写道:
> > > > > > > > > >
> > > > > > > > > > > +1 (binding)
> > > > > > > > > > >
> > > > > > > > > > > Thanks,
> > > > > > > > > > > Danny
> > > > > > > > > > >
> > > > > > > > > > > On Tue, Jan 9, 2024 at 9:31 AM Feng Jin 
> > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > +1 (non-binding)
> > > > > > > > > > > >
> > > > > > > > > > > > Best,
> > > > > > > > > > > > Feng Jin
> > > > > > > > > > > >
> > > > > > > > > > > > On Tue, Jan 9, 2024 at 5:29 PM Yuxin Tan <
> > > ta...@gmail.com>
> > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > +1 (non-binding)
> > > > > > > > > > > > >
> > > > > > > > > > > > > Best,
> > > > > > > > > > > > > Yuxin
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > Márton Balassi  于2024年1月9日周二 17:25写道:
> > > > > > > > > > > > >
> > > > > > > > > > > > > > +1 (binding)
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Tue, Jan 9, 2024 at 10:15 AM Leonard Xu <
> > > > > > xb...@gmail.com>
> > > > > > > > > > > wrote:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > +1(binding)
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > > Leonard
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 2024年1月9日 下午5:08,Yangze Guo 
> > > 写道:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > +1 (non-binding)
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > > > Yangze Guo
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > On Tue, Jan 9, 2024 at 5:06 PM Robert Metzger <
> > > > > > > > > > > rmetz...@apache.org
> > > > > > > > > > > > >
> > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > >> +1 (binding)
> > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > >> On Tue, Jan 9, 2024 at 9:54 AM Guowei Ma <
> > > > > > > gu...@gmail.com
> > > > > > > > >
> > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > >>> +1 (binding)
> > > > > > > > > > > > > > > >>> Best,
> > > > > > > > > > > > > > > >>> Guowei
> > > > > > > > > > > > > > > >>>
> > > > > > > > > > > > > > > >>>
> > > > > > > > > > > > > > > >>> On Tue, Jan 9, 2024 at 4:49 PM Rui Fan <
> > > > > > > 19...@gmail.com>
> > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > >>>
> > > > > > > > > > > > > > >  +1 (non-binding)
> > > > > > > > > > > > > > > 
> > > > > > > > > > > > > > >  Best,
> > > > > > > > > > > > > > >  Rui
> > > > > > > > > > > > > > > 
> > > > > > > > > > > > > > >  On Tue, Jan 9, 2024 at 4:41 PM Hang Ruan <
> > > > > > > > > > > > ruanhang1...@gmail.com>
> > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > 
> > > > > > > > > > > > > > > > +1 (non-binding)
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > > > Hang
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > gongzhongqiang 
> > > 于2024年1月9日周二
> > > > > > > > > > > > 16:25写道:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >> +1 non-binding
> > > > > > > > > > > > > > > >>
> > > 

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

2024-01-10 Thread Zakelly Lan
Hi everyone,

It seems we still don't have a consensus on the rules for boolean type
options. Let me recap the alternatives we have:

Option 1: Use enumeration options instead if possible. But this may cause
some name collisions or confusion as we discussed and we should unify the
statement everywhere.
Option 2: Use boolean options and add 'enabled' as the suffix.
Option 3: Use boolean options and ONLY add 'enabled' when there are more
detailed configurations under the same prefix, to prevent one name from
serving as a prefix to another.

I am inclined to Option 3, since it is more in line with current practice
and friendly for existing users. Also It reduces the length of
configuration names as much as possible.

Looking forward to your opinions! Thanks!


Best,
Zakelly

On Wed, Jan 10, 2024 at 3:30 PM Zakelly Lan  wrote:

> Hi Hangxiang,
>
> Thanks for your suggestions!
>
> 1. Could execution.recovery also contain some other behaviors about
>> recovery ? e.g. restart-strategy.
>
>
> That's a very good point. I realize that the word 'recovery' means way too
> many things. So I suggest picking a more specific word here, how about
> 'execution.state-recovery.*' ? Checkpointing and state recovery are
> corresponding terms and won't make ambiguity.
>
> 2. Could we also remove some legacy configuration value ? e.g. LEGACY Mode
>> for execution.savepoint-restore-mode/execution.recovery.claim-mode.
>
>
> I think we could create another FLIP for the deprecation of LEGACY mode.
>
>
>> 3. Could the local checkpoint be cleaned
>> if execution.checkpointing.local-copy.enabled is true and
>> execution.recovery.from-local is false ? I found it's also an issue if
>> current local-recovery from enabled to disabled. Maybe another ticket is
>> needed.
>
>
> IIUC, there is no clear ownership of the local copy files from the
> previous job and it's better to define one. This needs more discussion so
> we could create another thread for this. WDYT?
>
>
> Best,
> Zakelly
>
> On Tue, Jan 9, 2024 at 11:23 AM Hangxiang Yu  wrote:
>
>> Hi, Zakelly.
>> Thanks for driving this. Overall LGTM as we discussed offline.
>>
>> Some comments/suggestions just came to mind:
>> 1. Could execution.recovery also contain some other behaviors about
>> recovery ? e.g. restart-strategy.
>> 2. Could we also remove some legacy configuration value ? e.g. LEGACY Mode
>> for execution.savepoint-restore-mode/execution.recovery.claim-mode.
>> 3. Could the local checkpoint be cleaned
>> if execution.checkpointing.local-copy.enabled is true and
>> execution.recovery.from-local is false ? I found it's also an issue if
>> current local-recovery from enabled to disabled. Maybe another ticket is
>> needed.
>> 4. +1 for enabling execution.checkpointing.incremental by default which is
>> basically default configuration in our production environment.
>>
>>
>> On Mon, Jan 8, 2024 at 6:06 PM Zakelly Lan  wrote:
>>
>> > Hi Yun,
>> >
>> > Thanks for your comments!
>> >
>> >  1.  We shall not describe the configuration with its implementation for
>> > > 'execution.checkpointing.local-copy.*' options, for hashmap
>> > state-backend,
>> > > it would write two streams and for Rocksdb state-backend, it would use
>> > > hard-link for backup. Thus, I think
>> > > 'execution.checkpointing.local-backup.*' looks better.
>> >
>> > I agreed that we'd better name the option in user's perspective instead
>> of
>> > the implementation, thus I name it as a copy of the checkpoint in the
>> > local disk, regardless of the way of generating it. The word 'backup' is
>> > also suitable for this case, so I agree to change to
>> > 'execution.checkpointing.local-backup.*' if no one objects.
>> >
>> >  2.  What does the 'execution.checkpointing.data-inline-threshold'
>> mean? It
>> > > seems not so easy to understand.
>> >
>> > The 'execution.checkpointing.data-inline-threshold' (original one as
>> > 'state.storage.fs.memory-threshold') stands for the size threshold below
>> > which state chunks will store inline with the metadata, thus I call it
>> > 'data-inline-threshold'.
>> >
>> >
>> > Best,
>> > Zakelly
>> >
>> > On Mon, Jan 8, 2024 at 10:09 AM Yun Tang  wrote:
>> >
>> > > Hi Zakelly,
>> > >
>> > > Thanks for driving this topic. I have two concerns here:
>> > >
>> > >   1.  We shall not describe the configuration with its implementation
>> for
>> > > 'execution.checkpointing.local-copy.*' options, for hashmap
>> > state-backend,
>> > > it would write two streams and for Rocksdb state-backend, it would use
>> > > hard-link for backup. Thus, I think
>> > > 'execution.checkpointing.local-backup.*' looks better.
>> > >   2.  What does the 'execution.checkpointing.data-inline-threshold'
>> mean?
>> > > It seems not so easy to understand.
>> > >
>> > > Best
>> > > Yun Tang
>> > > 
>> > > From: Piotr Nowojski 
>> > > Sent: Thursday, January 4, 2024 22:37
>> > > To: dev@flink.apache.org 
>> > > Subject: Re: [DISCUSS] FLIP-406: Reorganize State & Checkpointing &

Re: Re: [VOTE] Accept Flink CDC into Apache Flink

2024-01-10 Thread Martijn Visser
+1 (binding)

On Wed, Jan 10, 2024 at 4:43 AM Xingbo Huang  wrote:
>
> +1 (binding)
>
> Best,
> Xingbo
>
> Dian Fu  于2024年1月10日周三 11:35写道:
>
> > +1 (binding)
> >
> > Regards,
> > Dian
> >
> > On Wed, Jan 10, 2024 at 5:09 AM Sharath  wrote:
> > >
> > > +1 (non-binding)
> > >
> > > Best,
> > > Sharath
> > >
> > > On Tue, Jan 9, 2024 at 1:02 PM Venkata Sanath Muppalla <
> > sanath...@gmail.com>
> > > wrote:
> > >
> > > > +1 (non-binding)
> > > >
> > > > Thanks,
> > > > Sanath
> > > >
> > > > On Tue, Jan 9, 2024 at 11:16 AM Peter Huang <
> > huangzhenqiu0...@gmail.com>
> > > > wrote:
> > > >
> > > > > +1 (non-binding)
> > > > >
> > > > >
> > > > > Best Regards
> > > > > Peter Huang
> > > > >
> > > > >
> > > > > On Tue, Jan 9, 2024 at 5:26 AM Jane Chan 
> > wrote:
> > > > >
> > > > > > +1 (non-binding)
> > > > > >
> > > > > > Best,
> > > > > > Jane
> > > > > >
> > > > > > On Tue, Jan 9, 2024 at 8:41 PM Lijie Wang <
> > wangdachui9...@gmail.com>
> > > > > > wrote:
> > > > > >
> > > > > > > +1 (non-binding)
> > > > > > >
> > > > > > > Best,
> > > > > > > Lijie
> > > > > > >
> > > > > > > Jiabao Sun  于2024年1月9日周二
> > 19:28写道:
> > > > > > >
> > > > > > > > +1 (non-binding)
> > > > > > > >
> > > > > > > > Best,
> > > > > > > > Jiabao
> > > > > > > >
> > > > > > > >
> > > > > > > > On 2024/01/09 09:58:04 xiangyu feng wrote:
> > > > > > > > > +1 (non-binding)
> > > > > > > > >
> > > > > > > > > Regards,
> > > > > > > > > Xiangyu Feng
> > > > > > > > >
> > > > > > > > > Danny Cranmer  于2024年1月9日周二 17:50写道:
> > > > > > > > >
> > > > > > > > > > +1 (binding)
> > > > > > > > > >
> > > > > > > > > > Thanks,
> > > > > > > > > > Danny
> > > > > > > > > >
> > > > > > > > > > On Tue, Jan 9, 2024 at 9:31 AM Feng Jin 
> > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > +1 (non-binding)
> > > > > > > > > > >
> > > > > > > > > > > Best,
> > > > > > > > > > > Feng Jin
> > > > > > > > > > >
> > > > > > > > > > > On Tue, Jan 9, 2024 at 5:29 PM Yuxin Tan <
> > ta...@gmail.com>
> > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > +1 (non-binding)
> > > > > > > > > > > >
> > > > > > > > > > > > Best,
> > > > > > > > > > > > Yuxin
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > Márton Balassi  于2024年1月9日周二 17:25写道:
> > > > > > > > > > > >
> > > > > > > > > > > > > +1 (binding)
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Tue, Jan 9, 2024 at 10:15 AM Leonard Xu <
> > > > > xb...@gmail.com>
> > > > > > > > > > wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > > +1(binding)
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > Leonard
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 2024年1月9日 下午5:08,Yangze Guo 
> > 写道:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > +1 (non-binding)
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > > Yangze Guo
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > On Tue, Jan 9, 2024 at 5:06 PM Robert Metzger <
> > > > > > > > > > rmetz...@apache.org
> > > > > > > > > > > >
> > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > >> +1 (binding)
> > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > >> On Tue, Jan 9, 2024 at 9:54 AM Guowei Ma <
> > > > > > gu...@gmail.com
> > > > > > > >
> > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > >>> +1 (binding)
> > > > > > > > > > > > > > >>> Best,
> > > > > > > > > > > > > > >>> Guowei
> > > > > > > > > > > > > > >>>
> > > > > > > > > > > > > > >>>
> > > > > > > > > > > > > > >>> On Tue, Jan 9, 2024 at 4:49 PM Rui Fan <
> > > > > > 19...@gmail.com>
> > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > >>>
> > > > > > > > > > > > > >  +1 (non-binding)
> > > > > > > > > > > > > > 
> > > > > > > > > > > > > >  Best,
> > > > > > > > > > > > > >  Rui
> > > > > > > > > > > > > > 
> > > > > > > > > > > > > >  On Tue, Jan 9, 2024 at 4:41 PM Hang Ruan <
> > > > > > > > > > > ruanhang1...@gmail.com>
> > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > 
> > > > > > > > > > > > > > > +1 (non-binding)
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Best,
> > > > > > > > > > > > > > > Hang
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > gongzhongqiang 
> > 于2024年1月9日周二
> > > > > > > > > > > 16:25写道:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >> +1 non-binding
> > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > >> Best,
> > > > > > > > > > > > > > >> Zhongqiang
> > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > >> Leonard Xu  于2024年1月9日周二
> > > > > 15:05写道:
> > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > >>> Hello all,
> > > > > > > > > > > > > > >>>
> > > > > > > > > > > > > > >>> This is the official vote whether to
> > accept 

Re: Re: [DISCUSS] FLIP-331: Support EndOfStreamWindows and isOutputOnEOF operator attribute to optimize task deployment

2024-01-10 Thread Xuannan Su
Hi all,

After several rounds of offline discussions with Xingtong and Jinhao,
we have decided to narrow the scope of the FLIP. It will now focus on
introducing OperatorAttributes that indicate whether an operator emits
records only after inputs have ended. We will also use the attribute
to optimize task scheduling for better resource utilization. Setting
the backlog status and optimizing the operator implementation during
the backlog will be deferred to future work.

In addition to the change above, we also make the following changes to
the FLIP to address the problems mentioned by Dong:
- Public interfaces are updated to reuse the GlobalWindows.
- Instead of making all outputs of the upstream operators of the
"isOutputOnlyAfterEndOfStream=true" operator blocking, we only make
the output of the operator with "isOutputOnlyAfterEndOfStream=true"
blocking. This can prevent the second problem Dong mentioned. In the
future, we may introduce an extra OperatorAttributes to indicate if an
operator has any side output.

I would greatly appreciate any comment or feedback you may have on the
updated FLIP.

Best regards,
Xuannan

On Tue, Sep 26, 2023 at 11:24 AM Dong Lin  wrote:
>
> Hi all,
>
> Thanks for the review!
>
> Becket and I discussed this FLIP offline and we agreed on several things
> that need to be improved with this FLIP. I will summarize our discussion
> with the problems and TODOs. We will update the FLIP and let you know once
> the FLIP is ready for review again.
>
> 1) Investigate whether it is possible to update the existing GlobalWindows
> in a backward-compatible way and re-use it for the same purpose
> as EndOfStreamWindows, without introducing EndOfStreamWindows as a new
> class.
>
> Note that GlobalWindows#getDefaultTrigger returns a NeverTrigger instance
> which will not trigger window's computation even on end-of-inputs. We will
> need to investigate its existing usage and see if we can re-use it in a
> backward-compatible way.
>
> 2) Let JM know whether any operator in the upstream of the operator with
> "isOutputOnEOF=true" will emit output via any side channel. The FLIP should
> update the execution mode of those operators *only if* all outputs from
> those operators are emitted only at the end of input.
>
> More specifically, the upstream operator might involve a user-defined
> operator that might emit output directly to an external service, where the
> emission operation is not explicitly expressed as an operator's output edge
> and thus not visible to JM. Similarly, it is also possible for the
> user-defined operator to register a timer
> via InternalTimerService#registerEventTimeTimer and emit output to an
> external service inside Triggerable#onEventTime. There is a chance that
> users still need related logic to output data in real-time, even if the
> downstream operators have isOutputOnEOF=true.
>
> One possible solution to address this problem is to add an extra
> OperatorAttribute to specify whether this operator might output records in
> such a way that does not go through operator's output (e.g. side output).
> Then the JM can safely enable the runtime optimization currently described
> in the FLIP when there is no such operator.
>
> 3) Create a follow-up FLIP that allows users to specify whether a source
> with Boundedness=bounded should have isProcessingBacklog=true.
>
> This capability would effectively introduce a 3rd strategy to set backlog
> status (in addition to FLIP-309 and FLIP-328). It might be useful to note
> that, even though the data in bounded sources are backlog data in most
> practical use-cases, it is not necessarily true. For example, users might
> want to start a Flink job to consume real-time data from a Kafka topic and
> specify that the job stops after 24 hours, which means the source is
> technically bounded while the data is fresh/real-time.
>
> This capability is more generic and can cover more use-case than
> EndOfStreamWindows. On the other hand, EndOfStreamWindows will still be
> useful in cases where users already need to specify this window assigner in
> a DataStream program, without bothering users to decide whether it is safe
> to treat data in a bounded source as backlog data.
>
>
> Regards,
> Dong
>
>
>
>
>
>
> On Mon, Sep 18, 2023 at 2:56 PM Yuxin Tan  wrote:
>
> > Hi, Dong,
> > Thanks for your efforts.
> >
> > +1 to this proposal,
> > I believe this will improve the performance in some mixture circumstances
> > of bounded and unbounded workloads.
> >
> > Best,
> > Yuxin
> >
> >
> > Xintong Song  于2023年9月18日周一 10:56写道:
> >
> > > Thanks for addressing my comments, Dong.
> > >
> > > LGTM.
> > >
> > > Best,
> > >
> > > Xintong
> > >
> > >
> > >
> > > On Sat, Sep 16, 2023 at 3:34 PM Wencong Liu 
> > wrote:
> > >
> > > > Hi Dong & Jinhao,
> > > >
> > > > Thanks for your clarification! +1
> > > >
> > > > Best regards,
> > > > Wencong
> > > >
> > > >
> > > >
> > > >
> > > >
> > > >
> > > >
> > > >
> > > >
> > > >
> > > >
> > > >
> > > >
> > > >
> > 

Re: [FLIP] Forward a flip to introduce minibatch optimization for Join

2024-01-10 Thread Leonard Xu
Hey, shuai 

I’ve added wiki permission for you, looking forward your streaming join 
optimization.

Best,
Leonard

[FLIP] Forward a flip to introduce minibatch optimization for Join

2024-01-10 Thread shuai xu
Hi all,

Currently, when performing cascading connections in Flink, there is a pain 
point of record amplification as mentioned in discussion similar to 
https://lists.apache.org/thread/2021fmwhtotl0okmtyc5b7tndlp3khf9,
I have implemented the optimization of POC and it works. I wonder to forward a 
FLIP to explain my plan. But there is no permission to create files in the 
Flink Improvement Proposals [1] space. I may need PMC to help me add 
permissions: My Jira account is xu_shuai_ The email is xushuai...@gmail.com 
.Thanks!

[jira] [Created] (FLINK-34049) Refactor classes related to window TVF aggregation to prepare for non-aligned windows

2024-01-10 Thread xuyang (Jira)
xuyang created FLINK-34049:
--

 Summary: Refactor classes related to window TVF aggregation to 
prepare for non-aligned windows
 Key: FLINK-34049
 URL: https://issues.apache.org/jira/browse/FLINK-34049
 Project: Flink
  Issue Type: Sub-task
  Components: Table SQL / Runtime
Affects Versions: 1.19.0
Reporter: xuyang


Refactor classes related to window TVF aggregation such as 
AbstractWindowAggProcessor to prepare for the implementation of non-aligned 
windows like session window



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


[jira] [Created] (FLINK-34048) Support Session Window Aggregate in table runtime

2024-01-10 Thread xuyang (Jira)
xuyang created FLINK-34048:
--

 Summary: Support Session Window Aggregate in table runtime 
 Key: FLINK-34048
 URL: https://issues.apache.org/jira/browse/FLINK-34048
 Project: Flink
  Issue Type: Sub-task
  Components: Table SQL / Runtime
Affects Versions: 1.19.0
Reporter: xuyang


We not only need FLINK-24024  to support session window agg in planner, but 
also need to support it in runtime.



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


[jira] [Created] (FLINK-34047) Inject GitHub Actions/Azure Pipelines env variables in uploading_watchdog.sh

2024-01-10 Thread Matthias Pohl (Jira)
Matthias Pohl created FLINK-34047:
-

 Summary: Inject GitHub Actions/Azure Pipelines env variables in 
uploading_watchdog.sh
 Key: FLINK-34047
 URL: https://issues.apache.org/jira/browse/FLINK-34047
 Project: Flink
  Issue Type: Sub-task
  Components: Build System / CI
Affects Versions: 1.18.0, 1.19.0
Reporter: Matthias Pohl


The workflow that's triggered by 
{{tools/azure-pipelines/uploading_watchdog.sh}} relies on CI specific 
environment variables. We should make the two different CI backends explicit in 
this script to improve the code readability.



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