Re: [VOTE] KIP-801 new authorizer for kip-500 kraft mode

2022-02-01 Thread Jason Gustafson
+1 Thanks! On Mon, Jan 31, 2022 at 6:20 PM Colin McCabe wrote: > Hi all, > > It looks like people using gmail are seeing the previous vote thread as > merged with the discuss thread, so let me create a new thread in order to > avoid confusion. Usually using a very different thread title works we

Re: [DISCUSS] KIP-836: Addition of Information in DescribeQuorumResponse about Voter Lag

2022-05-10 Thread Jason Gustafson
> Hello Niket, currently DescribeQuorumResponse is not a public API, we don’t have a Admin api or shell script to get DescribeQuorumResponse, so it’s unnecessary to submit a KIP to change it, you can just submit a PR to accomplish this. Hey Ziming, I think it is public. It was documented in KIP-59

Re: [DISCUSS] KIP-835: Monitor KRaft Controller Quorum Health

2022-05-10 Thread Jason Gustafson
Hi Jose, Thanks for the KIP. The approach sounds reasonable. By the way, I think one of the gaps we have today is when the leader gets partitioned from the remaining voters. I believe it continues acting as a leader indefinitely. I was considering whether this periodic write can address the issue

Re: [VOTE] KIP-836: Addition of Information in DescribeQuorumResponse about Voter Lag

2022-05-17 Thread Jason Gustafson
+1 Thanks for the KIP! On Mon, May 16, 2022 at 10:14 AM Niket Goel wrote: > Hi all, > > I would like to start a vote for KIP-836: > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-836%3A+Addition+of+Information+in+DescribeQuorumResponse+about+Voter+Lag > > Thanks > Niket Goel >

Re: [VOTE] KIP-833: Mark KRaft as Production Ready

2022-05-23 Thread Jason Gustafson
Thanks Colin. +1 from me. Very exciting! On Tue, May 17, 2022 at 10:53 AM Colin McCabe wrote: > Hi, > > I'd like to start the vote on KIP-833: Mark KRaft as Production Ready. > https://cwiki.apache.org/confluence/x/8xKhD > > thanks, > Colin >

Re: [VOTE] KIP-714: Client Metrics and Observability

