[jira] [Created] (FLINK-35419) scan.bounded.latest-offset makes queries never finish if the latest message is a EndTxn Kafka marker

2024-05-22 Thread Fabian Paul (Jira)
Fabian Paul created FLINK-35419: --- Summary: scan.bounded.latest-offset makes queries never finish if the latest message is a EndTxn Kafka marker Key: FLINK-35419 URL: https://issues.apache.org/jira/browse/FLINK

Re: [jira] [Created] (FLINK-30238) Unified Sink committer does not clean up state on final savepoint

2022-12-01 Thread Fabian Paul
Yes, the StreamingFileSink is not affected. Best, Fabian

Re: [jira] [Created] (FLINK-30238) Unified Sink committer does not clean up state on final savepoint

2022-11-29 Thread Fabian Paul
Hi folks, I did some initial investigation, and the problem seems twofold. If no post-commit topology is used, we do not run into a problem where we could lose data but since we do not clean up the state correctly, we will hit this [1] when trying to stop the pipeline with a savepoint after we

[jira] [Created] (FLINK-30238) Unified Sink committer does not clean up state on final savepoint

2022-11-29 Thread Fabian Paul (Jira)
Fabian Paul created FLINK-30238: --- Summary: Unified Sink committer does not clean up state on final savepoint Key: FLINK-30238 URL: https://issues.apache.org/jira/browse/FLINK-30238 Project: Flink

[ANNOUNCE] Apache Flink 1.15.3 released

2022-11-25 Thread Fabian Paul
The Apache Flink community is very happy to announce the release of Apache Flink 1.15.3, which is the third bugfix release for the Apache Flink 1.15 series. Apache Flink® is an open-source stream processing framework for distributed, high-performing, always-available, and accurate data streaming

[RESULT][VOTE] Release 1.15.3, release candidate #1

2022-11-24 Thread Fabian Paul
Hi all, @Yun sorry I forgot to post the result of the vote before continuing. I only created the tag in the repository after I received the third binding vote. I'm happy to announce that we have unanimously approved this release. There are 6 approving votes, 3 of which are binding: * Danny *

Re: [VOTE] Release 1.15.3, release candidate #1

2022-11-21 Thread Fabian Paul
ifacts. > > > - Verified that the source archives do not contain any binary file. > > > - Verified that all POM files point to the same version. > > > - Checked that the README.md file does not have anything unexpected. > > > - Checked that the source code tag looks

Re: [DISCUSS] Release Flink 1.15.3

2022-11-16 Thread Fabian Paul
r > wrote: > > > Hi Fabian, > > > > I'll try to have a look tomorrow. > > > > Cheers, Martijn > > > > On Tue, Nov 15, 2022 at 6:44 PM Fabian Paul wrote: > > > > > Hi all, > > > > > > The release vote for 1.15.3-rc1 is o

Re: [DISCUSS] Release Flink 1.15.3

2022-11-15 Thread Fabian Paul
wrote: > > Hi Fabian, > > I've added 1.15.4 as a new release version. > > Thanks, Martijn > > On Thu, Nov 10, 2022 at 5:18 PM Fabian Paul > wrote: >> >> I conclude that the community has accepted another release, and I will open >> the voting thread sh

Re: [VOTE] Release 1.15.3, release candidate #1

2022-11-15 Thread Fabian Paul
[6] https://github.com/apache/flink-web/pull/581 On Mon, Nov 14, 2022 at 8:45 AM Fabian Paul wrote: > > Hi everyone, > > I am still looking for volunteers to validate the release. I'll extend > the voting period by another 48hours, please try to give it some time. > > Best, >

Re: [VOTE] Release 1.15.3, release candidate #1

2022-11-13 Thread Fabian Paul
Hi everyone, I am still looking for volunteers to validate the release. I'll extend the voting period by another 48hours, please try to give it some time. Best, Fabian On Thu, Nov 10, 2022 at 5:18 PM Fabian Paul wrote: > > Hi everyone, Please review and vote on the release candid

[VOTE] Release 1.15.3, release candidate #1

2022-11-10 Thread Fabian Paul
Hi everyone, Please review and vote on the release candidate #1 for the version 1.15.3, as follows: [ ] +1, Approve the release [ ] -1, Do not approve the release (please provide specific comments) The complete staging area is available for your review, which includes: - JIRA release notes [1], -

Re: [DISCUSS] Release Flink 1.15.3

2022-11-10 Thread Fabian Paul
-config/FLINK/versions On Wed, Nov 2, 2022 at 2:07 PM Fabian Paul wrote: > Thanks for all the replies. @xintong I'll definitely come back to your > offer when facing steps that require PMC rights for the release. > > I checked the JIRA and found four blocking/critical issues affe

