From dev-return-110553-archive-asf-public=cust-asf.ponee.io@kafka.apache.org Thu Jan 16 14:01:20 2020 Return-Path: X-Original-To: archive-asf-public@cust-asf.ponee.io Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [207.244.88.153]) by mx-eu-01.ponee.io (Postfix) with SMTP id 81C2D18060E for ; Thu, 16 Jan 2020 15:01:20 +0100 (CET) Received: (qmail 126 invoked by uid 500); 16 Jan 2020 14:01:18 -0000 Mailing-List: contact dev-help@kafka.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@kafka.apache.org Delivered-To: mailing list dev@kafka.apache.org Received: (qmail 99721 invoked by uid 99); 16 Jan 2020 14:01:18 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd4-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 16 Jan 2020 14:01:18 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd4-us-west.apache.org (ASF Mail Server at spamd4-us-west.apache.org) with ESMTP id AED74C0620 for ; Thu, 16 Jan 2020 14:01:16 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd4-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -0.197 X-Spam-Level: X-Spam-Status: No, score=-0.197 tagged_above=-999 required=6.31 tests=[DKIM_SIGNED=0.1, DKIM_VALID=-0.1, DKIM_VALID_AU=-0.1, DKIM_VALID_EF=-0.1, RCVD_IN_DNSWL_NONE=-0.0001, RCVD_IN_MSPIKE_H3=0.001, RCVD_IN_MSPIKE_WL=0.001, SPF_HELO_NONE=0.001, SPF_PASS=-0.001, URIBL_BLOCKED=0.001] autolearn=disabled Authentication-Results: spamd4-us-west.apache.org (amavisd-new); dkim=pass (2048-bit key) header.d=gmail.com Received: from mx1-ec2-va.apache.org ([10.40.0.8]) by localhost (spamd4-us-west.apache.org [10.40.0.11]) (amavisd-new, port 10024) with ESMTP id M7Agb9HxZxwd for ; Thu, 16 Jan 2020 14:01:10 +0000 (UTC) Received-SPF: Pass (mailfrom) identity=mailfrom; client-ip=209.85.161.68; helo=mail-yw1-f68.google.com; envelope-from=mickael.maison@gmail.com; receiver= Received: from mail-yw1-f68.google.com (mail-yw1-f68.google.com [209.85.161.68]) by mx1-ec2-va.apache.org (ASF Mail Server at mx1-ec2-va.apache.org) with ESMTPS id 9B2DFBC589 for ; Thu, 16 Jan 2020 14:01:10 +0000 (UTC) Received: by mail-yw1-f68.google.com with SMTP id i126so12741709ywe.7 for ; Thu, 16 Jan 2020 06:01:10 -0800 (PST) DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=gmail.com; s=20161025; h=mime-version:references:in-reply-to:from:date:message-id:subject:to :content-transfer-encoding; bh=uygDbXBUZQVapkkztt5vmmD5ESttNvRUPbDE/keTtdw=; b=Rqz7Vpj44WTU1NL7BLY2d2rDzsOoRM8mwrX6ErJoAq4VKn9rhq0WFQKXsicbhcjxUm wWEAMlI46jVqHZMg32tCI2ydbyLrKKoI3Of3V1HhVFM2KDR3BnMi8H+Hck9Q5Hm4yGoR eRJaEgXD9tryc+TCFmO+miyjWScSQIUVaUilbS3yoXDtzuSNmVsUF5oHDY15JacCoZLS cL4ozr4o9rUv+0/BizzmuhkBcezifUiNofqhCq2PLfzOKAXJMrb5C07lWJooxGAxc5M7 EOCVJtwuFGnUcAJio3XEPYrbcFPJhWaVE1USwcALLDDMnDcSRK4CtkxeRBvEDxhDm5Kw ZEHw== X-Google-DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=1e100.net; s=20161025; h=x-gm-message-state:mime-version:references:in-reply-to:from:date :message-id:subject:to:content-transfer-encoding; bh=uygDbXBUZQVapkkztt5vmmD5ESttNvRUPbDE/keTtdw=; b=HQdgmBy/0rRAWqxo+Wbfd6X9BHas2DHiqhc2Zu6vCh6P1P8RsXlWkIKJYCNz6sC1Ey KtaMagyd60v8C7+qTcbMURaqZuMYip0yG/tdhpxWO2kzulZSa3J6OGHvRwkaPNahyyIn oHTs2111VaSJaZQlSD8r1mIOhDPTCr5HwMxUdqrfCftmy240RcmHIAF7DscLphSYfplG p/vvEMaeCmGQpHk+AhY3jdB9EQeiZNoP1vJWnmJgMbtmewdwGlIETjv5afjQCYS/Tofw WZCAUENLrdyR4hoiw4YUdQ5b11iJMozkNuW/H/HjTA7mKHc4LaLi4VQOIsSJBb/Z/AHC Y+fw== X-Gm-Message-State: APjAAAVEewURlHRIuD3nQkt52qiS0tDV5UNbGRorpePJwdDUFrXYLDmd Pg+p2Eo+OdiTVCRTKQqVnWsN5vFWGwkk1DuxHSfHZw== X-Google-Smtp-Source: APXvYqw5/CN7Iw4/5nHqlBa1Uh7lxQq2Al7XdXloJcNVf1kqTFD8+RTTkKKS1EW3xbv/PgDDlX8kYE97LtiwBumPmGQ= X-Received: by 2002:a0d:d182:: with SMTP id t124mr25498073ywd.96.1579183269638; Thu, 16 Jan 2020 06:01:09 -0800 (PST) MIME-Version: 1.0 References: In-Reply-To: From: Mickael Maison Date: Thu, 16 Jan 2020 14:00:57 +0000 Message-ID: Subject: Re: [VOTE] KIP-409: Allow creating under-replicated topics and partitions To: dev Content-Type: text/plain; charset="UTF-8" Content-Transfer-Encoding: quoted-printable Hi Manna, In your example, the topic 'dummy' is not under replicated. It just has 1 replica. A topic under replicated is a topic with less ISRs than replicas. Having under replicated topics is relatively common in a Kafka cluster, it happens everytime is broker is down. However Kafka does not permit it to happen at topic creation. Currently at creation, Kafka requires to have at least as many brokers as the replication factor. This KIP addresses this limitation. Regarding your 2nd point. When rack awareness is enabled, Kafka tries to distribute partitions across racks. When all brokers in a rack are down (ie: a zone is offline), you can end up with partitions not well distributed even with rack awareness. There are currently no easy way to track such partitions so I decided to not attempt addressing this issue in this KIP. I hope that answers your questions. On Wed, Jan 15, 2020 at 4:10 PM Kamal Chandraprakash wrote: > > +1 (non-binding). Thanks for the KIP! > > On Mon, Jan 13, 2020 at 1:58 PM M. Manna wrote: > > > 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 misundersto= od > > it) - currently it's possible (even though I am strongly against it) to > > create Kafka topics which are under-replicated; despite all brokers bei= ng > > online. This the the output of an intentionally under-replicated topic > > "dummy" with p=3D6 and RF=3D1 (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=3Dgzip,min.insync.replicas=3D2,cleanup.policy= =3Ddelete,segment.bytes=3D10485760,max.message.bytes=3D10642642,retention.b= ytes=3D20971520 > > 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 > > 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 wro= te: > > > > > > > > +1 non-binding. Thanks! > > > > > > > > On Wed, Dec 18, 2019 at 1:05 PM S=C3=B6nke Liebau > > > > wrote: > > > > > > > > > Hi Mickael, > > > > > > > > > > thanks for your response! That all makes perfect sense and I cann= ot > > > > > 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 w= hile > > > > > 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=C3=B6nke > > > > > > > > > > On Wed, 18 Dec 2019 at 11:20, Mickael Maison < > > mickael.maison@gmail.com > > > > > > > > > wrote: > > > > > > > > > > > > Thanks S=C3=B6nke for the feedback. > > > > > > > > > > > > I debated this point quite a bit before deciding to base creati= on > > > > > > around "min.insync.replicas". > > > > > > > > > > > > For me, the goal of this KIP is to enable administrators to pro= vide > > > > > > 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. Thi= s > > > > > > 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 chang= e > > this > > > > > > assumption. If this is something someone wants, I think it shou= ld > > be > > > > > > handled in a different KIP that targets that use case. By relyi= ng > > 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) ins= tead > > > of > > > > > > simply min.insync.replicas. I have updated the KIP to explicitl= y > > > > > > mention this point. > > > > > > > > > > > > I hope that answers your question, let me know. > > > > > > Thanks > > > > > > > > > > > > > > > > > > On Mon, Dec 16, 2019 at 4:38 PM S=C3=B6nke Liebau > > > > > > 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=C3=B6nke > > > > > > > > > > > > > > On Thu, 12 Dec 2019 at 22:44, Ryanne Dolan < > > ryannedolan@gmail.com> > > > > > wrote: > > > > > > > > > > > > > > > > +1 non-binding, thx > > > > > > > > > > > > > > > > On Thu, Dec 12, 2019 at 6:09 AM Mickael Maison < > > > > > mickael.maison@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 > > > > > > > > > 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+crea= ting+under-replicated+topics+and+partitions > > > > > > > > > > > > > > > > > > > > Thanks > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > -- > > > > > > > S=C3=B6nke Liebau > > > > > > > Partner > > > > > > > Tel. +49 179 7940878 > > > > > > > OpenCore GmbH & Co. KG - Thomas-Mann-Stra=C3=9Fe 8 - 22880 We= del - > > > Germany > > > > > > > > > > > > > > > > > > > > -- > > > > > S=C3=B6nke Liebau > > > > > Partner > > > > > Tel. +49 179 7940878 > > > > > OpenCore GmbH & Co. KG - Thomas-Mann-Stra=C3=9Fe 8 - 22880 Wedel = - Germany > > > > > > > > > > > > > > >