2022-05-24 Thread Jason Gustafson
+1 Thanks Magnus! On Tue, May 17, 2022 at 5:43 AM Magnus Edenhill wrote: > Hey all, > > It's that time of year again where we re-restart this vote thread after > some additional > discussions on the disco thread and minor adjustments&clarifications to the > KIP. > > We're currently at +5 (non-bi

Re: [DISCUSS] Apache Kafka 3.3.0 Release

2022-07-14 Thread Jason Gustafson
Hey Jose, Thanks for volunteering to manage the release! KIP-833 is currently slotted for 3.3. We've been getting some help from Jack Vanlighty to validate the raft implementation in TLA+ and with frameworks like Jepsen. The specification is written here if anyone is interested: https://github.com

Re: [DISCUSS] KIP-854 Separate configuration for producer ID expiry

2022-07-26 Thread Jason Gustafson
Hi Justine, Thanks for the KIP. Although I hate seeing new configurations, I think this is a good change. Combining these timeout behaviors into a single configuration was definitely a mistake, but we didn't anticipate the problem with the producer id cache. I do wonder if we can make the default

Re: [DISCUSS] KIP-854 Separate configuration for producer ID expiry

2022-07-27 Thread Jason Gustafson
gt; compatibility story -- especially since there is the chance for > producer > > > IDs to expire silently. > > > I do think that 1 day is fairly reasonable. If I don't hear any > > conflicting > > > opinions I can go ahead and update the default. > >

Re: [DISCUSS]: Including TLA+ in the repo

2022-07-28 Thread Jason Gustafson
Yeah, good idea. I'm happy to submit the specs I wrote for normal kafka replication. It will make them more accessible and I have long been looking for help reviewing them. Hopefully it will also provide a better chance to keep them in sync with the codebase as we update protocols. -Jason On Wed,

Re: [DISCUSS] KIP-854 Separate configuration for producer ID expiry

2022-08-02 Thread Jason Gustafson
setting the > > value that transactional.id.expiration.ms has? > > > > Let me know your thoughts. > > Thanks, > > Justine > > > > On Fri, Jul 29, 2022 at 10:54 AM Ismael Juma wrote: > > > > > +1 for having 1 day as the default and for includin

Re: [DISCUSS] KIP-858: Handle JBOD broker disk failure in KRaft

2022-08-08 Thread Jason Gustafson
Hi Igor, Thanks for the KIP. It looks like it's on a good track. I have a few suggestions to throw into the mix: 1. (nit): Instead of "storage id," maybe we could call it "directory id"? It seems a little clear since each log dir gets a unique id. 2. Rather than introducing a new RPC to communica

Re: [VOTE] KIP-854 Separate configuration for producer ID expiry

2022-08-09 Thread Jason Gustafson
Thanks Justine, +1 from me. On Tue, Aug 9, 2022 at 1:12 AM Sagar wrote: > Thanks for the KIP. > > +1(non-binding) > > Sagar. > > On Tue, Aug 9, 2022 at 1:13 PM David Jacot > wrote: > > > Thanks for the KIP, Justine. The proposal makes sense to me. I am +1 > > (binding). > > > > Cheers, > > Davi

Re: [VOTE] KIP-903: Replicas with stale broker epoch should not be allowed to join the ISR

2023-02-21 Thread Jason Gustafson
Hi Calvin, thanks for the KIP! A couple questions below: - Would it make sense to make the broker epoch a tagged field inside the Fetch request? It is a little annoying to increase the size of consumer fetch requests for a field that they will not use. Perhaps we could factor out broker id and bro

Re: [VOTE] KIP-903: Replicas with stale broker epoch should not be allowed to join the ISR

2023-02-22 Thread Jason Gustafson
Hi Calvin, The `BrokerState` struct I suggested would replace the `BrokerId` field in older versions. { "name": "ReplicaId", "type": "int32", "versions": "0-13", "entityType": "brokerId", "about": "The broker ID of the follower, of -1 if this request is from a consumer." }, { "name"

Re: [DISCUSS] KIP-909: Allow clients to rebootstrap DNS lookup failure

2023-02-28 Thread Jason Gustafson
Hi Philip, An alternative is not to fail at all. Every other network error is caught and handled internally in the client. We see this case as different because a DNS resolution error may imply misconfiguration. Could it also imply that the DNS server is unavailable? I'm not sure why that case sho

Re: [DISCUSS] KIP-909: Allow clients to rebootstrap DNS lookup failure

2023-02-28 Thread Jason Gustafson
to have a timeout which bounds the total time that the client should wait to connect to the bootstrap servers? Something like ` bootstrap.servers.connection.timeout.ms`. -Jason On Tue, Feb 28, 2023 at 10:44 AM Jason Gustafson wrote: > Hi Philip, > > An alternative is not to fail at all.

Re: [DISCUSS] KIP-909: Allow clients to rebootstrap DNS lookup failure

2023-02-28 Thread Jason Gustafson
e pre-flight check, > as it seems exposing an API might be harder to push through. > > Thanks! > P > > On Tue, Feb 28, 2023 at 10:53 AM Jason Gustafson > > wrote: > > > One more random thought I had just as I pushed send. We're currently > > treating this

Re: [VOTE] KIP-903: Replicas with stale broker epoch should not be allowed to join the ISR

2023-02-28 Thread Jason Gustafson
Thanks Calvin, +1 from me. On Mon, Feb 27, 2023 at 9:41 AM Calvin Liu wrote: > Hi Jason, > Updated the fields accordingly. Also, rename the BrokerState to > ReplicaState. > Thanks. > > On Wed, Feb 22, 2023 at 4:38 PM Jason Gustafson > > wrote: > > > Hi Calvin,

Re: [VOTE] KIP-915: Txn and Group Coordinator Downgrade Foundation

2023-03-30 Thread Jason Gustafson
+1. Thanks Jeff. On Thu, Mar 30, 2023 at 7:41 AM Alexandre Dupriez < alexandre.dupr...@gmail.com> wrote: > Thanks for the KIP and clarifications, Jeff. > > +1 (non binding) > > Le jeu. 30 mars 2023 à 14:41, David Jacot > a écrit : > > > > Thanks for the KIP, Jeff. > > > > +1 (binding) > > > > On

Re: [DISCUSS] KIP-909: Allow clients to rebootstrap DNS lookup failure

2023-04-19 Thread Jason Gustafson
Hey Phillip, The KIP looks good. 5 minutes seems like a reasonable tradeoff. I do wonder if it is necessary to treat bootstrap timeout as a fatal error though. It seems possible that the exception might be caught by handlers in existing applications which may not expect that the client needs to be

Re: [DISCUSS] KIP-909: Allow clients to rebootstrap DNS lookup failure

2023-04-20 Thread Jason Gustafson
aybe we can call this > *bootstrap.auto.retry.ms > <http://bootstrap.auto.retry.ms> *instead, to indicate a configurable > period of automatic retry. What do you think? > > Cheers, > P > > On Wed, Apr 19, 2023 at 7:17 PM Jason Gustafson > > wrote: > > > Hey Philli

Re: [DISCUSS] KIP-909: Allow clients to rebootstrap DNS lookup failure

2023-04-21 Thread Jason Gustafson
nd we can throw a fatal, BootstrapResolutionException (so > not connection exception anymore) afterward. > > I think that aligns with the goal of this KIP. > > P > > On Thu, Apr 20, 2023 at 9:23 AM Jason Gustafson > > wrote: > > > Hey Philip, > > > >

Re: KIP-919: Allow AdminClient to Talk Directly with the KRaft Controller Quorum

2023-04-21 Thread Jason Gustafson
Hey Colin, The KIP makes sense overall. Nice to clarify the contract between clients and the controllers. The use of `DirectToKRaftControllerQuorum` will help prevent misconfiguration. In fact, I wonder if we can return a fatal error instead of NOT_CONTROLLER so that the client would immediately f

Re: [VOTE] KIP-909: DNS Resolution Fallures Should Not Fail the Client

2023-04-25 Thread Jason Gustafson
+1 Thanks Philip! On Thu, Apr 13, 2023 at 7:49 AM Kirk True wrote: > +1 (non-binding) > > > On Apr 10, 2023, at 1:53 PM, Philip Nee wrote: > > > > Hey everyone! > > > > I'm starting a vote for KIP-909: DNS Resolution Fallures Should Not Fail > > the Client

Re: [DISCUSS] KIP-691: Transactional Producer Exception Handling

2021-01-27 Thread Jason Gustafson
Hi Boyang, Thanks for the iterations here. I think this is something we should have done a long time ago. It sounds like there are two API changes here: 1. We are introducing the `CommitFailedException` to wrap abortable errors that are raised from `commitTransaction`. This sounds fine to me. As

Re: [DISCUSS] KIP-691: Transactional Producer Exception Handling

2021-01-28 Thread Jason Gustafson
n for the thoughts. > > On Wed, Jan 27, 2021 at 11:52 AM Jason Gustafson > wrote: > > > Hi Boyang, > > > > Thanks for the iterations here. I think this is something we should have > > done a long time ago. It sounds like there are two API changes here: > > &

Re: [DISCUSS] KIP-691: Transactional Producer Exception Handling

2021-01-29 Thread Jason Gustafson
ll makes sense to use an umbrella exception in case > > in > > > > the future we add more non-fatal cases even though today we only > have a > > > > few. > > > > > > > > > > > > Guozhang > > > > > > > > On Thu,

Re: [DISCUSS] KIP-691: Transactional Producer Exception Handling

2021-01-29 Thread Jason Gustafson
; onCompletion(RecordMetadata metadata, SendFailure sendFailure) will become > the only API to be implemented. > > On Fri, Jan 29, 2021 at 3:42 PM Jason Gustafson > wrote: > > > Hi Boyang, > > > > Do you think it's necessary to deprecate the old `onCompletion

Re: [DISCUSS] KIP-706: Add method "Producer#produce" to return CompletionStage instead of Future

2021-01-29 Thread Jason Gustafson
Hi Chia-Ping, I think this is a great idea. It is a pity that we cannot continue to use the `send` verb, but I don't see how we can. I know we considered `transmit` as another option which is closer to `send`. That would avoid the redundancy when people choose the common "producer" variable name.

Re: [DISCUSS] KIP-706: Add method "Producer#produce" to return CompletionStage instead of Future

2021-01-30 Thread Jason Gustafson
ence/x/PSfZCQ). Hence, please feel free > to > > replace "produce" by "transmit" in KIP-691. > > > > Best, > > Chia-Ping > > > > On 2021/01/30 00:48:38, Jason Gustafson wrote: > > > Hi Chia-Ping, > > > > > > I think

Re: [DISCUSS] KIP-706: Add method "Producer#produce" to return CompletionStage instead of Future

2021-01-30 Thread Jason Gustafson
); CompletionStage send(TopicPartition topicPartition, Message message); -Jason On Sat, Jan 30, 2021 at 11:30 AM Jason Gustafson wrote: > I think this still makes sense as a separate KIP. For KIP-691, we are just > looking to help define the error contract for the new API. > > -Jason > &