Re: [DISCUSS] Release Flink 1.15.3

2022-11-02 Thread Fabian Paul
ct 25, 2022 at 11:44 AM Xintong Song > > > > wrote: > > > > > > > >> Thanks for bringing this up, Fabian. > > > >> > > > >> +1 for creating a 1.15.3 release. I've also seen users requiring > this > > > >&g

[DISCUSS] Release Flink 1.15.3

2022-10-25 Thread Fabian Paul
Hi all, I want to start the discussion of creating a new 1.15 patch release (1.15.3). The last 1.15 release is almost two months old, and since then, ~60 tickets have been closed, targeting 1.15.3. It includes critical changes to the sink architecture, including: - Reverting the sink metric

[jira] [Created] (FLINK-29583) Ensure correct subtaskId and checkpointId is set during committer state migration

2022-10-11 Thread Fabian Paul (Jira)
Fabian Paul created FLINK-29583: --- Summary: Ensure correct subtaskId and checkpointId is set during committer state migration Key: FLINK-29583 URL: https://issues.apache.org/jira/browse/FLINK-29583

[jira] [Created] (FLINK-29512) Align SubtaskCommittableManager checkpointId with CheckpointCommittableManagerImpl checkpointId during recovery

2022-10-05 Thread Fabian Paul (Jira)
Fabian Paul created FLINK-29512: --- Summary: Align SubtaskCommittableManager checkpointId with CheckpointCommittableManagerImpl checkpointId during recovery Key: FLINK-29512 URL: https://issues.apache.org/jira/browse

[jira] [Created] (FLINK-29509) Set correct subtaskId during recovery of committables

2022-10-05 Thread Fabian Paul (Jira)
Fabian Paul created FLINK-29509: --- Summary: Set correct subtaskId during recovery of committables Key: FLINK-29509 URL: https://issues.apache.org/jira/browse/FLINK-29509 Project: Flink Issue

[jira] [Created] (FLINK-27493) Forward all numeric Kafka metrics to Flink's metrics system

2022-05-05 Thread Fabian Paul (Jira)
Fabian Paul created FLINK-27493: --- Summary: Forward all numeric Kafka metrics to Flink's metrics system Key: FLINK-27493 URL: https://issues.apache.org/jira/browse/FLINK-27493 Project: Flink

[jira] [Created] (FLINK-27486) Reduce ArchUnit violations in connector base module

2022-05-04 Thread Fabian Paul (Jira)
Fabian Paul created FLINK-27486: --- Summary: Reduce ArchUnit violations in connector base module Key: FLINK-27486 URL: https://issues.apache.org/jira/browse/FLINK-27486 Project: Flink Issue Type

[jira] [Created] (FLINK-27484) Reduce ArchUnit violations in the project

2022-05-04 Thread Fabian Paul (Jira)
Fabian Paul created FLINK-27484: --- Summary: Reduce ArchUnit violations in the project Key: FLINK-27484 URL: https://issues.apache.org/jira/browse/FLINK-27484 Project: Flink Issue Type

[jira] [Created] (FLINK-27480) KafkaSources sharing the groupId might lead to InstanceAlreadyExistException warning

2022-05-03 Thread Fabian Paul (Jira)
Fabian Paul created FLINK-27480: --- Summary: KafkaSources sharing the groupId might lead to InstanceAlreadyExistException warning Key: FLINK-27480 URL: https://issues.apache.org/jira/browse/FLINK-27480

[jira] [Created] (FLINK-26701) Relocation of connector-base might break user jars due to changed imports

2022-03-17 Thread Fabian Paul (Jira)
Fabian Paul created FLINK-26701: --- Summary: Relocation of connector-base might break user jars due to changed imports Key: FLINK-26701 URL: https://issues.apache.org/jira/browse/FLINK-26701 Project

[jira] [Created] (FLINK-26633) Elasticsearch connector does not report recordsSend metric

2022-03-14 Thread Fabian Paul (Jira)
Fabian Paul created FLINK-26633: --- Summary: Elasticsearch connector does not report recordsSend metric Key: FLINK-26633 URL: https://issues.apache.org/jira/browse/FLINK-26633 Project: Flink

[jira] [Created] (FLINK-26613) Stateful unified Sink V2 upgrades only work when operator uids are given

2022-03-11 Thread Fabian Paul (Jira)
Fabian Paul created FLINK-26613: --- Summary: Stateful unified Sink V2 upgrades only work when operator uids are given Key: FLINK-26613 URL: https://issues.apache.org/jira/browse/FLINK-26613 Project

[jira] [Created] (FLINK-26516) Sink V2 is not state compatible with Sink V1

