Re: Flink Kafka Partition Discovery Never Forgets Deleted Partitions

2021-09-14 Thread Constantinos Papadopoulos
and RELIABLY processed all of the data from that topic (which may > not be true). This would effectively lead to AT_MOST_ONCE delivery > guarantees (in other words, we are OK with loosing data), which is a > trade-off that _in_my_opinion_ we shouldn't make here. > > Best,

Re: Flink Kafka Partition Discovery Never Forgets Deleted Partitions

2021-09-14 Thread Constantinos Papadopoulos
Hi all, Thank you for the replies, they are much appreciated. I'm sure I'm missing something obvious here, so bear with me... Fabian, regarding: "Flink will try to recover from the previous checkpoint which is invalid by now because the partition is not available anymore." The above would happ

Re: Flink Kafka Partition Discovery Never Forgets Deleted Partitions

2021-09-14 Thread Constantinos Papadopoulos
verer (even if it's an opt-in capability). Would the Flink community be open to a contribution that does this? Best regards, Constantinos Papadopoulos On Tue, Sep 14, 2021 at 12:54 PM David Morávek wrote: > Hi Constantinos, > > The partition discovery doesn't support topic /

Flink Kafka Partition Discovery Never Forgets Deleted Partitions

2021-09-14 Thread Constantinos Papadopoulos
We are on Flink 1.12.1, we initialize our FlinkKafkaConsumer with a topic name *pattern*, and we have partition discovery enabled. When our product scales up, it adds new topics. When it scales down, it removes topics. The problem is that the FlinkKafkaConsumer never seems to forget partitions th

multi-tenancy without a kafka partition per tenant

2019-10-21 Thread Constantinos Papadopoulos
We have a multi-tenancy scenario where: - the source will be Kafka, and a Kafka partition could contain data from multiple tenants - our sink will send data to a different DB instance, depending on the tenant Is there a way to prevent slowness in one tenant from slowing other tenants