Re: [DISCUSS] KIP-706: Add method "Producer#produce" to return CompletionStage instead of Future

2021-01-30 Thread Jason Gustafson
? I think > that's what .NET does. > 2. If we eventually allow a send to a topic id instead of topic name, would > that result in two additional overloads? > > Ismael > > On Sat, Jan 30, 2021 at 11:38 AM Jason Gustafson > wrote: > > > For the sake of having ano

Re: [DISCUSS] KIP-706: Add method "Producer#produce" to return CompletionStage instead of Future

2021-01-30 Thread Jason Gustafson
cepts `Uuid`, or we could add a `TopicId` type which implements `SendTarget`. -Jason On Sat, Jan 30, 2021 at 1:11 PM Jason Gustafson wrote: > Yeah, good question. I guess we always tend to regret using lower-level > types in these APIs. Perhaps there should be some kind of interface: &g

Re: [DISCUSS] KIP-618: Atomic commit of source connector records and offsets

2021-02-01 Thread Jason Gustafson
Hi Chris, If we add the new `fenceProducers` admin API, can we return the information from the `InitProducerId` response (i.e. producer id and epoch)? We may not have a use case for it yet, but I don't see any harm exposing it for the future. For example, we could allow this state to be provided t

Re: [DISCUSS] KIP-691: Transactional Producer Exception Handling

2021-02-02 Thread Jason Gustafson
gt; > > a richer return type. Let me know if you are good with this, and > whether > > > Chia-Ping is also happy here :) > > > > > > Best, > > > Boyang > > > > > > On Fri, Jan 29, 2021 at 4:23 PM Jason Gustafson > > wrote: > > > >

[DISCUSS] KIP-735: Increase default consumer session timeout

2021-04-23 Thread Jason Gustafson
Hi All, I've created a KIP to change the default consumer session timeout as well as some related behavior: https://cwiki.apache.org/confluence/display/KAFKA/KIP-735%3A+Increase+default+consumer+session+timeout. Have a look and let me know what you think. -Jason