2022-03-07 Thread Fabian Paul (Jira)
Fabian Paul created FLINK-26516: --- Summary: Sink V2 is not state compatible with Sink V1 Key: FLINK-26516 URL: https://issues.apache.org/jira/browse/FLINK-26516 Project: Flink Issue Type: Bug

[jira] [Created] (FLINK-26445) Unified Sink cannot restore committer state

2022-03-01 Thread Fabian Paul (Jira)
Fabian Paul created FLINK-26445: --- Summary: Unified Sink cannot restore committer state Key: FLINK-26445 URL: https://issues.apache.org/jira/browse/FLINK-26445 Project: Flink Issue Type: Bug

[jira] [Created] (FLINK-26416) Release Testing:

2022-02-28 Thread Fabian Paul (Jira)
Fabian Paul created FLINK-26416: --- Summary: Release Testing: Key: FLINK-26416 URL: https://issues.apache.org/jira/browse/FLINK-26416 Project: Flink Issue Type: Improvement Components

[jira] [Created] (FLINK-26358) Operator maxParallelism is lost during chaining

2022-02-24 Thread Fabian Paul (Jira)
Fabian Paul created FLINK-26358: --- Summary: Operator maxParallelism is lost during chaining Key: FLINK-26358 URL: https://issues.apache.org/jira/browse/FLINK-26358 Project: Flink Issue Type

Re: [ANNOUNCE] Flink 1.15 Feature Freeze

2022-02-23 Thread Fabian Paul
Hi all, I would like to merge the following PR [1]. It has been approved before the feature freeze but no one had time to merge it, unfortunately. The feature is very contained and only adds a simple capability to the Elastic connector when used with Flink SQL. If there are no concerns until end

Re: [DISCUSS] Looking for maintainers for Cassandra connector or decide to remove connector

2022-02-22 Thread Fabian Paul
Hi Marco, Great to hear that you put some thought into the topic. Judging from the past we already tried once to support multiple external versions within one connector (ElasticSearch) and it complicates things a lot. So if it makes your development easier to create a different module that should

Re: [DISCUSS] Release Flink 1.14.4

2022-02-22 Thread Fabian Paul
EDIT: Wrong link before https://issues.apache.org/jira/browse/FLINK-26304 On Tue, Feb 22, 2022 at 4:55 PM Fabian Paul wrote: > > Hi Konstantin, > > Thanks for all the efforts driving the release. From my side, > FLINK-26018 can also be seen as some kind of new feature tha

Re: [DISCUSS] Release Flink 1.14.4

2022-02-22 Thread Fabian Paul
Hi Konstantin, Thanks for all the efforts driving the release. From my side, FLINK-26018 can also be seen as some kind of new feature that was planned but never implemented. Of course, it would be great to have it because it currently blocks the migration from the FlinkKafkaConsumer to the

[jira] [Created] (FLINK-26304) GlobalCommitter can receive failed committables

2022-02-22 Thread Fabian Paul (Jira)
Fabian Paul created FLINK-26304: --- Summary: GlobalCommitter can receive failed committables Key: FLINK-26304 URL: https://issues.apache.org/jira/browse/FLINK-26304 Project: Flink Issue Type

[jira] [Created] (FLINK-26254) KafkaSink might violate order of sequence numbers and risk exactly-once processing

2022-02-18 Thread Fabian Paul (Jira)
Fabian Paul created FLINK-26254: --- Summary: KafkaSink might violate order of sequence numbers and risk exactly-once processing Key: FLINK-26254 URL: https://issues.apache.org/jira/browse/FLINK-26254

Re: [ANNOUNCE] New Flink PMC members: Igal Shilman, Konstantin Knauf and Yun Gao

