code
UNRELEASED_INSTANCE_ID during that time period. In case of failure of
the existing member, the new member has to wait until its session
expires.
Best,
David
On Wed, Oct 19, 2022 at 10:05 AM David Jacot wrote:
>
> Hi Jun,
>
> Thanks for your thorough review. There is already a vote thread if y
ngrades would still be possible.
>
> 3. The doc mentions that member metadata is stored in separate records in
> order to avoid the batch limit. The group assignment, on the other hand, is
> still stored as a single record. Will that be a scalability problem?
>
> Thanks,
> Jason
ents. In the metadata, we might identify
> all the member IDs that are covered by the assignment. Then when we load
> the assignment, we can validate that all the member assignment records
> are present. If not, then we consider it invalid and begin a
> new assignment.
> Just a though
gt; > think that we should do this in this KIP?
> >
> > That is a reasonable idea. I am wondering if there are lighter weight
> > options
> > though. Suppose that we used separate records for assignment metadata
> > and individual member assignments. In the metadat
Hi Mathieu,
Thanks for the effort that you have put in creating this KIP. I just read
it again and I am still confused by the use cases and the motivation. I
suppose that this works for your data model but it does not seem to be a
general pattern.
Overall, I stick to the comment that I made in th
ent on the latest update:
>
>
> Den mån 24 okt. 2022 kl 16:26 skrev David Jacot >:
>
> > * Jason pointed out that the member id handling is a tad weird. The
> > group coordinator generates the member id and then trusts the member
> > when it rejoins the group.
st one." means?
>
> Thanks,
>
> Jun
>
>
> On Mon, Oct 24, 2022 at 7:49 AM Magnus Edenhill wrote:
>
> > Hi, one minor comment on the latest update:
> >
> >
> > Den mån 24 okt. 2022 kl 16:26 skrev David Jacot
> > > >:
> >
> > &
minor comments pending in the discussion thread, and
> > > one
> > > > > is about whether we should merge PreparePartitionAssignment with HB.
> > > But
> > > > I
> > > > > think the KIP itself is in pretty good shape now. Thanks!
>
s for the reply.
>
> The KIP mentioned downgrade support in a future KIP. So, with this KIP,
> once the new records have been generated on the coordinator, there is no
> path to downgrade the broker, is that correct?
>
> Thanks,
>
> Jun
>
> On Tue, Oct 25, 2022 at 7:1
or 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.
> >
>
Congrats, Bruno! Well deserved.
Le mer. 2 nov. 2022 à 06:12, Randall Hauch a écrit :
> Congratulations, Bruno!
>
> On Tue, Nov 1, 2022 at 11:20 PM Sagar wrote:
>
> > Congrats Bruno!
> >
> > Sagar.
> >
> > On Wed, Nov 2, 2022 at 7:51 AM deng ziming
> > wrote:
> >
> > > Congrats!
> > >
> > > --
Hi Rajini,
Thanks for the KIP. I have a few questions/comments:
01. If I understood correctly, the plan is to add new assignors which
are rack aware. Is this right? I wonder if it is a judicious choice
here. The main drawback is that clients must be configured correctly
in order to get the benefi
be implemented.
Best,
David
On Fri, Nov 4, 2022 at 10:55 AM David Jacot wrote:
>
> Hi Rajini,
>
> Thanks for the KIP. I have a few questions/comments:
>
> 01. If I understood correctly, the plan is to add new assignors which
> are rack aware. Is this right? I wonder if it
Hi Travis,
Thanks for the KIP. That seems to be a useful addition. I have a few
concerns/comments:
01: Managed Kafka services do not necessarily run a specific version or may
not want to expose it. I suppose that they could keep an empty string.
02: I am a bit concerned by clients that could mis
be used.
> >
> > 03) Updated the KIP to use version 3, thanks.
> >
> > If there are no concerns or further comments, I will start voting later
> > today.
> >
> > Thank you,
> >
> > Rajini
> >
> >
> > On Fri, Nov 4, 2022 at 9:5
+1 (binding). Thanks for the KIP, Rajini!
On Tue, Nov 15, 2022 at 9:26 PM Maulin Vasavada
wrote:
>
> +1 (non-binding).
>
> Makes sense, Rajini. This would be a great addition.
>
> On Tue, Nov 15, 2022 at 10:55 AM Rajini Sivaram
> wrote:
>
> > Hi all,
> >
> > I would like to start vote on KIP-881
+1. Thanks for volunteering!
On Wed, Nov 16, 2022 at 4:09 PM Chris Egerton wrote:
>
> Hi all,
>
> I'd like to volunteer to be the release manager for the next bugfix
> release, 3.3.2.
>
> If there are no objections, I'll send out a release plan by EOD (Eastern
> Time) Friday that includes a list
Hi all,
I have drafted a very small KIP which proposes to support nullable
struct in the Kafka protocol. This is something that we plan to use
for KIP-848.
The KIP is here: https://cwiki.apache.org/confluence/x/YJIODg
Please let me know what you think.
Best,
David
; All that said: I think this information is worth it and unlikely to be
> > misused. IMO, ApiVersions is the main place to include this information,
> > but another alternative is ClusterMetadata. Adding these fields to
> > ClusterMetadata might be a bit awkward and may return st
> > > it will pass but of course if anything changes, just let me know.
> > >
> > > David, I'm fine with aiming to include KIP-866 in the 3.4 release as well
> > > since this
> > > seems to be a critical part of the zookeeper removal/migration. Please
&
ons with DescribeCluster, and added the original ApiVersions idea
> > to the list of rejected alternatives. Please take a look again and let me
> > know what you think.
> >
> > Thank you!
> > - Travis
> >
> > On 2022/12/02 15:35:09 David Jacot wrote:
> &
Hi all,
As this KIP-893 is trivial and non-controversial, I would like to
start the vote on it. The KIP is here:
https://cwiki.apache.org/confluence/x/YJIODg
Thanks,
David
er wrote:
> >
> > +1 (binding)
> >
> > Thanks,
> > -John
> >
> > On Mon, Dec 5, 2022, at 16:57, Kirk True wrote:
> >> +1 (non-binding)
> >>
> >> On Mon, Dec 5, 2022, at 10:05 AM, Colin McCabe wrote:
> >>> +1 (bin
ter response” rejected alternative.
>
> Let me know what you think,
> - Travis
>
> On 2022/12/02 17:25:37 David Jacot wrote:
> > Yeah, it is too late for 3.4. I have a few more comments:
> >
> > 04. `nullable-string` is not a valid type. You have to use &qu
Hi Sandeep,
Thanks for the KIP. Overall, your proposal seems to be a useful
addition to the command line tool. I made a first pass on it and I
have a few comments:
01. Adding `removeMembersFromConsumerGroup(String groupId,
Collection memberIds)` does not seem necessary because you can
already spe
Hi Sophie,
We just found a blocker for 3.4.0:
https://issues.apache.org/jira/browse/KAFKA-14532. The PR is on the
way.
Best,
David
On Sat, Dec 17, 2022 at 1:08 AM Sophie Blee-Goldman
wrote:
>
> Thanks Jose & Kirk. I agree both those fixes should be included in the 3.4
> release
>
> On Fri, Dec
trunk and is quite small. If you agree, I'll merge the fix to the 3.4
> branch.
>
> Thanks!
> David
>
> On Tue, Dec 20, 2022 at 7:53 AM David Jacot
> wrote:
>
> > Hi Sophie,
> >
> > We just found a blocker for 3.4.0:
> > https://issues.apache.org/jir
Hi all,
The PMC of Apache Kafka is pleased to announce a new Kafka committer Justine
Olshan.
Justine has been contributing to Kafka since June 2019. She contributed 53
PRs including the following KIPs.
KIP-480: Sticky Partitioner
KIP-516: Topic Identifiers & Topic Deletion State Improvements
KIP
> > Tom
> > > >
> > > >
> > > > On Wed, Jan 27, 2021 at 12:05 PM Magnus Edenhill
> > wrote:
> > > >
> > > > > Hey Thomas,
> > > > >
> > > > > I'm late to the game.
> > > > >
> > > > &g
Thanks for the KIP, Justine. +1 (binding)
On Fri, Feb 3, 2023 at 1:36 AM Matthias J. Sax wrote:
> Thanks for the KIP!
>
> +1 (binding)
>
>
> On 2/2/23 4:18 PM, Artem Livshits wrote:
> > (non-binding) +1. Looking forward to the implementation and fixing the
> > issues that we've got.
> >
> > -Ar
I performed the following validations:
* Verified all checksums and signatures.
* Built from source and ran unit tests.
* Ran the first quickstart steps for both ZK and KRaft.
* Spotchecked the Javadocs.
I am +1 (binding). Thanks for running the release!
Best,
David
On Tue, Jan 31, 2023 at 8:00
Hi Calvin,
Thanks for the KIP! The overall approach looks reasonable to me. I have a
few questions/comments:
01. I wonder if the leader should also verify the broker epochs based on
its metadata cache before sending the AlterPartition request to the
controller. Imagine the case where a follower n
Thanks for volunteering Mickael!
On Wed, Feb 8, 2023 at 4:23 PM Ismael Juma wrote:
> Thanks Mickael!
>
> Ismael
>
> On Wed, Feb 8, 2023 at 7:06 AM Mickael Maison
> wrote:
>
> > Hi,
> >
> > I'd like to volunteer as release manager for the next release, Apache
> > Kafka 3.5.0.
> > If there are no
> because the registration request has any delay? But the broker will not
> > > start fetching before the registration success.
> > > On the other hand, if the follower fetches with the stale broker epoch,
> > is
> > > it good enough to ask the leader to hold to i
Hi all,
I wrote a short KIP to surface the consumer's metadata in the
`describeConsumerGroups` of the `KafkaAdminClient`. The metadata is already
present in the protocol but not exposed yet. I think that it is handy when
troubleshooting consumer groups.
https://cwiki.apache.org/confluence/display
change. Please, let us know if you
have any objections.
Best,
David
On Mon, Oct 14, 2019 at 4:42 PM Xu Jianhai wrote:
> +1
>
> On Mon, Oct 14, 2019 at 3:47 PM David Jacot wrote:
>
> > Hi all,
> >
> > Jun,
> > The new fields are not flexible fields while the r
Hi Michael,
Please, excuse me for my late feedback. I've got a few questions/comments
while reviewing the KIP.
1. I would suggest to clearly state in the documentation of the state field
that omitting it or providing an empty list means "all".
2. Have you considered including the state in the re
Congrats!
On Wed, Jan 15, 2020 at 12:00 AM James Cheng wrote:
> Congrats Colin, Vahid, and Manikumar!
>
> -James
>
> > On Jan 14, 2020, at 10:59 AM, Tom Bentley wrote:
> >
> > Congratulations!
> >
> > On Tue, Jan 14, 2020 at 6:57 PM Rajini Sivaram
> > wrote:
> >
> >> Congratulations Colin, Vah
Hi all,
I just posted KIP-559: Make the Kafka Protocol Friendlier with L7 Proxies:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-559%3A+Make+the+Kafka+Protocol+Friendlier+with+L7+Proxies
Have a look and let me know what you think.
Best,
David
+1 (non-binding)
Thanks for the KIP!
David
Le ven. 17 janv. 2020 à 19:25, Colin McCabe a écrit :
> Hi all,
>
> I'm going to close the vote later today.
>
> thanks,
> Colin
>
>
> On Wed, Jan 15, 2020, at 06:34, M. Manna wrote:
> > +1 (Binding) - a long awaited KIP to have a simpler partition
>
Hi all,
FYI - I will start a vote on Monday if there no comments.
Regards,
David
Le mer. 15 janv. 2020 à 13:48, David Jacot a écrit :
> Hi all,
>
> I just posted KIP-559: Make the Kafka Protocol Friendlier with L7 Proxies:
>
> https://cwiki.apache.org/confluence/display/KAFKA/
Hi all,
I would like to start a vote on KIP-559: Make the Kafka Protocol Friendlier
with L7 Proxies.
The KIP is here:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-559%3A+Make+the+Kafka+Protocol+Friendlier+with+L7+Proxies
Thanks,
David
chema.
>
> Thanks,
> Jason
>
> On Tue, Jan 21, 2020 at 9:35 AM David Jacot wrote:
>
> > Hi all,
> >
> > I would like to start a vote on KIP-559: Make the Kafka Protocol
> Friendlier
> > with L7 Proxies.
> >
> > The KIP is here:
> >
>
te:
>
> > This is awesome! +1 (non binding)
> > Eno
> >
> > On Tue, Jan 21, 2020 at 10:00 PM Gwen Shapira wrote:
> > >
> > > Thank you for the KIP. Awesomely cloud-native improvement :)
> > >
> > > +1 (binding)
> > >
> > >
>
; > On Thu, Jan 23, 2020 at 1:55 PM Guozhang Wang
> wrote:
> > >
> > > Yeah that makes sense, it is a good-to-have if we can push through
> this in
> > > 2.5 but if we do not have bandwidth that's fine too :)
> > >
> > > Guozhang
> >
David Jacot wrote:
> Hi all,
>
> The vote has passed with +5 binding votes (Jason Gustafson, David Arthur,
> Gwen Shapira,
> Guozhang Wang, Harsha Chintalapani) and +2 non-binding votes (Eno
> Thereska, Satish Duggana).
>
> Thanks to everyone!
>
> Best,
> David
>
requires Describe on the Group.
> > >
> > > 3) Yes if ListGroups response includes the state, it makes sense to
> > > expose it via the command line tool and the AdminClient. With
> > > ConsumerGroupCommand, to avoid compatibility issues we can only print
Hi all,
I've put together a very small KIP which proposes to add the leader epoch
in the
StopReplicaRequest in order to make it robust to reordering:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-570%3A+Add+leader+epoch+in+StopReplicaRequest
Please take a look at the KIP and let me know w
e existing semantics of overriding any existing epoch?
>
> -Jason
>
> On Tue, Feb 11, 2020 at 12:48 PM David Jacot wrote:
>
> > Hi all,
> >
> > I've put together a very small KIP which proposes to add the leader epoch
> > in the
> > StopReplicaRequest in
I have updated the KIP. I will start a vote on Monday if there are no
further comments by then.
Best,
David
On Wed, Feb 19, 2020 at 5:59 PM David Jacot wrote:
> Hey Jason,
>
> You're right. The leader epoch is not bumped when a topic is deleted. Using
> a sentinel (e.g
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
Congrats!
Le jeu. 27 févr. 2020 à 06:58, Vahid Hashemian
a écrit :
> Congratulations Konstantine!
>
> Regards,
> --Vahid
>
> On Wed, Feb 26, 2020 at 6:49 PM Boyang Chen
> wrote:
>
> > Congrats Konstantine!
> >
> > On Wed, Feb 26, 2020 at 6:32 PM Manikumar
> > wrote:
> >
> > > Congrats Konstant
duce
> > > > the confusion but I am not sure it would be better. What do you
> think?
> > > Yes absolutely, it makes sense to reuse this existing flag. Good catch!
> > >
> > > > 2. Regarding the output of the command line when "--states" is used,
+1 (non-binding). Thanks for the KIP!
David
On Thu, Feb 6, 2020 at 10:45 PM Colin McCabe wrote:
> Hi Mickael,
>
> Thanks for the KIP. I left a comment on the DISCUSS thread as well.
>
> best,
> Colin
>
>
> On Thu, Feb 6, 2020, at 08:58, Mickael Maison wrote:
> > Hi Manikumar,
> >
> > I believe
r 2, 2020 at 11:20 AM Gwen Shapira wrote:
>
> > +1
> >
> > On Mon, Feb 24, 2020, 2:16 AM David Jacot wrote:
> >
> > > Hi all,
> > >
> > > I would like to start a vote on KIP-570: Add leader epoch in
> > > StopReplicaRequest
> &g
Hi all,
The vote has passed with +3 binding votes (Jason Gustafson, Gwen Shapira,
Jun Rao).
Thanks to everyone!
Best,
David
On Wed, Mar 4, 2020 at 9:02 AM David Jacot wrote:
> Hi Jun,
>
> You're right. I have noticed it while implementing it. I plan to use a
> default
>
Hi Aneel,
Thank you for the KIP. I agree that managing complex configurations is not
easy
with the current tool. Having the possibility to use properties file sounds
quite
handy to me. It makes it easier to edit and to reuse base configurations.
I wonder if we should also add a `--delete-config-f
dicate that the program should read form STDIN. This would be difficult
> to trigger accidentally, and more in line with the traditional conventions.
>
> On Mon, Mar 9, 2020, at 08:47, David Jacot wrote:
> > I wonder if we should also add a `--delete-config-file` as a counterpart
>
> user's point of view, if they mess up and forget to supply an
> argument, or
> > > > for some reason the parser doesn't treat something like an argument,
> the
> > > > program will appear to hang in a confusing way.
> > > >
> > > &g
+1 (non-binding), thanks for the KIP!
David
On Mon, Mar 16, 2020 at 4:06 PM Aneel Nazareth wrote:
> Hello all,
>
> Thanks to the folks who have given feedback. I've incorporated the
> suggestions, and think that this is now ready for a vote:
>
>
> https://cwiki.apache.org/confluence/display/KAF
; > > +1 (non-binding). I used the staged Scala 2.13 binaries and the staging
> > > Maven repository to run my tests. All seems to work fine.
> > >
> > > Thanks for running the release David!
> > >
> > > Jakub
> > >
> > > On Thu, Dec 2
Hello Kafka users, developers and client-developers,
This is the second candidate for release of Apache Kafka 3.1.0.
* Apache Kafka supports Java 17
* The FetchRequest supports Topic IDs (KIP-516)
* Extend SASL/OAUTHBEARER with support for OIDC (KIP-768)
* Add broker count metrics (KIP-748)
* Dif
Hey,
Done. Thanks for your interest!
Best,
David
On Fri, Jan 14, 2022 at 10:58 AM Daan Gertis wrote:
>
> Hey everyone,
>
> I would like to contribute a KIP that I think makes sense when building
> event-driven architectures at scale. I have been working with kafka for the
> past 6 years or so
t;- Briefly Walked Through 3.1 Documentation, Javadocs and Protocol Pages
>
> +1 (non-binding) for the release candidate
>
> Thanks for running this release
>
> Israel Ekpo
> Lead Instructor, IzzyAcademy.com
> https://www.youtube.com/c/izzyacademy
> https://izzyacade
Hi folks,
I'd like to start a discussion for KIP-814: Static membership protocol
should let the
leader skip assignment. This is a small extension to the static
membership protocol
to address KAFKA-13435.
The KIP is here: https://cwiki.apache.org/confluence/x/C5-kCw.
Please let me know what you t
the "Kafka Streams demo
> > and
> > the app development tutorial" links still reference the 2.5 release.
> > This is a very minor issue and it doesn't require a new RC, we can simply
> > update the links once released.
> >
> > +1(binding)
> >
This vote passes with 7 +1 votes (4 bindings) and no 0 or -1 votes.
+1 votes
PMC Members:
* Randall Hauch
* Bill Bejeck
* Mickael Maison
* David Jacot
Committers:
* No votes
Community:
* Michal Tóth
* Jakub Scholz
* Israel Ekpo
0 votes
* No votes
-1 votes
* No votes
Vote thread:
https
> the subscribed topics. The returning leader would not know about it
> because it has no way to collect the full assignment. Do you think this is
> a problem?
>
> Thanks,
> Jason
>
> On Wed, Jan 19, 2022 at 7:27 AM David Jacot wrote:
>
> > Hi folks,
> >
> >
Urban, David Arthur, David Jacot, David Mao, Dmitriy Fishman, Edoardo
Comar, Ewen Cheslack-Postava, Greg Harris, Guozhang Wang, Igor Soarez,
Ismael Juma, Israel Ekpo, Ivan Ponomarev, Jakub Scholz, James Galasyn,
Jason Gustafson, Jeff Kim, Jim Galasyn, JoeCqupt, Joel Hamill, John
Gray, John Roesler
t; from the consumer's own subscription). The leader will then send an empty
> > > `SyncGroup` request to collect its own assignment. Do I have that right?
> > >
> > > I think there might still be an edge case in this proposal (assuming I've
> > > understood
x27;s right. We are thinking about a new consumer protocol but
we don't have anything concrete to share with the community yet.
We hope to have something in the near future.
Best,
David
On Wed, Jan 26, 2022 at 3:10 PM David Jacot wrote:
>
> Hey Jason,
>
> I've updated the
Hi all,
If there are no further comments. I plan to start a vote thread
on Monday.
Best,
David
On Wed, Jan 26, 2022 at 3:16 PM David Jacot wrote:
>
> Hi Hector,
>
> Thanks for your comments.
>
> The idea is to add a flag to the onJoinLeader leader method. We
> mus
n. Is that right?
> I think you should make it clear in the KIP.
>
> Thank you.
> Luke
>
> On Fri, Jan 28, 2022 at 8:00 PM David Jacot
> wrote:
>
> > Hi all,
> >
> > If there are no further comments. I plan to start a vote thread
> > on Monday.
&g
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
Hey,
Thanks for the KIP. I have a few comments:
1. I think that it would be better to name the KIP: "GetOffsetShell
should support max-timestamp"
or something like that as this is the initial intent of the change.
2. There is a typo: `OffsetSpce` -> `OffsetSpec`.
3. It would be great if we coul
Hi Hector,
Thanks for the KIP. I finally had a bit of time to read it.
I understand that a few services have been leveraging Kafka's Group Membership
Protocol to do leader election and/or service discovery. However, I am
not entirely
convinced that Kafka should be used in that way because special
retry until `default.api.timeout.ms` is reached
and it seems that the admin client does the same by default. Do
you confirm this?
Best,
David
On Mon, Jan 31, 2022 at 11:12 AM David Jacot wrote:
>
> Hey,
>
> Thanks for the KIP. I have a few comments:
>
> 1. I think that it would
at
> we use
> different timeout in `KafkaConsumer.beginningOffsets` and
> `KafkaConsumer.endOffsets`,
> I will fix this.
>
> Please help to review the KIP and the bug, thank you.
>
> Best,
> Ziming Deng
>
>
> > On Feb 1, 2022, at 6:31 PM, David Jacot
Hi José,
Thanks for the KIP.
I have one question regarding how fetch from followers will
work when the leader is recovering. My understanding is that
the leader will reject any produce and fetch requests with a
NOT_LEADER_OR_FOLLOWER error while the followers
will fence any fetch requests based o
ship protocol should let the
> > leader skip assignment
> >
> > Hi David,
> >
> > Thanks for the KIP.
> >
> > I'm +1(non-binding)
> >
> > Thanks.
> > Luke
> >
> > Jason Gustafson 於 2022年2月1日 週二 上午7:11
> > 寫道:
> >
Sancio
wrote:
>
> Thanks for the feedback David Jacot
>
> David Jacot wrote:
> > I have one question regarding how fetch from followers will
> > work when the leader is recovering. My understanding is that
> > the leader will reject any produce and fetch requests with a
&g
+1. Thanks for volunteering, Bruno!
Le ven. 4 févr. 2022 à 16:03, Bruno Cadonna a écrit :
> Hi,
>
> I'd like to volunteer to be the release manager for our next
> feature release, 3.2.0. If there are no objections, I'll send
> out the release plan soon.
>
> Best,
> Bruno
>
nGroupRequest` instead, and also add a field
> >> into `JoinGroupResponse` in `JoinGroupResponseMember` field. That way, the
> >> broker can identify the old member from `JoinGroupRequest`. And the
> >> assignor can also get the "generation" info via the `Subscription`
&g
Hi Dongjin,
Thanks for the KIP.
It is not so clear to me why we decided not to support OFF in the
first place. I understand that entirely disabling a logger is rare.
I find the KIP a bit week at the moment for two reasons:
1) The KIP says that the levels that we use are not fully
consistent wit
Hi team,
I find our ever growing back of PRs a little frustrating, don't
you? I just made a pass over all the list and a huge chunk
of the PRs are abandoned, outdated or irrelevant with the
current code base. For instance, we still have PRs opened
back in 2015.
There is not a Github Action [1] fo
+1, thanks!
Le lun. 7 févr. 2022 à 17:41, José Armando García Sancio
a écrit :
> Thanks for the vote Jason.
>
> This is a friendly reminder to vote on this KIP.
>
> --
> -José
>
rev-owned partitions
> > > on the subscription protocol directly. So we can add the `StickyAssignor`
> > > in your paragraph talking about `CooperativeStickyAssignor` as well.
> > >
> > > * This sentence: "otherwise, we'll take the ownedPartitions as d
rations, the main difference
> are default.api.timeout.ms <http://default.api.timeout.ms/>, retries,
> request.timeout.ms <http://request.timeout.ms/>,metadata.max.age.ms
> <http://metadata.max.age.ms/>. Please take a look again when you are free.
>
> >
Hi Matthew,
Thanks for the KIP. I have a few minor comments:
1. In the public interface section, could we spell out
the configurations that we are changing with this
KIP? The name does not change but the semantic is
so it is good to be clear.
2. In the proposed changes section, I would rather
me
For reference, here is the KIP:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-815%3A++Support+max-timestamp+in+GetOffsetShell
Thanks for the KIP! +1 (binding)
Best,
David
Le mar. 22 févr. 2022 à 04:03, deng ziming a
écrit :
> Hey all, I'm starting the voting on KIP-815.
>
> This suppor
) {
> return generationId;
> }
>
> Thank you.
> Luke
>
> On Mon, Feb 21, 2022 at 5:24 PM David Jacot
> wrote:
>
> > Hi Luke,
> >
> > I apologize for my late reply. I was out for a while.
> >
> > Coming back to my previous point, could you
> - 4 +1(binding) votes (Luke, David, Mickael and John)
>
> Thanks to all that participated in the discussion and voting,
>
> --
> Ziming Deng
>
>
> > On Feb 22, 2022, at 2:56 PM, David Jacot wrote:
> >
> > For reference, here is the KIP:
> > https://
+1 (binding). Thanks for the KIP!
On Fri, Jan 28, 2022 at 6:13 PM Tom Bentley wrote:
>
> Hi Luke,
>
> Thanks for the KIP, +1 (binding).
>
> Kind regards,
>
> Tom
>
> On Wed, 19 Jan 2022 at 13:16, Luke Chen wrote:
>
> > Hi all,
> >
> > Bump this thread to see if there are other comments to this K
Hi Artem,
Thanks for the KIP! I have a few comments:
1. In the preamble of the proposed change section, there is still a
mention of the
-1 approach. My understanding is that we have moved away from it now.
2. I am a bit concerned by the trick suggested about the DefaultPartitioner and
the Unifor
Hi Sergio,
I wonder if « max-bytes » would be a better name than « max-batches-size ».
The intend is more explicit. What do you think?
Best,
David
Le sam. 5 mars 2022 à 10:36, Luke Chen a écrit :
> Hi Sergio,
>
> Thanks for the explanation! Very clear!
> I think we should put this example and
Thanks for running the release, Mickael.
I performed the following validations:
* Verified all checksums and signatures.
* Built from source and ran unit tests.
* Ran the first quickstart steps for both ZK and KRaft.
* Spotchecked the Javadocs.
However, the document still references 3.0.0 in all
+1. Thanks Tom!
Le mer. 9 mars 2022 à 19:10, Bill Bejeck a écrit :
> Thanks Tom! It's a +1 for me.
>
> -Bill
>
> On Wed, Mar 9, 2022 at 12:00 PM Ismael Juma wrote:
>
> > Thanks Tom. +1
> >
> > Ismael
> >
> >
> > On Wed, Mar 9, 2022 at 8:10 AM Tom Bentley wrote:
> >
> > > Hi,
> > >
> > > I'd l
Hi Artem,
Thanks for the KIP. This is a really nice improvement!
+1 (binding) from me.
David
On Tue, Mar 22, 2022 at 9:35 PM Jun Rao wrote:
>
> Hi, Artem,
>
> Thanks for the KIP. +1 from me.
>
> Jun
>
> On Mon, Mar 21, 2022 at 4:52 PM Artem Livshits
> wrote:
>
> > Hi all,
> >
> > I'd like to
Hi all,
Thanks Dejan for bringing this up. Relaxing this constraint seems
reasonable to me. I guess we would have to relax it for the keystores
at some point in the future as well (with Java 18).
Let's wait a few days to see what others think about this.
Best,
David
On Wed, Mar 23, 2022 at 8:46
Hi Sergio,
I made a suggestion a few weeks ago about the name about the parameter but
haven’t got a response for it. Did you consider it?
Do we need to update the rejected alternatives section to mention the
alternative options discussed in this thread?
Thanks,
David
Le ven. 25 mars 2022 à 03:4
201 - 300 of 1197 matches
Mail list logo