Re: [DISCUSS] KIP-735: Increase default consumer session timeout

2021-04-26 Thread Jason Gustafson
t; On Sat, 24 Apr 2021 at 05:58, Ismael Juma wrote: > > > > Thanks for the KIP, +1. > > > > Ismael > > > > On Fri, Apr 23, 2021 at 9:29 AM Jason Gustafson > > > wrote: > > > > > Hi All, > > > > > > I've

Re: [DISCUSS] KIP-734: Improve AdminClient.listOffsets to return timestamp and offset for the record with the largest timestamp

2021-04-27 Thread Jason Gustafson
Hi Tom, I had the same question as David. It sounds like this will require a bump to the `ListOffsets` API? Otherwise, looking at the code, the sentinel would be interpreted as a timestamp query and it looks like it would end up returning the smallest timestamp ;). A second thought I had is wheth

Re: [DISCUSS] KIP-735: Increase default consumer session timeout

2021-04-27 Thread Jason Gustafson
d > > On Tue, Apr 27, 2021 at 2:52 AM Satish Duggana > wrote: > > > Tthanks Jason for the clarification. > > > > On Tue, 27 Apr 2021 at 05:42, Jason Gustafson > > wrote: > > > > > > Hi Satish, > > > > > > Typically we update

[VOTE] KIP-735: Increase default consumer session timeout

2021-04-28 Thread Jason Gustafson
Hi All, I'd like to start a vote on KIP-735: https://cwiki.apache.org/confluence/display/KAFKA/KIP-735%3A+Increase+default+consumer+session+timeout. +1 from myself obviously -Jason

Re: [VOTE] KIP-735: Increase default consumer session timeout

2021-04-29 Thread Jason Gustafson
> > >>> -Bill > >>> > >>> On Wed, Apr 28, 2021 at 7:47 PM Guozhang Wang > >> wrote: > >>> > >>>> +1. Thanks Jason! > >>>> > >>>> On Wed, Apr 28, 2021 at 12:50 PM Gwen Sh

Re: [VOTE] KIP-735: Increase default consumer session timeout

2021-05-06 Thread Jason Gustafson
Calling the vote here. Thanks everyone! The total is +8 with no -1 votes. The binding total is +7 with votes from myself, Sophie, Gwen, Guozhang, Bill, David, and Bruno. -Jason On Thu, Apr 29, 2021 at 10:05 AM Jason Gustafson wrote: > Hey All, > > Thanks everyone for the votes. >

Re: [VOTE] KIP-334 - Include partitions in exceptions raised during consumer record deserialization/validation

2021-06-02 Thread Jason Gustafson
+1 overall. There is one complication which I think we may have to address in the implementation. The KIP documents an override to `fillInStackTrace`. Is that necessary? The trace would indeed be useful in this case because the new exception will wrap the exception raised from the Deserializer. Unf

Re: [DISCUSS] KIP-724: Drop support for message formats v0 and v1

2021-06-04 Thread Jason Gustafson
Hi Ismael, I agree it would be awesome to drop support for the old formats! A few comments below: 1. Suppose that a partition with 3 replicas begins at v0. One broker is upgraded to v2 with the new "force upgrade" config as part of a rolling restart, which leaves two replicas on v0 and one on v2.

Re: [ANNOUNCE] New committer: Rajini Sivaram

2017-04-24 Thread Jason Gustafson
Woohoo! Great work, Rajini! On Mon, Apr 24, 2017 at 3:27 PM, Jun Rao wrote: > Congratulations, Rajini ! Thanks for all your contributions. > > Jun > > On Mon, Apr 24, 2017 at 2:06 PM, Gwen Shapira wrote: > > > The PMC for Apache Kafka has invited Rajini Sivaram as a committer and we > > are ple

Re: [VOTE] KIP-126 - Allow KafkaProducer to split and resend oversized batches.

2017-05-08 Thread Jason Gustafson
+1 On Thu, May 4, 2017 at 9:11 PM, Michael Pearce wrote: > +1 > > > > From: Joel Koshy > Sent: Friday, May 5, 2017 4:32:42 AM > To: dev@kafka.apache.org > Subject: Re: [VOTE] KIP-126 - Allow KafkaProducer to split and resend > oversized batches. > > +1 >

Re: [VOTE] KIP-140: Add administrative RPCs for adding, deleting, and listing ACLs

2017-05-10 Thread Jason Gustafson
Hi Colin, Thanks for the KIP. Looks good overall. One thing I wasn't too clear about is whether new resource types, operations, or permissions require a version bump for the three new request types. On reading the proposal, it sort of sounds like the intent is not to bump the versions and let the

Re: [DISCUSS] KIP-133: List and Alter Configs Admin APIs

2017-05-10 Thread Jason Gustafson
Hey Ismael, Thanks for the KIP. The use of the Describe API might be a little limited if it always returns the full set of topics for the requested resource. I wonder if we can let the client provide a list of the configs they are interested in. Perhaps something like this: DescribeConfigs Reques

Re: [VOTE] KIP-140: Add administrative RPCs for adding, deleting, and listing ACLs

