Hi Mikael,

Apologies for last minute question, as I just caught up with it. Thanks for
your work on the KIP.

Just trying to get your thoughts on one thing (I might have misunderstood
it) - currently it's possible (even though I am strongly against it) to
create Kafka topics which are under-replicated; despite all brokers being
online. This the the output of an intentionally under-replicated topic
"dummy" with p=6 and RF=1 (with a 3 node cluster)


virtualadmin@kafka-broker-machine-1:/opt/kafka/bin$ ./kafka-topics.sh
--create --topic dummy --partitions 6 --replication-factor 1
--bootstrap-server localhost:9092
virtualadmin@kafka-broker-machine-1:/opt/kafka/bin$ ./kafka-topics.sh
--describe --topic dummy  --bootstrap-server localhost:9092
Topic:dummy     PartitionCount:6        ReplicationFactor:1
Configs:compression.type=gzip,min.insync.replicas=2,cleanup.policy=delete,segment.bytes=10485760,max.message.bytes=10642642,retention.bytes=20971520
        Topic: dummy    Partition: 0    Leader: 3       Replicas: 3
Isr: 3
        Topic: dummy    Partition: 1    Leader: 1       Replicas: 1
Isr: 1
        Topic: dummy    Partition: 2    Leader: 2       Replicas: 2
Isr: 2
        Topic: dummy    Partition: 3    Leader: 3       Replicas: 3
Isr: 3
        Topic: dummy    Partition: 4    Leader: 1       Replicas: 1
Isr: 1
        Topic: dummy    Partition: 5    Leader: 2       Replicas: 2
Isr: 2

 This is with respect to the following statement on your KIP (i.e.
under-replicated topic creation is also permitted when none is offline):

*but note that this may already happen (without this KIP) when
> topics/partitions are created while all brokers in a rack are offline (ie:
> an availability zone is offline). Tracking topics/partitions not optimally
> spread across all racks can be tackled in a follow up KIP.  *




Did you mean to say that such under-replicated topics (including
human-created ones) will be handled in a separete KIP ?

Regards,


On Mon, 13 Jan 2020 at 10:15, Mickael Maison <mickael.mai...@gmail.com>
wrote:

> Hi all.
>
> With 2.5.0 approaching, bumping this thread once more as feedback or
> votes would be nice.
>
> Thanks
>
> On Wed, Dec 18, 2019 at 1:59 PM Tom Bentley <tbent...@redhat.com> wrote:
> >
> > +1 non-binding. Thanks!
> >
> > On Wed, Dec 18, 2019 at 1:05 PM Sönke Liebau
> > <soenke.lie...@opencore.com.invalid> wrote:
> >
> > > Hi Mickael,
> > >
> > > thanks for your response! That all makes perfect sense and I cannot
> > > give any actual use cases for where what I asked about would be useful
> > > :)
> > > It was more the idle thought if this might be low hanging fruit while
> > > changing this anyway to avoid having to circle back later on and
> > > wanted to at least mention it.
> > >
> > > I am totally happy either way!
> > >
> > > Best regards,
> > > Sönke
> > >
> > > On Wed, 18 Dec 2019 at 11:20, Mickael Maison <mickael.mai...@gmail.com
> >
> > > wrote:
> > > >
> > > > Thanks Sönke for the feedback.
> > > >
> > > > I debated this point quite a bit before deciding to base creation
> > > > around "min.insync.replicas".
> > > >
> > > > For me, the goal of this KIP is to enable administrators to provide
> > > > higher availability. In a 3 node cluster configured for high
> > > > availability (3 replicas, 2 min ISR), by enabling this feature,
> > > > clusters should be fully usable even when 1 broker is down. This
> > > > should cover all "normal" maintenance operations like a rolling
> > > > restart or just the recovery of a broker.
> > > >
> > > > At the moment, when creating a topic/partition, the assumption is
> that
> > > > the resource will be fully functioning. This KIP does not change this
> > > > assumption. If this is something someone wants, I think it should be
> > > > handled in a different KIP that targets that use case. By relying on
> > > > "min.insync.replicas", we don't break any assumptions the user has
> and
> > > > this should be fully transparent from the user point of view.
> > > >
> > > > About "min.insync.replicas", one caveat that is not explicit in the
> > > > KIP is that it's currently possible to create topics with less
> > > > replicas than this settings. For that reason, I think the
> > > > implementation will actually rely on min(replicas, min-isr) instead
> of
> > > > simply min.insync.replicas. I have updated the KIP to explicitly
> > > > mention this point.
> > > >
> > > > I hope that answers your question, let me know.
> > > > Thanks
> > > >
> > > >
> > > > On Mon, Dec 16, 2019 at 4:38 PM Sönke Liebau
> > > > <soenke.lie...@opencore.com.invalid> wrote:
> > > > >
> > > > > Hi Michael,
> > > > >
> > > > > that sounds like a useful addition! I can't help but wonder
> whether by
> > > > > leaving in the restriction that "min.insync.replicas" has to be
> > > > > satisfied we'll be back here in a years time because someone has a
> > > > > scenario where he or she wants to go below that :)
> > > > > I don't have a strong opinion either way to be honest, just a
> random
> > > > > thought when reading the KIP.
> > > > >
> > > > > Best regards,
> > > > > Sönke
> > > > >
> > > > > On Thu, 12 Dec 2019 at 22:44, Ryanne Dolan <ryannedo...@gmail.com>
> > > wrote:
> > > > > >
> > > > > > +1 non-binding, thx
> > > > > >
> > > > > > On Thu, Dec 12, 2019 at 6:09 AM Mickael Maison <
> > > mickael.mai...@gmail.com>
> > > > > > wrote:
> > > > > >
> > > > > > > Bumping this thread, I've not seen any votes or feedback.
> > > > > > >
> > > > > > > On Wed, Nov 13, 2019 at 12:17 PM Mickael Maison
> > > > > > > <mickael.mai...@gmail.com> wrote:
> > > > > > > >
> > > > > > > > Hi all,
> > > > > > > >
> > > > > > > > I'd like to start a vote on KIP-409: Allow creating
> > > under-replicated
> > > > > > > > topics and partitions
> > > > > > > >
> > > > > > >
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-409%3A+Allow+creating+under-replicated+topics+and+partitions
> > > > > > > >
> > > > > > > > Thanks
> > > > > > >
> > > > >
> > > > >
> > > > >
> > > > > --
> > > > > Sönke Liebau
> > > > > Partner
> > > > > Tel. +49 179 7940878
> > > > > OpenCore GmbH & Co. KG - Thomas-Mann-Straße 8 - 22880 Wedel -
> Germany
> > >
> > >
> > >
> > > --
> > > Sönke Liebau
> > > Partner
> > > Tel. +49 179 7940878
> > > OpenCore GmbH & Co. KG - Thomas-Mann-Straße 8 - 22880 Wedel - Germany
> > >
> > >
>

Reply via email to