Hi Colin,
Thanks for the proposal. A few initial comments comments/questions below:
1. I don't follow why we need a separate configuration for
`controller.listeners`. The current listener configuration already allows
users to specify multiple listeners, which allows them to define internal
endpoi
we chose this value?
>
> On Mon, Jul 13, 2020 at 10:09 AM Jason Gustafson
> wrote:
>
> > Hi Colin,
> >
> > Thanks for the proposal. A few initial comments comments/questions below:
> >
> > 1. I don't follow why we need a separate configuration for
&g
> 208. Performance. Do we plan to do group commit (e.g. buffer pending
> appends during a flush and then flush all accumulated pending records
> together in the next flush) for better throughput?
>
> 209. "the leader can actually defer fsync until it knows "quorum.siz
Hi Matthias,
Sorry for jumping in so late here. I am trying to understand why it was
necessary to deprecate `retries` in the producer. One of the use cases that
I see in practice is setting `retries` to 0. This allows applications to
control the retry semantics themselves. For example, I have seen
now all the kafka interactions are through writes and watches).
> > >> >
> > >> > This is a very good question. For our v1 implementation we are not
> > >> aiming
> > >> to guarantee linearizable read, which
> > >> would be cons
llable field
> consistently in all new requests?
>
> Jun
>
> On Wed, Jul 29, 2020 at 12:20 PM Jason Gustafson
> wrote:
>
> > Hey Jun,
> >
> > I added a section on "Cluster Bootstrapping" which discusses clusterId
> > generation and the process t
Hi All, I'd like to start a vote on this proposal:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-595%3A+A+Raft+Protocol+for+the+Metadata+Quorum.
The discussion has been active for a bit more than 3 months and I think the
main points have been addressed. We have also moved some of the pieces
Hi Jose,
Thanks for the proposal. I think there are three main motivations for
snapshotting over the existing compaction semantics.
First we are arguing that compaction is a poor semantic fit for how we want
to model the metadata in the cluster. We are trying to view the changes in
the cluster as
anywhere!
-Jason
On Mon, Aug 3, 2020 at 4:36 PM Jose Garcia Sancio
wrote:
> +1.
>
> Thanks for the detailed KIP!
>
> On Mon, Aug 3, 2020 at 11:03 AM Jason Gustafson
> wrote:
> >
> > Hi All, I'd like to start a vote on this proposal:
> >
> https://c
Hey Randall,
Thanks for putting this together. I think it would be great if the blog
included the names of the release contributors. It's an easy way to give
some recognition. I know we have done that in the past.
Thanks,
Jason
On Wed, Aug 5, 2020 at 8:25 AM Randall Hauch wrote:
> I've prepare
Hi Ryan,
Thanks for the proposal. Just a few quick questions:
1. I wonder if we need to bother with `enable.dynamic.config`, especially
if the default is going to be true anyway. I think users who don't want to
use this capability can just not set dynamic configs. The only case I can
see an expli
h regards to the Quorum State file, do we have anything that helps us
> detect corruption?
>
> Ismael
>
>
> On Mon, Aug 3, 2020 at 11:03 AM Jason Gustafson
> wrote:
>
> > Hi All, I'd like to start a vote on this proposal:
> >
> >
> https://c
Hi All,
I've added a proposal to handle the problem of hanging transactions:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-664%3A+Provide+tooling+to+detect+and+abort+hanging+transactions.
In theory, this should never happen. In practice, we have hit one bug where
it was possible and there
ons is considerable. I understand the mantra of "never
> fall too many versions behind" but the reality of it is that it isn't
> always the case. Even if the version is relatively recent, an upgrade may
> still not be possible for some time, and a quicker resolution may be
&
rote:
>
> > Yes, that definitely sounds reasonable. Thanks, Jason!
> >
> > Ron
> >
> > On Wed, Aug 26, 2020 at 3:03 PM Jason Gustafson
> > wrote:
> >
> > > Hey Ron,
> > >
> > > We do not typically backport new APIs to older
al.id.expiration.ms in the
> > DescribeProducersResponse?
> >
> > > That's an interesting thought as well. Are you trying to avoid the need
> > to
> > specify it through the command line? The tool could also query the value
> > with DescribeConfigs I
Hi Gokul,
Thanks, I think it makes sense to use a separate exception type. +1 on
Sophie's suggestion for `TransactionAbortedException`.
Extending from `RetriableException` seems reasonable as well. I guess the
only question is whether it's safe to catch it as a `RetriableException`
and apply comm
t is legitimate to return the
> producer
> > information to her anyways. But that is debatable in the meta point 3)
> > above.
> > 7.3 "Describing Transactions": we should also explain how that would be
> > executed, e.g. at least we should clarify that we would fi
y original thinking was that this metric might
help an operator identify partitions specifically, but that was before I
added the --find-hanging command. I guess we can remove it.
-Jason
On Mon, Aug 31, 2020 at 12:28 PM Jason Gustafson wrote:
> Hey Guozhang,
>
> Thanks for the detaile
> Hmm, but the "TxnStartOffset" is not included in the DescribeProducers
response either?
Oh, I accidentally called it `CurrentTxnStartTimestamp` in the schema.
Fixed now!
-Jason
On Mon, Aug 31, 2020 at 3:04 PM Guozhang Wang wrote:
> On Mon, Aug 31, 2020 at 12:28 PM Jason Gu
Hi All,
I'd like to start a vote on KIP-664:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-664%3A+Provide+tooling+to+detect+and+abort+hanging+transactions.
Thanks for all the feedback!
Best,
Jason
+1 Thanks for the KIP!
On Thu, Sep 3, 2020 at 12:25 PM Gokul Srinivas wrote:
> Hi,
>
> I would like to call a vote on the following KIP.
>
> *KIP *-
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-654:+Aborted+transaction+with+non-flushed+data+should+throw+a+non-fatal+exception
> <
> h
s Jason, I do not have more comments on the KIP then.
> >
> > On Mon, Aug 31, 2020 at 3:19 PM Jason Gustafson
> > wrote:
> >
> > > > Hmm, but the "TxnStartOffset" is not included in the
> DescribeProducers
> > > response either?
> > &
ll *hanging
> transactions in the cluster, do I need to write the script on my own? Maybe
> we could discuss a bit on whether we would like to support a more holistic
> API, or this is good for now.
>
>
> On Thu, Sep 10, 2020 at 7:53 AM Tom Bentley wrote:
>
> > Sounds
Hi Jose,
A few comments/questions below:
1. There is a comment in the proposal which suggests that we will maintain
multiple snapshots:
> Having multiple snapshots is useful for minimizing re-fetching of the
snapshot when a new snapshot is generated.
However, the document later says that snapsh
On Tue, Sep 8, 2020 at 3:04 PM Ron Dagostino
> wrote:
> > >
> > > > +1 (non-binding) -- Thanks, Jason!
> > > >
> > > > Ron
> > > >
> > > > On Tue, Sep 8, 2020 at 2:04 PM Jason Gustafson
> > > wrote:
> > > >
> >
Hi Justine,
Thanks for picking up this work. I have a few questions/comments:
1. It sounds like the directory structure is still going to be based on
topic names. Do I have that right? One complication is that the
LeaderAndIsr request does not include the topic name any longer. This means
that a
Hey Boyang,
Thanks for the update. This seems like the best thing we can do. The
alternative would be to always ensure that the forwarded APIs are safe for
conversion between versions, but that would restrict the flexibility that
the versioning is providing. It would also be a large effort to avoi
Thanks Jose. Makes sense overall. A few specific responses below:
> Generally the number of snapshots on disk will be one. I suspect that
users will want some control over this. We can add a configuration
option that doesn't delete, or advances the log begin offset past, the
N latest snapshots. We
n get rid of
> IBP in the future. So does this change prevent us from removing IBP in the
> future?
>
> Thanks,
>
> Jun
>
> On Thu, Sep 24, 2020 at 12:46 PM Jason Gustafson
> wrote:
>
> > Hey Boyang,
> >
> > Thanks for the update. This seems like
e KIP. Most of the changes mentioned in the KIP are for supporting topic
> deletion and I believe that is why the produce request was listed under
> future work.
>
> 4. This sounds like it might be a good solution, but I will need to discuss
> more with KIP-500 folks to get the details rig
t; > > changes here are fine.
> > > > >
>
> Yeah. I think it would be good to make IBP a feature flag, as long as it
> could be changed without doing a second rolling restart. We actually don't
> want to have too many feature flags, since it blows up the test matrix.
>
>
Hey Andrew,
+1 on the KIP. For many users of Kafka, it may not be fully understood how
much of a challenge client monitoring is. With tens of clients in a
cluster, it is already difficult to coordinate metrics collection. When
there are thousands of clients, and when the cluster operator has no
co
+1 Thanks for the KIP
On Tue, Oct 3, 2023 at 12:30 PM David Jacot wrote:
> Thanks for the KIP. +1 from me as well.
>
> Best,
> David
>
> Le mar. 3 oct. 2023 à 20:54, Jun Rao a écrit :
>
> > Hi, Mayank,
> >
> > Thanks for the detailed explanation in the KIP. +1 from me.
> >
> > Jun
> >
> > On We
Hey Andrew,
Thanks for the KIP. Just clarifying a couple small details.
1. I assume any broker can handle this API, so admin clients will choose a
node randomly?
2. Does the controller need to support this API? If not, we can drop
"controller" from "listeners."
Thanks,
Jason
On Thu, Nov 16, 202
+1 Thanks for the KIP!
On Mon, Nov 20, 2023 at 9:31 AM Jun Rao wrote:
> Hi, Andrew,
>
> Thanks for the KIP. +1
>
> Jun
>
> On Thu, Nov 16, 2023 at 9:12 AM Andrew Schofield <
> andrew_schofield_j...@outlook.com> wrote:
>
> > Hi Apoorv,
> > Thanks for your vote.
> >
> > Initially, I put support fo
The KIP makes sense. +1
On Mon, Nov 20, 2023 at 12:37 PM David Arthur
wrote:
> Thanks Colin,
>
> +1 from me
>
> -David
>
> On Tue, Nov 14, 2023 at 3:53 PM Colin McCabe wrote:
>
> > Hi all,
> >
> > I'd like to call a vote for KIP-1001: Add CurrentControllerId metric.
> >
> > Take a look here:
>
Hey Alyssa,
Thanks for the KIP! I have a couple questions:
1. I might have missed it in the KIP, but could you clarify what happens
when a pre-vote fails (i.e. a majority of voters reject the potential
candidacy)? The transition descriptions only mention what happens if the
prospective leader lea
request includes a bumped epoch when in
> actuality there is no epoch bump. I've changed the VoteRequest json a bit
> in the KIP, let me know what you think.
>
> On Mon, Nov 27, 2023 at 1:40 PM Jason Gustafson >
> wrote:
>
> > Hey Alyssa,
> >
> > Thanks f
Hey Raman,
Thanks for the KIP! I think it makes sense. I agree that this becomes
especially useful in the context of KIP-939 because transactions can last
an indefinite amount of time, but it is useful even today. A large cluster
may have a very large number of ongoing transactions at any time, so
+1 Thanks for the KIP!
On Wed, Mar 13, 2024 at 5:13 AM Omnia Ibrahim
wrote:
> I had a look at the discussion thread and the KIP looks exciting.
> +1 non-binding
>
> Best
> Omnia
>
> On 1 Dec 2023, at 19:06, Artem Livshits
> wrote:
>
> Hello,
>
> This is a voting thread for
>
> https://cwiki.apa
Hey Ismael,
I'm considering if we can do something in this KIP for the SASL baggage
we've accumulated. Prior to the existence of the `SaslHandshake` API, we
supported the raw SASL protocol. The main gap was that it did not support
negotiation of the SASL method. This was fixed in
https://cwiki.apa
Minor correction: only `SaslHandshake` was introduced in KIP-43.
`SaslAuthenticate` came later in KIP-152.
On Thu, Dec 7, 2023 at 3:18 PM Jason Gustafson wrote:
> Hey Ismael,
>
> I'm considering if we can do something in this KIP for the SASL baggage
> we've accumulated.
+1 Thanks for the KIP! Nice to see progress with the raft protocol.
On Thu, Dec 7, 2023 at 5:10 PM Alyssa Huang
wrote:
> Hey folks,
>
> I would like to start a vote on Pre-vote 😉 Thank you Jose, Jason, Luke,
> and Jun for your comments on the discussion thread!
>
> Here's the link to the proposa
Hi Jose,
Thanks for the KIP! A few initial questions below:
1. In the user experience section, the user is expected to provide supply
the UUID for each voter. I'm assuming this is the directory.id coming from
KIP-858. I thought it was generated by the format comand automatically? It
seems like we
ely? Or do we give up and return a timeout error if the new voter
cannot catch up? Probably the latter makes the most sense.
Thanks,
Jason
On Tue, Jan 9, 2024 at 11:42 PM Colin McCabe wrote:
> On Tue, Jan 9, 2024, at 17:07, Jason Gustafson wrote:
> > Hi Jose,
> >
> > T
HI Raman,
Thanks for the KIP! +1 from me.
One small thing: we will probably have to overload the constructor for
TransactionDescription in order to add the new update time field to avoid
breaking the API. We might consider whether we need the overload to be
public or not.
Best,
Jason
On Tue, Ja
Hey Jose,
A few more questions:
1. When adding a voter, the KIP proposes to return a timeout error if the
voter cannot catch up in time. It might be more useful to return a more
specific error so that an operator can understand why the timeout occurred.
Alternatively, perhaps we could keep the ge
+1 Thanks for the KIP!
On Wed, Feb 21, 2024 at 9:15 AM Jun Rao wrote:
> Hi, Apoorv,
>
> Thanks for the KIP. +1
>
> Jun
>
> On Mon, Feb 19, 2024 at 2:32 PM Apoorv Mittal
> wrote:
>
> > Hi,
> > I’d like to start the voting for KIP-1019: Expose method to determine
> > Metric Measurability.
> >
> >
Thanks Jose. +1. Great KIP!
On Fri, Mar 29, 2024 at 11:16 AM Jun Rao wrote:
> Hi, Jose,
>
> Thanks for the KIP. +1
>
> Jun
>
> On Fri, Mar 29, 2024 at 9:55 AM José Armando García Sancio
> wrote:
>
> > Hi all,
> >
> > I would like to call a vote to adopt KIP-853.
> >
> > KIP: https://cwiki.apach
+1 Thanks for the KIP!
On Tue, Mar 24, 2020 at 10:31 AM Boyang Chen
wrote:
> +1 (non-binding).
>
> agree with Guozhang for the warning.
>
> On Tue, Mar 24, 2020 at 9:57 AM Guozhang Wang wrote:
>
> > +1 (binding).
> >
> > One minor thing is that upon starting, when we find the specified
> > `max
#x27;d like to add a per-partition DeletePartition field to
> combine
> > > everything in
> > > one request. This will reduce the number of requests sent to each
> broker
> > > and also
> > > increase the batching. I've already implemented it.
> >
Hey Boyang,
Thanks for the KIP. I think the main problem we've identified here is that
the current errors conflate transaction timeouts with producer fencing. The
first of these ought to be recoverable, but we cannot distinguish it. The
suggestion to add a new error code makes sense to me, but it
Hi All,
I'd like to start a discussion on KIP-595:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-595%3A+A+Raft+Protocol+for+the+Metadata+Quorum.
This proposal specifies a Raft protocol to ultimately replace Zookeeper as
documented in KIP-500. Please take a look and share your thoughts.
A
Hi Boyang,
A few minor questions below:
1. You mention UNKNOWN_PRODUCER_ID in 2.a under Resilience Improvements. I
assume that should be INVALID_PRODUCER_EPOCH? I am not sure this case makes
sense for 2.5 clients which would view this error as fatal regardless of
whatever the broker does. Not sur
the
> Raft protocol
> just to be used in the management of metadata in Zookeeper or we will also
> use it to replace the current logical of managing log-replica since the
> algorithm we used to manage log-replica is analogous to Raft.
>
>
> On Fri, Apr 17, 2020 at 7:45 AM
t; to
> > > > Jason first and go back here :)
> > > > I think the partition leader will have no change of returned code
> after
> > > we
> > > > discussed that only new client should be able to retry.
> > > >
> > > >
> > &g
As promised, here is a link to the current prototype:
https://github.com/confluentinc/kafka/tree/kafka-raft.
Thanks,
Jason
On Mon, Apr 20, 2020 at 10:53 AM Jason Gustafson wrote:
> Hi Deng,
>
> Thanks for the question. I mentioned this in the rejected alternatives
> section.
; > It's amazing to see this coming together :)
> > > > > >
> > > > > > I haven't had a chance to read in detail, but I read the outline
> > and
> > > a
> > > > > few
> > > > > > things jumped out at me.
+1 This looks useful.
-Jason
On Mon, May 18, 2020 at 9:59 AM Guozhang Wang wrote:
> Zhiguo, thanks for the KIP. +1 from me.
>
>
> Guozhang
>
> On Fri, May 15, 2020 at 2:21 PM Zhiguo Huang
> wrote:
>
> > Thanks to everyone for their input. I've incorporated the changes, and I
> > think this is
Hi Colin,
Looks good. I just had one question. It sounds like your intent is to
change kafka-configs.sh so that the --zookeeper flag is only supported for
bootstrapping. I assume in the case of SCRAM that we will only make this
change after the broker API is available?
Thanks,
Jason
On Tue, May
Hi David,
This looks good. I just have a few comments:
1. I'm not sure it's totally fair to describe the current notification
mechanism as "best-effort." At least it guarantees that the controller will
eventually see the event. In any case, I think we might want a stronger
contract going forward.
Sounds good. +1 from me.
On Tue, May 19, 2020 at 5:41 PM Colin McCabe wrote:
> On Tue, May 19, 2020, at 09:31, Jason Gustafson wrote:
> > Hi Colin,
> >
> > Looks good. I just had one question. It sounds like your intent is to
> > change kafka-configs.sh so that t
+1 I agree with Guozhang that broker epoch will need a separate discussion.
Thanks!
Jason
On Thu, May 28, 2020 at 9:34 AM Guozhang Wang wrote:
> David, thanks for the KIP. I'm +1 on it as well.
>
> One note is that in post-ZK world, we would need a different way to get
> broker epoch since it i
Hi All
The PMC for Apache Kafka has invited Deng Ziming to become a committer,
and we are excited to announce that he has accepted!
Ziming has been contributing to Kafka for about three years. He has authored
more than 100 patches and helped to review nearly as many. In particular,
he made signif
Hi David,
A few questions below:
1. In regard to this comment:
> Every member is uniquely identified by a UUID. This is called the Member
ID. This UUID is generated on the server side and given to the member when
it joins the group. It is used in all the communication with the group
coordinator
transactional markers. That would
> allow us to abort the transaction when the coordinator is reloaded. I
> wanted to cover this separately from the current proposal. Do you
> think that we should do this in this KIP?
>
> Cheers,
> David
>
> On Wed, Oct 19, 2022 at 6:53 PM J
+1 Thanks for all the hard work.
-Jason
On Tue, Oct 25, 2022 at 7:17 AM David Jacot
wrote:
> Hi all,
>
> The vote has been open for a while. I plan to close it on Friday if
> there are no further comments in the discussion thread.
>
> Best,
> David
>
> On Wed, Oct 19, 2022 at 6:10 PM Jun Rao w
Thanks, +1 from me. I suspect we might be in for at least one surprise with
the re-implemented controller RPCs, but I agree the alternative has risks
as well.
Best,
Jason
On Mon, Nov 14, 2022 at 12:00 PM Colin McCabe wrote:
> On Fri, Nov 11, 2022, at 08:59, David Arthur wrote:
> > Thanks, Colin
Hi Justine,
Thanks for the proposal.
I was thinking about the implementation a little bit. In the current
proposal, the behavior depends on whether we have an old or new client. For
old clients, we send `DescribeTransactions` and verify the result and for
new clients, we send `AddPartitionsToTxn`
ID and one validation per 1 produce request.
> >
> > Finally with respect to the authorizations, I think it makes sense to
> skip
> > topic authorizations, but I'm a bit confused by the "leader ID" field.
> > Wouldn't we just want to flag the request
+1. Thanks Justine!
-Jason
On Tue, Jan 10, 2023 at 3:46 PM Colt McNealy wrote:
> (non-binding) +1. Thank you for the KIP, Justine! I've read it; it makes
> sense to me and I am excited for the implementation.
>
> Colt McNealy
> *Founder, LittleHorse.io*
>
>
> On Tue, Jan 10, 2023 at 10:46 AM Ju
n a set of brokers in the response, adding a brokerId field to
> > identify which
> > broker a specific broker status response is associated with seems
> straight
> > forward.
> >
> > I’ll start writing a proposal so that we have something concrete to
> > di
+1 (binding)
Verified release notes and ran through the 2.12 quickstart.
Thanks Randall!
On Thu, Nov 14, 2019 at 8:52 PM Gwen Shapira wrote:
> +1 (binding)
>
> Thanks Randall. Verified signatures and tests.
>
> On Fri, Oct 25, 2019 at 7:10 AM Randall Hauch wrote:
> >
> > Hello all, we identif
This seems not very controversial, so I will go ahead and move to a vote in
the next couple days.
Thanks,
Jason
On Wed, Nov 13, 2019 at 10:23 AM Jason Gustafson wrote:
> Hey Colin,
>
> Thanks for the suggestions. I've clarified the use of the API overrides
> and the reque
I'd like to start a vote on KIP-533:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-533%3A+Add+default+api+timeout+to+AdminClient
.
+1 from me
Thanks,
Jason
to
> be
> > updated as it currently mentions "request timeout". An example from
> > CreateTopicsOptions:
> >
> > "Set the request timeout in milliseconds for this operation or {@code
> null}
> > if the default request timeout for the AdminClient should
Hi Manikumar,
We've gotten to the bottom of
https://issues.apache.org/jira/browse/KAFKA-9212 and I think it should be a
blocker. The impact of this bug is that consumers cannot make progress
while a reassignment is in progress. The problem is missing logic in the
controller to update its metadata
Hi Brian,
Thanks for the proposal! I have a couple comments/questions:
1. I'm having a hard time understanding the point of `QuotaEntity.Type`. It
sounds like this might be just for convenience since the APIs are using
string types. If so, I think it's a bit misleading to represent it as an
enum.
Hi David,
Just one question. In addition to including the protocol type, I'm
wondering if there is value in adding the protocol name to SyncGroup? This
would potentially give you the ability to parse the "user data" field in
the consumer group schema.
Thanks,
Jason
On Tue, Jan 21, 2020 at 9:35 A
nse. Let me update the KIP.
>
> Thanks,
> David
>
> On Tue, Jan 21, 2020 at 12:52 PM Jason Gustafson
> wrote:
>
> > Hi David,
> >
> > Just one question. In addition to including the protocol type, I'm
> > wondering if there is value in adding the p
Thanks for the proposal. Looks good overall. I wanted to suggest a possible
name change. I was considering something like `idle.metadata.expiration.ms`
or maybe `metadata.max.idle.ms`. Thoughts?
-Jason
On Tue, Jan 21, 2020 at 11:38 AM Guozhang Wang wrote:
> Got it.
>
> I was proposing that we
ata.max.age.ms for determining refresh period (which is an
> actual
> period).
>
> I've updated the docs.
>
> Thanks,
> Brian
>
> On Wed, Jan 22, 2020 at 6:19 PM Jason Gustafson
> wrote:
>
> > Thanks for the proposal. Looks good overall. I wanted
+1
Nice to see this API finally resolved.
-Jason
On Tue, Jan 28, 2020 at 4:30 AM Rajini Sivaram
wrote:
> +1 (binding)
>
> Thanks for the KIP, Brian!
>
> Regards,
>
> Rajini
>
> On Mon, Jan 27, 2020 at 10:35 PM Colin McCabe wrote:
>
> > Thanks, Brian.
> >
> > +1 (binding)
> >
> > best,
> > Col
Hey David,
Thanks, it makes sense to prevent reordering, especially for the case of
reassignment. When a topic is deleted, however, I am not sure we will have
a bumped epoch to send. I guess for that case, we could send a sentinel
which would take the existing semantics of overriding any existing
The PMC for Apache Kafka has invited Konstantine Karantasis as a committer
and we
are pleased to announce that he has accepted!
Konstantine has contributed 56 patches and helped to review even more. His
recent work includes a major overhaul of the Connect task management system
in order to support
+1
On Mon, Feb 24, 2020 at 2:16 AM David Jacot wrote:
> Hi all,
>
> I would like to start a vote on KIP-570: Add leader epoch in
> StopReplicaRequest
>
> The KIP is here:
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-570%3A+Add+leader+epoch+in+StopReplicaRequest
>
> Thanks,
> David
>
Hi Sanjana,
The KIP looks good to me. I had just one question about the default
behavior. As I understand, if the user has specified `retry.backoff.ms`
explicitly, then we will not apply the default max backoff. As such,
there's no way to get the benefit of this feature if you are providing a `
re
l learn
> that afterwards by reading the release notes introducing this KIP anyways.
>
>
> Guozhang
>
> On Thu, Mar 19, 2020 at 3:10 PM Jason Gustafson
> wrote:
>
> > Hi Sanjana,
> >
> > The KIP looks good to me. I had just one question about the default
> &
Hi Jose,
Thanks for the KIP. Just a minor question about this:
> This means that the leader will not allow followers to join the ISR until
it has recovered from the unclean leader election.
If I understand correctly, the main reason for this is to avoid the need to
propagate the "IsUnclean" flag
Hey David,
Thanks for the proposal. This was a tricky bug and I think your approach is
probably the best way forward.
It would be helpful to add a little more detail to the proposal. When the
coordinator detects that the static leader is returning, it will set
`skipAssignment` to true in the `Joi
For a little background on why `AlterIsr` returns the state back in the
response, originally the idea was that the partition leader could use the
response to reset its own state after a failed request. The tricky thing
for ISR changes is always ensuring that the partition leader reflects a
worst-ca
Hi Colin,
Thanks for the writeup. I had one question about bootstrapping. For the
brokers, I understand that listener startup is delayed until the Authorizer
is ready. However, I was not very clear how this would work for the
controller listeners. We may need them to startup before the metadata lo
l
> is a bit weird at the moment so we need to fix it anyway. We could
> use this KIP to fix the protocol and do a subsequent KIP in the future for
> the server side monitoring if we need it.
>
> Best,
> David
>
> On Fri, Jan 21, 2022 at 7:51 PM Jason Gustafson
> wrote:
>
Hey Jose,
Thanks for the updates. I noticed that `LeaderRecoveryState` is marked as
ignorable in the `AlterPartition` request. It would be helpful to
understand the motivation for that.
Thanks,
Jason
On Wed, Jan 26, 2022 at 10:25 AM Colin McCabe wrote:
> On Wed, Jan 26, 2022, at 09:14, José Ar
>> There are still a few use cases where super.users won't be required, but
> >> it may be useful in many cases to have this early start functionality.
> >>
> >> Leaving aside the preceding discussion, do you agree with starting up
> all
> >> endpo
do anyway. That is why we filed this issue:
https://issues.apache.org/jira/browse/KAFKA-13621. That said, I agree it
would be desirable if the same approach could be used by both the broker
and controller.
-Jason
On Thu, Jan 27, 2022 at 4:14 PM Colin McCabe wrote:
> On Thu, Jan 27, 2022, at 15:5
+1 Thanks!
On Wed, Jan 26, 2022 at 10:48 AM José Armando García Sancio
wrote:
> Hi all,
>
> I'd like to start a vote on KIP-704: Send a hint to the partition
> leader to recover the partition.
>
> KIP wiki: https://cwiki.apache.org/confluence/x/kAZRCg
> Discussion thread:
> https://lists.apache.
+1 Thanks!
On Mon, Jan 31, 2022 at 12:17 AM David Jacot wrote:
> Hi all,
>
> I'd like to start a vote about KIP-814: Static membership protocol
> should let the leader
> skip assignment.
>
> The KIP is here: https://cwiki.apache.org/confluence/x/C5-kCw.
>
> Best,
> David
>
other configs (which are the same as in AclAuthorizer).
>
> best,
> Colin
>
> On Thu, Jan 27, 2022, at 16:38, Jason Gustafson wrote:
> >> My intention here was to avoid exposing Kafka clients to
> > AUTHORIZER_NOT_READY_ERROR. If this error is only ever returned to
>
801 - 900 of 3206 matches
Mail list logo