2017-05-10 Thread Jason Gustafson
ient simply won't have any idea about it). But we are > doing what we can for compatibility. > > cheers, > Colin > > > On Wed, May 10, 2017, at 12:26, Jason Gustafson wrote: > > Hi Colin, > > > > Thanks for the KIP. Looks good overall. One thing I wasn'

Re: [VOTE] KIP-133: List and Alter Configs Admin APIs (second attempt)

2017-05-10 Thread Jason Gustafson
+1 Thanks for the KIP. On Wed, May 10, 2017 at 8:55 AM, Jun Rao wrote: > Hi, Ismael, > > Thanks for the KIP. Looks good overall. A couple of minor comments. > > 1. Currently, quotas can be updated at the combination > level. So, it seems that we need to reflect that somehow in both the wire > p

Re: [VOTE] KIP-140: Add administrative RPCs for adding, deleting, and listing ACLs

2017-05-10 Thread Jason Gustafson
10, 2017 at 4:41 PM, Colin McCabe wrote: > On Wed, May 10, 2017, at 15:54, Jason Gustafson wrote: > > Hey Colin, > > > > Thanks, I think continuing to bump the protocol makes sense. It's nice to > > be consistent with the other APIs. In the KIP, you have the followin

Re: [VOTE] KIP-140: A new thread for adding administrative RPCs for adding deleting and listing ACLs

2017-05-10 Thread Jason Gustafson
+1 On Wed, May 10, 2017 at 3:01 PM, Ismael Juma wrote: > Repeating my vote here, +1 (binding). > > Ismael > > On Wed, May 10, 2017 at 9:24 PM, Colin McCabe wrote: > > > Hi all, > > > > Some folks said that the previous VOTE thread was getting collapsed by > > gmail into a different thread, so I

Re: [VOTE] KIP-155 Add range scan for windowed state stores

2017-05-10 Thread Jason Gustafson
+1 On Wed, May 10, 2017 at 1:45 PM, Xavier Léauté wrote: > Thank you for the feedback Michal. > > While I agree the return may be a little bit more confusing to reason > about, the reason for doing so was to keep the range query interfaces > consistent with their single-key counterparts. > > In

Re: Reg: [VOTE] KIP 157 - Add consumer config options to streams reset tool

2017-06-02 Thread Jason Gustafson
Thanks. +1 On Thu, Jun 1, 2017 at 9:40 PM, Matthias J. Sax wrote: > +1 > > Thanks for updating the KIP! > > -Matthias > > On 6/1/17 6:18 PM, Bill Bejeck wrote: > > +1 > > > > Thanks, > > Bill > > > > On Thu, Jun 1, 2017 at 7:45 PM, Guozhang Wang > wrote: > > > >> +1 again. Thanks. > >> > >> On

Re: [VOTE] KIP-746: Revise KRaft Metadata Records

2021-06-09 Thread Jason Gustafson
+1 Thanks Colin! On Thu, Jun 3, 2021 at 4:30 PM Colin McCabe wrote: > Hi all, > > I'd like to call a vote for KIP-746: Revise KRaft Metadata Records. This > is a minor KIP which revises the KRaft metadata records slightly for the > upcoming 3.0 release. > > The KIP is at: https://cwiki.apache.or

Re: [VOTE] KIP-724: Drop support for message formats v0 and v1

2021-06-09 Thread Jason Gustafson
+1 Thanks Ismael! On Wed, Jun 9, 2021 at 11:29 AM Ismael Juma wrote: > Hi all, > > Consensus was reached in the discussion thread and part of what is proposed > has to happen by 3.0, so starting the vote for KIP-724: > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-724%3A+Drop+support

Re: [VOTE] KIP-748: Add Broker Count Metrics (restarted)

2021-08-24 Thread Jason Gustafson
Thanks Colin! +1 On Mon, Aug 23, 2021 at 1:37 PM David Arthur wrote: > Thanks, Colin. Looks good to me! > > +1 binding > > -David > > On Mon, Aug 16, 2021 at 3:37 PM David Jacot wrote: > > > Hi Colin, > > > > Thanks for restarting this KIP. > > > > +1 (binding) > > > > Best, > > David > > > > L

Re: [VOTE]: KIP-604: Remove ZooKeeper Flags from the Administrative Tools

2021-11-10 Thread Jason Gustafson
ason, and > Jason Gustafson, and 1 non-binding vote from David Jacot, the vote passes. > > thanks, all. > Colin > > > On Wed, May 20, 2020, at 18:16, Jason Gustafson wrote: > > Sounds good. +1 from me. > > > > On Tue, May 19, 2020 at 5:41 PM Colin McCabe wrote:

Re: [DISCUSS] KIP-778 KRaft Upgrades

2021-11-17 Thread Jason Gustafson
Hi David, Forgive me if this ground has been covered already. Today, we have a few other things that we have latched onto the IBP, such as upgrades to the format of records in __consumer_offsets. I've been assuming that metadata.version is not covering this. Is that right or is there some other pl

Re: [DISCUSS] KIP-778 KRaft Upgrades