2022-02-16 Thread Fabian Paul
Congrats to all three of you, well deserved. Best, Fabian On Wed, Feb 16, 2022 at 2:23 PM Robert Metzger wrote: > > Hi all, > > I would like to formally announce a few new Flink PMC members on the dev@ > list. The PMC has not done a good job of always announcing new PMC members > (and

[jira] [Created] (FLINK-26119) AsyncSinkWriterStateSerializer needs to be PublicEvolving

2022-02-14 Thread Fabian Paul (Jira)
Fabian Paul created FLINK-26119: --- Summary: AsyncSinkWriterStateSerializer needs to be PublicEvolving Key: FLINK-26119 URL: https://issues.apache.org/jira/browse/FLINK-26119 Project: Flink

[jira] [Created] (FLINK-26118) AsyncSinks do not support downscaling with state

2022-02-14 Thread Fabian Paul (Jira)
Fabian Paul created FLINK-26118: --- Summary: AsyncSinks do not support downscaling with state Key: FLINK-26118 URL: https://issues.apache.org/jira/browse/FLINK-26118 Project: Flink Issue Type

Re: [VOTE] Remove Twitter connector

2022-02-03 Thread Fabian Paul
This connector is really a relict of the past. +1 (binding) Best, Fabian On Mon, Jan 31, 2022 at 11:47 AM Martijn Visser wrote: > > Hi everyone, > > I would like to open up a vote to remove the Twitter connector in Flink > 1.15. This was brought up previously for a discussion [1]. > > The vote

Re: [VOTE] Deprecate NiFi connector

2022-02-03 Thread Fabian Paul
Thanks for driving the deprecation efforts. +1 (binding) Best, Fabian On Mon, Jan 31, 2022 at 11:47 AM Martijn Visser wrote: > > Hi everyone, > > I would like to open up a vote to deprecate NiFi in Flink 1.15 and remove > it in the next version. I've previously mentioned that we were looking

[jira] [Created] (FLINK-25921) Support different input parallelism for preCommit topology

2022-02-02 Thread Fabian Paul (Jira)
Fabian Paul created FLINK-25921: --- Summary: Support different input parallelism for preCommit topology Key: FLINK-25921 URL: https://issues.apache.org/jira/browse/FLINK-25921 Project: Flink

[jira] [Created] (FLINK-25920) Allow receiving updates of CommittableSummary

2022-02-02 Thread Fabian Paul (Jira)
Fabian Paul created FLINK-25920: --- Summary: Allow receiving updates of CommittableSummary Key: FLINK-25920 URL: https://issues.apache.org/jira/browse/FLINK-25920 Project: Flink Issue Type: Sub

[jira] [Created] (FLINK-25919) Sink V2 improvements and followups

2022-02-02 Thread Fabian Paul (Jira)
Fabian Paul created FLINK-25919: --- Summary: Sink V2 improvements and followups Key: FLINK-25919 URL: https://issues.apache.org/jira/browse/FLINK-25919 Project: Flink Issue Type: Improvement

[jira] [Created] (FLINK-25857) Add committer metrics to track the status of committables

2022-01-27 Thread Fabian Paul (Jira)
Fabian Paul created FLINK-25857: --- Summary: Add committer metrics to track the status of committables Key: FLINK-25857 URL: https://issues.apache.org/jira/browse/FLINK-25857 Project: Flink

[jira] [Created] (FLINK-25726) Implement GlobalCommitter as custom post commit topology

2022-01-20 Thread Fabian Paul (Jira)
Fabian Paul created FLINK-25726: --- Summary: Implement GlobalCommitter as custom post commit topology Key: FLINK-25726 URL: https://issues.apache.org/jira/browse/FLINK-25726 Project: Flink Issue

Re: [DISCUSS] FLIP-208: Update KafkaSource to detect EOF based on de-serialized record

2022-01-12 Thread Fabian Paul
11, 2022 at 11:46 PM Fabian Paul wrote: > > > Hi Dong, > > > > I wouldn't change the org.apache.flink.api.connector.source.Source > > interface because it either breaks existing sinks or we introduce it > > as some kind of optional. I deem both options as not

Re: [DISCUSS] FLIP-208: Update KafkaSource to detect EOF based on de-serialized record

2022-01-11 Thread Fabian Paul
ll continue to be determined with the API > KafkaSourceBuilder::setBounded(..) and > KafkaSourceBuilder::setUnbounded(..). Does this answer your question? > > Thanks, > Dong > > > > > > > > > > On Mon, Jan 10, 2022 at 8:01 PM Fabian Paul wrote: > > > Hi

[jira] [Created] (FLINK-25608) Mark metrics as Public(Evolving)

2022-01-11 Thread Fabian Paul (Jira)
Fabian Paul created FLINK-25608: --- Summary: Mark metrics as Public(Evolving) Key: FLINK-25608 URL: https://issues.apache.org/jira/browse/FLINK-25608 Project: Flink Issue Type: Improvement

[jira] [Created] (FLINK-25591) Use FileSource for StreamExecutionEnvironment.readFiles

2022-01-10 Thread Fabian Paul (Jira)
Fabian Paul created FLINK-25591: --- Summary: Use FileSource for StreamExecutionEnvironment.readFiles Key: FLINK-25591 URL: https://issues.apache.org/jira/browse/FLINK-25591 Project: Flink Issue

Re: [DISCUSS] FLIP-208: Update KafkaSource to detect EOF based on de-serialized record

2022-01-10 Thread Fabian Paul
Hi Dong, Thank you for updating the FLIP and making it applicable for all sources. I am a bit unsure about the implementation part. I would propose to add a source mixin interface that implements `getRecordEvaluator` and sources that want to allow dynamically stopping implement that interface.

Re: [VOTE] Create a separate sub project for FLIP-188: flink-store

2022-01-10 Thread Fabian Paul
Hi all, I just wanted to give my two cents for the build system discussion. In general, I agree with David's opinion to start new projects with Gradle but during the development of the external connector repository, we found some difficulties that still need to be solved. I do not want to force

[jira] [Created] (FLINK-25578) Graduate Sink V1 interfaces to PublicEvolving and deprecate them

2022-01-07 Thread Fabian Paul (Jira)
Fabian Paul created FLINK-25578: --- Summary: Graduate Sink V1 interfaces to PublicEvolving and deprecate them Key: FLINK-25578 URL: https://issues.apache.org/jira/browse/FLINK-25578 Project: Flink

[jira] [Created] (FLINK-25575) Implement StreamGraph translation for Sink V2 interfaces

2022-01-07 Thread Fabian Paul (Jira)
Fabian Paul created FLINK-25575: --- Summary: Implement StreamGraph translation for Sink V2 interfaces Key: FLINK-25575 URL: https://issues.apache.org/jira/browse/FLINK-25575 Project: Flink Issue

[jira] [Created] (FLINK-25574) Update Async Sink to use decomposed interfaces

2022-01-07 Thread Fabian Paul (Jira)
Fabian Paul created FLINK-25574: --- Summary: Update Async Sink to use decomposed interfaces Key: FLINK-25574 URL: https://issues.apache.org/jira/browse/FLINK-25574 Project: Flink Issue Type: Sub

[jira] [Created] (FLINK-25573) Update Kafka Sink to use decomposed interfaces

2022-01-07 Thread Fabian Paul (Jira)
Fabian Paul created FLINK-25573: --- Summary: Update Kafka Sink to use decomposed interfaces Key: FLINK-25573 URL: https://issues.apache.org/jira/browse/FLINK-25573 Project: Flink Issue Type: Sub

[jira] [Created] (FLINK-25572) Update File Sink to use decomposed interfaces

2022-01-07 Thread Fabian Paul (Jira)
Fabian Paul created FLINK-25572: --- Summary: Update File Sink to use decomposed interfaces Key: FLINK-25572 URL: https://issues.apache.org/jira/browse/FLINK-25572 Project: Flink Issue Type: Sub

[jira] [Created] (FLINK-25571) Update Elasticsearch Sink to use decomposed interfaces

2022-01-07 Thread Fabian Paul (Jira)
Fabian Paul created FLINK-25571: --- Summary: Update Elasticsearch Sink to use decomposed interfaces Key: FLINK-25571 URL: https://issues.apache.org/jira/browse/FLINK-25571 Project: Flink Issue

[jira] [Created] (FLINK-25570) Introduce Sink V2 extension APIs

2022-01-07 Thread Fabian Paul (Jira)
Fabian Paul created FLINK-25570: --- Summary: Introduce Sink V2 extension APIs Key: FLINK-25570 URL: https://issues.apache.org/jira/browse/FLINK-25570 Project: Flink Issue Type: Sub-task

[jira] [Created] (FLINK-25569) Introduce decomposed Sink V2 interfaces

2022-01-07 Thread Fabian Paul (Jira)
Fabian Paul created FLINK-25569: --- Summary: Introduce decomposed Sink V2 interfaces Key: FLINK-25569 URL: https://issues.apache.org/jira/browse/FLINK-25569 Project: Flink Issue Type: Sub-task

[RESULT][VOTE] FLIP-191: Extend unified Sink interface to support small file compaction

2022-01-07 Thread Fabian Paul
I am happy to announce that FLIP-191 [1] has been accepted by this vote [2]. There are 5 approving votes, 3 of which are binding: * Martijn Visser (non-binding) * Yun Gao (binding) * Arvid Heise (binding) * Guowei Ma (binding) * Jing Ge (non-binding) There are no disapproving votes. Thanks

[jira] [Created] (FLINK-25555) FLIP-191: Extend unified Sink interface to support small file compaction

2022-01-06 Thread Fabian Paul (Jira)
Fabian Paul created FLINK-2: --- Summary: FLIP-191: Extend unified Sink interface to support small file compaction Key: FLINK-2 URL: https://issues.apache.org/jira/browse/FLINK-2 Project

[VOTE] FLIP-191: Extend unified Sink interface to support small file compaction

2022-01-04 Thread Fabian Paul
Hi everyone, I'd like to start a vote on FLIP-191: Extend unified Sink interface to support small file compaction [1] that has been discussed in this thread [2]. The vote will be open for at least 72 hours unless there is an objection or not enough votes. Best, Fabian [1]

Re: Re: [DISCUSS] FLIP-191: Extend unified Sink interface to support small file compaction

2022-01-03 Thread Fabian Paul
ture > as long as we know how to solve it~ > > Best, > Yun > > > ------ > From:Arvid Heise > Send Time:2021 Dec. 16 (Thu.) 19:54 > To:dev > Cc:Yun Gao > Subject:Re: Re: [DISCUSS] FLIP-191: Extend uni

Re: [DISCUSS] GHA migration roadmap

2021-12-16 Thread Fabian Paul
Hi Nico, Thanks a lot for drafting the proposal. I really like the fully-fledged phasing model. All in all, I am +1 to move away from azure and can only second all the points you have mentioned. I only want to clarify one point. So far my understanding was that the GHA resources are managed on a

Re: Re: [DISCUSS] FLIP-191: Extend unified Sink interface to support small file compaction

2021-12-16 Thread Fabian Paul
> > On Thu, Dec 16, 2021 at 9:23 AM Fabian Paul wrote: > > > Hi Yun, > > > > Thanks for your fast feedback. Let me clarify your points. > > > > 1. We solve it by using StreamExchangeMode.BATCH before any exchange. > > That obviously doesn’t help with

Re: Re: [DISCUSS] FLIP-191: Extend unified Sink interface to support small file compaction

2021-12-16 Thread Fabian Paul
iles >> >> > >> > from multiple checkpoints, and rewriter(compact) them together once >> > >> > the >> >> > >> > total file size or number of files reach the threshold. >> >> > >> > Step-4: A single parallelism rewrite(compac

Re: [DISCUSS] FLIP-191: Extend unified Sink interface to support small file compaction

2021-12-13 Thread Fabian Paul
nd latency is very important. I think we need to separate the > >> > write and compact phase. For the write phase, there include the Step-1 > >> > and Step-2, we sink data as file and commit it pre checkpoint and > >> > regardless > >> > of w

Re: [DISCUSS] Releasing Flink 1.14.1

2021-12-09 Thread Fabian Paul
Actually I meant https://issues.apache.org/jira/browse/FLINK-25126 sorry for the confusion. On Thu, Dec 9, 2021 at 4:55 PM Fabian Paul wrote: > > Hi Martijn, > > I just opened the backport for > https://issues.apache.org/jira/browse/FLINK-25132. The changes are > already ap

Re: [DISCUSS] Releasing Flink 1.14.1

2021-12-09 Thread Fabian Paul
> I'd like to backport the fix of FLINK-19142 to 1.14.1. > >> > The backport is in progress. > >> > Will update it here when it is done. > >> > > >> > Thanks, > >> > Zhu > >> > > >> > Jingsong Li 于2021年12月

[jira] [Created] (FLINK-25222) Remove NetworkFailureProxy used for Kafka connector tests

2021-12-08 Thread Fabian Paul (Jira)
Fabian Paul created FLINK-25222: --- Summary: Remove NetworkFailureProxy used for Kafka connector tests Key: FLINK-25222 URL: https://issues.apache.org/jira/browse/FLINK-25222 Project: Flink

Re: [DISCUSS] FLIP-196: Source API stability guarantees

2021-12-07 Thread Fabian Paul
Hi all, Thanks Till for starting this discussion. It is great to see these facts written down since they definitely caused friction in the past because of different interpretations. Overall I agree with everything being said in this FLIP. I was just wondering whether we can put the label

[jira] [Created] (FLINK-25206) Add config option to disable configurations in the user program

2021-12-07 Thread Fabian Paul (Jira)
Fabian Paul created FLINK-25206: --- Summary: Add config option to disable configurations in the user program Key: FLINK-25206 URL: https://issues.apache.org/jira/browse/FLINK-25206 Project: Flink

Re: [DISCUSS] Releasing Flink 1.14.1

2021-12-03 Thread Fabian Paul
I just opened a PR for https://issues.apache.org/jira/browse/FLINK-25126 I'll expect to merge it sometime next week. Best, Fabian On Fri, Dec 3, 2021 at 10:49 AM Martijn Visser wrote: > > Hi all, > > Just a status update on the open blockers for 1.14.1: > *

Re: [ANNOUNCE] New Apache Flink Committer - Ingo Bürk

2021-12-02 Thread Fabian Paul
Thanks for always pushing Ingo. Congratulations! Best, Fabian On Thu, Dec 2, 2021 at 4:24 PM Till Rohrmann wrote: > > Hi everyone, > > On behalf of the PMC, I'm very happy to announce Ingo Bürk as a new Flink > committer. > > Ingo has started contributing to Flink since the beginning of this

Re: [ANNOUNCE] New Apache Flink Committer - Matthias Pohl

2021-12-02 Thread Fabian Paul
Congrats and well deserved. Best, Fabian On Thu, Dec 2, 2021 at 4:42 PM Ingo Bürk wrote: > > Congrats, Matthias! > > On Thu, Dec 2, 2021 at 4:28 PM Till Rohrmann wrote: > > > Hi everyone, > > > > On behalf of the PMC, I'm very happy to announce Matthias Pohl as a new > > Flink committer. > > >

[jira] [Created] (FLINK-25134) Unused RetryRule in KafkaConsumerTestBase swallows retries

2021-12-01 Thread Fabian Paul (Jira)
Fabian Paul created FLINK-25134: --- Summary: Unused RetryRule in KafkaConsumerTestBase swallows retries Key: FLINK-25134 URL: https://issues.apache.org/jira/browse/FLINK-25134 Project: Flink

Re: [DISCUSS] Releasing Flink 1.14.1

2021-11-24 Thread Fabian Paul
Hi Martijn, Thanks for bringing up this topic. I think it would be great to release a patch version of 1.14 before the end of the year. Currently, FLINK-24596 is in progress and I would block the release until it is merged because it unblocks

[jira] [Created] (FLINK-25041) E2E tar ball cache fails without error message if target directory not specified

2021-11-24 Thread Fabian Paul (Jira)
Fabian Paul created FLINK-25041: --- Summary: E2E tar ball cache fails without error message if target directory not specified Key: FLINK-25041 URL: https://issues.apache.org/jira/browse/FLINK-25041

Re: [DISCUSS] Definition of Done for Apache Flink

2021-11-16 Thread Fabian Paul
Hi all, Maybe I am the devil's advocate but I see the stability of master and the definition of done as disjunct properties. I think it is more a question of prioritization that test instabilities are treated as critical tickets and have to be addressed before continuing any other work. It will

Re: [ANNOUNCE] New Apache Flink Committer - Fabian Paul

2021-11-15 Thread Fabian Paul
Thanks for the warm welcome, I am looking forward to continuing working with you all. Best, Fabian

Re: [DISCUSS] Update Policy for old releases

2021-11-11 Thread Fabian Paul
Thanks for bringing up this topic Piotr. I also think we should try to decouple our release cycles from our support plans. Currently we are very limited by the approach because faster release cycles result in also faster deprecation of versions. Therefore I am also favoring version 2 where we

[jira] [Created] (FLINK-24858) TypeSerializer version mismatch during eagerly restore

2021-11-10 Thread Fabian Paul (Jira)
Fabian Paul created FLINK-24858: --- Summary: TypeSerializer version mismatch during eagerly restore Key: FLINK-24858 URL: https://issues.apache.org/jira/browse/FLINK-24858 Project: Flink Issue

Re: [NOTICE] Please keep flink-examples up to date

2021-11-08 Thread Fabian Paul
Hi Seth, Thanks for brining up this topic. Huge appreciations that you take this over initially and we should definitely take care as a community to what we show beginner users. We can also take the examples as show cases about things we have developed and are proud of. Best, Fabian

Re: [DISCUSS] FLIP-191: Extend unified Sink interface to support small file compaction

2021-11-08 Thread Fabian Paul
Hi all, Thanks for the lively discussions. I am really excited to see so many people participating in this thread. It also underlines the need that many people would like to see a solution soon. I have updated the FLIP and removed the parallelism configuration because it is unnecessary since

[jira] [Created] (FLINK-24773) KafkaCommitter should fail on unknown Exception

2021-11-04 Thread Fabian Paul (Jira)
Fabian Paul created FLINK-24773: --- Summary: KafkaCommitter should fail on unknown Exception Key: FLINK-24773 URL: https://issues.apache.org/jira/browse/FLINK-24773 Project: Flink Issue Type

[jira] [Created] (FLINK-24765) Upgrade Kafka dependency

2021-11-04 Thread Fabian Paul (Jira)
Fabian Paul created FLINK-24765: --- Summary: Upgrade Kafka dependency Key: FLINK-24765 URL: https://issues.apache.org/jira/browse/FLINK-24765 Project: Flink Issue Type: Improvement

Re: [DISCUSS] FLIP-191: Extend unified Sink interface to support small file compaction

2021-11-03 Thread Fabian Paul
Hi David and Till, Thanks for your great feedback. One definitely confusing point in the FLIP is who is doing the actual compaction. The compaction will not be done by the CommittableAggregator operator but the committers so it should also not affect the checkpointing duration or have a

[DISCUSS] FLIP-191: Extend unified Sink interface to support small file compaction

2021-11-02 Thread Fabian Paul
Hi all, More and more data lake sinks rely on columnar formats which benefit from few larger files than a lot of small files (read amplification). Our current FileSink cannot ensure a certain size when writing to an external filesystem which I call the small file compaction problem.

[jira] [Created] (FLINK-24647) ClusterUncaughtExceptionHandler does not log the exception

2021-10-26 Thread Fabian Paul (Jira)
Fabian Paul created FLINK-24647: --- Summary: ClusterUncaughtExceptionHandler does not log the exception Key: FLINK-24647 URL: https://issues.apache.org/jira/browse/FLINK-24647 Project: Flink

[jira] [Created] (FLINK-24612) Kafka test container creates a large amount of logs

2021-10-21 Thread Fabian Paul (Jira)
Fabian Paul created FLINK-24612: --- Summary: Kafka test container creates a large amount of logs Key: FLINK-24612 URL: https://issues.apache.org/jira/browse/FLINK-24612 Project: Flink Issue Type

[jira] [Created] (FLINK-24608) Sinks built with the unified sink framework do not receive timestamps when used in Table API

2021-10-21 Thread Fabian Paul (Jira)
Fabian Paul created FLINK-24608: --- Summary: Sinks built with the unified sink framework do not receive timestamps when used in Table API Key: FLINK-24608 URL: https://issues.apache.org/jira/browse/FLINK-24608

[jira] [Created] (FLINK-24530) GlobalCommitter might not commit all records on drain

2021-10-13 Thread Fabian Paul (Jira)
Fabian Paul created FLINK-24530: --- Summary: GlobalCommitter might not commit all records on drain Key: FLINK-24530 URL: https://issues.apache.org/jira/browse/FLINK-24530 Project: Flink Issue

[jira] [Created] (FLINK-24488) KafkaRecordSerializationSchemaBuilder does not forward timestamp

2021-10-08 Thread Fabian Paul (Jira)
Fabian Paul created FLINK-24488: --- Summary: KafkaRecordSerializationSchemaBuilder does not forward timestamp Key: FLINK-24488 URL: https://issues.apache.org/jira/browse/FLINK-24488 Project: Flink

Re: [Flink blogs]

2021-09-30 Thread Fabian Paul
Hi Etienne, Thanks for reaching out I think your list already looks very appealing. > * - metrics (https://github.com/apache/flink/pull/14510): it was > dealing with delimiters. I think it is a bit low level for a blog post ? > * I am also unsure whether this a good fit to present. I can only

[jira] [Created] (FLINK-24397) Reduce TableSchema usage in Table API connectors

2021-09-28 Thread Fabian Paul (Jira)
Fabian Paul created FLINK-24397: --- Summary: Reduce TableSchema usage in Table API connectors Key: FLINK-24397 URL: https://issues.apache.org/jira/browse/FLINK-24397 Project: Flink Issue Type

[jira] [Created] (FLINK-24382) RecordsOut metric for sinks is inaccurate

2021-09-27 Thread Fabian Paul (Jira)
Fabian Paul created FLINK-24382: --- Summary: RecordsOut metric for sinks is inaccurate Key: FLINK-24382 URL: https://issues.apache.org/jira/browse/FLINK-24382 Project: Flink Issue Type

[jira] [Created] (FLINK-24372) Deprecate Elasticsearch Sinkfunctions

2021-09-24 Thread Fabian Paul (Jira)
Fabian Paul created FLINK-24372: --- Summary: Deprecate Elasticsearch Sinkfunctions Key: FLINK-24372 URL: https://issues.apache.org/jira/browse/FLINK-24372 Project: Flink Issue Type: Sub-task

[jira] [Created] (FLINK-24371) Support SinkWriter preCommit without the need of a committer

2021-09-24 Thread Fabian Paul (Jira)
Fabian Paul created FLINK-24371: --- Summary: Support SinkWriter preCommit without the need of a committer Key: FLINK-24371 URL: https://issues.apache.org/jira/browse/FLINK-24371 Project: Flink

[jira] [Created] (FLINK-24368) Collect logs of Flink container in e2e source test suite

2021-09-24 Thread Fabian Paul (Jira)
Fabian Paul created FLINK-24368: --- Summary: Collect logs of Flink container in e2e source test suite Key: FLINK-24368 URL: https://issues.apache.org/jira/browse/FLINK-24368 Project: Flink Issue

[jira] [Created] (FLINK-24347) KafkaSource cannot checkpoint if the parallelism is higher than the partition number

2021-09-21 Thread Fabian Paul (Jira)
Fabian Paul created FLINK-24347: --- Summary: KafkaSource cannot checkpoint if the parallelism is higher than the partition number Key: FLINK-24347 URL: https://issues.apache.org/jira/browse/FLINK-24347

  1   2   >