2021-11-17 Thread Jason Gustafson
PM Colin McCabe wrote: > On Wed, Nov 17, 2021, at 11:28, Jason Gustafson wrote: > > Hi David, > > > > Forgive me if this ground has been covered already. Today, we have a few > > other things that we have latched onto the IBP, such as upgrades to the > > format of

Re: [DISCUSS] KIP-778 KRaft Upgrades

2021-11-17 Thread Jason Gustafson
process more convenient. Thanks, Jason On Wed, Nov 17, 2021 at 6:20 PM Jason Gustafson wrote: > A few additional questions: > > 1. Currently the IBP tells us what version of individual inter-broker RPCs > will be used. I think the plan in this KIP is to use ApiVersions request >

Re: [DISCUSS] Apache Kafka 3.1.0 release

2021-12-13 Thread Jason Gustafson
Hi David, I think we should get https://issues.apache.org/jira/browse/KAFKA-13488. This is a follow-up to https://issues.apache.org/jira/browse/KAFKA-12257, which was previously considered a 3.0 blocker. Without the additional patch, the bug causing the consumer to get stuck can still occur in a c

Re: Kafka New Consumer Performance Test ThroughPut Degradation

2015-09-02 Thread Jason Gustafson
Hi Poorna, We've been looking at the same issue as part of KAFKA-2489. I suspect the initial rebalance is causing the test to timeout, but we're still investigating. Thanks, Jason On Wed, Sep 2, 2015 at 2:03 PM, Poorna Chandra Tejashvi Reddy < pctre...@gmail.com> wrote: > Hi, > > I have checked

Re: New consumer subscribe then seek

2015-09-08 Thread Jason Gustafson
Hey Phil, You've stumbled onto one of the tricky aspects of the new consumer that we've been talking about recently. KafkaConsumer.subscribe() is asynchronous in the sense that it will return before partitions have been assigned. We could make it synchronous, but we wouldn't be able to guarantee h

Re: Review Request 36858: Patch for KAFKA-2120

2015-09-10 Thread Jason Gustafson
lay timeout detection. Maybe I'm missing something? - Jason Gustafson On Sept. 10, 2015, 1:56 a.m., Mayuresh Gharat wrote: > > --- > This is an automatically generated e-mail. To reply, visit: > https:

Re: Review Request 36858: Patch for KAFKA-2120

2015-09-11 Thread Jason Gustafson
> On Sept. 11, 2015, 1:44 a.m., Jason Gustafson wrote: > > clients/src/main/java/org/apache/kafka/clients/InFlightRequests.java, line > > 140 > > <https://reviews.apache.org/r/36858/diff/9/?file=1066895#file1066895line140> > > > > This line is puzzli

Re: Review Request 36858: Patch for KAFKA-2120

2015-09-14 Thread Jason Gustafson
> On Sept. 11, 2015, 1:44 a.m., Jason Gustafson wrote: > > clients/src/main/java/org/apache/kafka/clients/InFlightRequests.java, line > > 140 > > <https://reviews.apache.org/r/36858/diff/9/?file=1066895#file1066895line140> > > > > This line is puzzli

Re: [VOTE] KIP-28 - Add a processor client for stream data processing

2015-09-22 Thread Jason Gustafson
+1 (non-binding) On Mon, Sep 21, 2015 at 11:39 PM, Ewen Cheslack-Postava wrote: > +1 (non-binding) > > On Mon, Sep 21, 2015 at 6:57 PM, Jay Kreps wrote: > > > +1 > > > > -Jay > > > > On Mon, Sep 21, 2015 at 6:28 PM, Guozhang Wang > wrote: > > > > > Hello all, > > > > > > I would like to start

Re: [DISCUSS] KIP-35 - Retrieve protocol version

2015-09-28 Thread Jason Gustafson
Having the version API can make clients more robust, so I'm in favor. One note on the addition of the "rack" field. Since this is a broker-specific setting, the client would have to query BrokerMetadata for every new broker it connects to (unless we also expose rack in TopicMetadata). This is also

Re: [DISCUSS] KIP-35 - Retrieve protocol version

2015-10-06 Thread Jason Gustafson
g an empty response > back > > to > > >>> the > > >>> > > > client, would it be better for the broker to return a response > > that > > >>> > gives > > >>> > > > some more info to the client regarding the min vers

Re: [DISCUSS] KIP-37 - Add namespaces in Kafka

2015-10-14 Thread Jason Gustafson
Hey Ashish, thanks for the write-up. I think having a namespace capability is a useful feature for Kafka, in particular with the addition of the authorization layer. I probably prefer Jay's hierarchical approach if we're going to embed the namespace in the topic name since it seems more general. Th

[DISCUSS] KIP-40 - Extend GroupMetadata request to return group and member status

2015-10-25 Thread Jason Gustafson
Hi Kafka Devs, Currently, the new consumer provides no way to view a group's status except by inspecting coordinator and consumer logs. This includes listing the members of the group and their partition assignments. For the old consumer, tools could read this information directly from Zookeeper, b

Re: [DISCUSS] KIP-40 - Extend GroupMetadata request to return group and member status

2015-10-28 Thread Jason Gustafson
, 2015 at 12:59 PM, Gwen Shapira wrote: > > > Looks awesome to me :) > > > > This will allow to both list all groups and to retrieve offsets for > > specific groups. > > > > Since 3 days passed with no comments, would you like to start a vote? > > > >

Re: [DISCUSS] KIP-40 - Extend GroupMetadata request to return group and member status

2015-10-28 Thread Jason Gustafson
l allow to both list all groups and to retrieve offsets for > > > specific groups. > > > > > > Since 3 days passed with no comments, would you like to start a vote? > > > > > > On Sun, Oct 25, 2015 at 6:29 PM, Jason Gustafson > > > wrote: > > > &g

Re: [DISCUSS] KIP-40 - Extend GroupMetadata request to return group and member status

2015-10-28 Thread Jason Gustafson
empty list to represent "all" > can potentially generate a large response if there are many groups. > > Since this is marked as an 0.9.0 blocker, it would be great if other people > can review this KIP soon. > > Thanks, > > Jun > > > On Wed, Oct 28, 201

Re: [DISCUSS] KIP-40 - Extend GroupMetadata request to return group and member status

2015-10-29 Thread Jason Gustafson
Thanks, Jason On Thu, Oct 29, 2015 at 8:37 AM, Neha Narkhede wrote: > I like Jun's suggestion of adding ListGroups and DescribeGroup. It keeps > the abstractions simple and avoids overloading existing requests. > > On Wed, Oct 28, 2015 at 8:08 PM, Jason Gustafson > wrote

Re: [DISCUSS] KIP-40 - Extend GroupMetadata request to return group and member status

2015-10-29 Thread Jason Gustafson
Forgot to include a link to the changed KIP: https://cwiki.apache.org/confluence/display/KAFKA/KIP-40%3A+ListGroups+and+DescribeGroup -Jason On Thu, Oct 29, 2015 at 9:43 AM, Jason Gustafson wrote: > Hi All, > > I've updated the KIP for ListGroups and DescribeGroup. This does sim

[VOTE] KIP-40 ListGroups and DescribeGroup

2015-10-29 Thread Jason Gustafson
Since we're crunching a little on the 0.9 release, I'd like to begin voting for KIP-40. If you agree overall, but have minor objections with the schemas, we can discuss them on the JIRA. Summary: Add ListGroups and DescribeGroup APIs for admin tooling to view consumer groups and status. KIP: https

Re: 0.9.0 release branch

2015-11-02 Thread Jason Gustafson
I added KAFKA-2691 as well, which improves client handling of authorization errors. -Jason On Mon, Nov 2, 2015 at 10:25 AM, Becket Qin wrote: > Hi Jun, > > I added KAFKA-2722 as a blocker for 0.9. It fixes the ISR propagation > scalability issue we saw. > > Thanks, > > Jiangjie (Becket) Qin > >

Re: group protocol/metadata documentation

2015-11-24 Thread Jason Gustafson
Hey Dana, Have a look at this wiki, which has more detail on the consumer's embedded protocol: https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Client-side+Assignment+Proposal . At the moment, the group protocol supports consumer groups and kafka connect groups. Kafka tooling currently dep

Re: group protocol/metadata documentation

2015-11-27 Thread Jason Gustafson
userdata for something > more detailed in the future, like rack-aware placements? > > One other question: in what circumstances would consumer processes in a > single group want to use different topic subscriptions rather than > configure a new group? > > Thanks again, > > -

Re: partitionsFor method doesn't return latest partition metadata

2015-11-30 Thread Jason Gustafson
Hey Siyuan, Are you also consuming from the topics you are querying with partitionsFor? I'm guessing the problem is that we're pulling the metadata from a local cache, and it's not refreshed as quick as you're expecting (the default is 5 minutes). We only use the cache for topics which have partit

Re: [DISCUSS] KIP-631: The Quorum-based Kafka Controller

2020-09-29 Thread Jason Gustafson
Hey Colin, Thanks for the hard work on this proposal. I'm gradually coming over to the idea of the controllers having separate IDs. One of the benefits is that it allows us to separate the notion of controller liveness from broker liveness, which has always been a tricky detail. I think it's fair

Re: [VOTE] KIP-630: Kafka Raft Snapshot

2020-10-05 Thread Jason Gustafson
+1 Thanks for the KIP! -Jason On Mon, Oct 5, 2020 at 9:03 AM Ron Dagostino wrote: > Thanks for the KIP, Jose. +1 (non-binding) from me. > > I do have one comment/confusion. > > Upon re-reading the latest version, I am confused about the name of > the proposed "metadata.snapshot.min.records" co

Re: [VOTE] KIP-673: Emit JSONs with new auto-generated schema

2020-10-05 Thread Jason Gustafson
+1 Thanks for the KIP! On Fri, Oct 2, 2020 at 4:37 PM Colin McCabe wrote: > Thanks, Anastasia! This will be a lot easier to maintain. > > +1 (binding) > > best, > Colin > > On Wed, Sep 30, 2020, at 23:57, David Jacot wrote: > > Thanks for the KIP, Anastasia. > > > > +1 (non-binding) > > > > On

Re: [VOTE] 2.7.0 RC3

2020-12-02 Thread Jason Gustafson
Hi Bill, I found a potential blocker here: https://issues.apache.org/jira/browse/KAFKA-10799. A patch should be along shortly. Best, Jason On Mon, Nov 30, 2020 at 7:27 AM Bill Bejeck wrote: > Thanks for the vote, Gwen. > > Here's an update for Jenkins build > > * Successful Jenkins builds fo

Re: [VOTE] voting on KIP-631: the quorum-based Kafka controller

2020-12-16 Thread Jason Gustafson
+1 Thanks Colin for all the iterations. My only request is to change "controller.connect" to "controller.quorum.voters." I think it's important to emphasize that this must be the full set of voters unlike "zookeeper.connect." In the future, I think we can consider supporting an additional config li

Re: [VOTE] KIP-695: Improve Streams Time Synchronization

2020-12-16 Thread Jason Gustafson
Hi John, Just one question. It wasn't very clear to me exactly when the metadata would be returned in `ConsumerRecords`. Would we /always/ include the metadata for all partitions that are assigned, or would it be based on the latest fetches? Thanks, Jason On Fri, Dec 11, 2020 at 4:07 PM John Roe

Re: [DISCUSS] KIP-687: Automatic Reloading of Security Store

2021-01-06 Thread Jason Gustafson
Thanks Boyang. Someone mentioned my email never showed up, but basically I suggested tying the refresh configuration more directly to the configurations it would affect. I'm happy with the updates. -Jason On Tue, Jan 5, 2021 at 8:34 PM Boyang Chen wrote: > Thanks Jason for the feedback. I separ

Re: [VOTE] KIP-687: Automatic Reloading of Security Store

2021-01-06 Thread Jason Gustafson
+1 Thank Boyang On Wed, Jan 6, 2021 at 12:27 PM Boyang Chen wrote: > Hey folks, > > just bumping up this thread and see if you have further comments. > > On Wed, Dec 16, 2020 at 3:25 PM Boyang Chen > wrote: > > > Hey there, > > > > I would like to start the voting for KIP-687: > > https://cwiki

[jira] [Resolved] (KAFKA-4514) Add Codec for ZStandard Compression

2018-10-09 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4514?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-4514. Resolution: Fixed Fix Version/s: 2.1.0 > Add Codec for ZStandard Compress

[jira] [Created] (KAFKA-7567) Clean up internal metadata usage for consistency and extensibility

2018-10-29 Thread Jason Gustafson (JIRA)
Jason Gustafson created KAFKA-7567: -- Summary: Clean up internal metadata usage for consistency and extensibility Key: KAFKA-7567 URL: https://issues.apache.org/jira/browse/KAFKA-7567 Project: Kafka

[jira] [Created] (KAFKA-7568) Return leader epoch in ListOffsets responses

2018-10-29 Thread Jason Gustafson (JIRA)
Jason Gustafson created KAFKA-7568: -- Summary: Return leader epoch in ListOffsets responses Key: KAFKA-7568 URL: https://issues.apache.org/jira/browse/KAFKA-7568 Project: Kafka Issue Type

[jira] [Created] (KAFKA-7570) Make internal offsets/transaction schemas forward compatible

2018-10-30 Thread Jason Gustafson (JIRA)
Jason Gustafson created KAFKA-7570: -- Summary: Make internal offsets/transaction schemas forward compatible Key: KAFKA-7570 URL: https://issues.apache.org/jira/browse/KAFKA-7570 Project: Kafka

[jira] [Created] (KAFKA-7571) Add system tests for downgrading Kafka

2018-10-30 Thread Jason Gustafson (JIRA)
Jason Gustafson created KAFKA-7571: -- Summary: Add system tests for downgrading Kafka Key: KAFKA-7571 URL: https://issues.apache.org/jira/browse/KAFKA-7571 Project: Kafka Issue Type

[jira] [Resolved] (KAFKA-7567) Clean up internal metadata usage for consistency and extensibility

2018-10-30 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7567?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-7567. Resolution: Fixed Fix Version/s: 2.2.0 > Clean up internal metadata usage

[jira] [Resolved] (KAFKA-7568) Return leader epoch in ListOffsets responses

2018-11-01 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7568?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-7568. Resolution: Fixed Fix Version/s: 2.2.0 > Return leader epoch in ListOffs

[jira] [Created] (KAFKA-7601) Handle message format downgrades during upgrade of message format version

2018-11-06 Thread Jason Gustafson (JIRA)
Jason Gustafson created KAFKA-7601: -- Summary: Handle message format downgrades during upgrade of message format version Key: KAFKA-7601 URL: https://issues.apache.org/jira/browse/KAFKA-7601 Project

[jira] [Created] (KAFKA-7604) Flaky Test `ConsumerCoordinatorTest.testRebalanceAfterTopicUnavailableWithPatternSubscribe`

2018-11-07 Thread Jason Gustafson (JIRA)
Jason Gustafson created KAFKA-7604: -- Summary: Flaky Test `ConsumerCoordinatorTest.testRebalanceAfterTopicUnavailableWithPatternSubscribe` Key: KAFKA-7604 URL: https://issues.apache.org/jira/browse/KAFKA-7604

<    5   6   7   8   9   10   11   12   13   14   >