Re: [DISCUSS] KIP-1038: Add Custom Error Handler to Producer

2024-05-09 Thread Artem Livshits
t; > To me, a handler like this, is to some extend "business logic" and if a > > > user gets business logic wrong, it's hard to protect them. -- We would > > > of course provide best practice guidance in the JaveDocs, and explain > > > that a handler should

Re: [DISCUSS] KIP-1038: Add Custom Error Handler to Producer

2024-05-07 Thread Artem Livshits
Hi Alieh, Thanks for the KIP. The motivation talks about very specific cases, but the interface is generic. [AL1] If the interface evolves in the future I think we could have the following confusion: 1. A user implemented SWALLOW action for both RecordTooLargeException and

[jira] [Resolved] (KAFKA-16352) Transaction may get get stuck in PrepareCommit or PrepareAbort state

2024-03-20 Thread Artem Livshits (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16352?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Artem Livshits resolved KAFKA-16352. Fix Version/s: 3.8.0 Reviewer: Justine Olshan Resolution: Fixed

Re: [DISCUSS] KIP-1022 Formatting and Updating Features

2024-03-08 Thread Artem Livshits
Hi Justine, > Are you suggesting it should be called "transaction protocol version" or "TPV"? I don't mind that, but just wanted to clarify if we want to include protocol or if simply "transaction version" is enough. My understanding is that "metadata version" is the version of metadata

[jira] [Created] (KAFKA-16352) Transaction may get get stuck in PrepareCommit or PrepareAbort state

2024-03-07 Thread Artem Livshits (Jira)
Artem Livshits created KAFKA-16352: -- Summary: Transaction may get get stuck in PrepareCommit or PrepareAbort state Key: KAFKA-16352 URL: https://issues.apache.org/jira/browse/KAFKA-16352 Project

Re: [DISCUSS] KIP-939: Support Participation in 2PC

2024-03-01 Thread Artem Livshits
Hi Jun, > 32. ... metric name ... I've updated the metric name to be *kafka.coordinator.transaction:type=TransactionStateManager,name=ActiveTransactionOpenTimeMax.* Let me know if it works. -Artem On Thu, Feb 29, 2024 at 12:03 PM Artem Livshits wrote: > Hi Jun, > > >

Re: [DISCUSS] KIP-939: Support Participation in 2PC

2024-02-29 Thread Artem Livshits
f we provide a new functionality, we should make it easy such that > the application developer only needs to implement it in one way, which is > always correct. Then, we can consider what additional things are needed to > make the operator comfortable enabling it. > > Jun > > O

Re: [DISCUSS] KIP-1022 Formatting and Updating Features

2024-02-28 Thread Artem Livshits
Hi Justine, Thank you for the KIP. I think the KIP is pretty clear and makes sense to me. Maybe it would be good to give a little more detail on the implementation of feature mapping and how the tool would validate the feature combinations. For example, I'd expect that bin/kafka-storage.sh

Re: [DISCUSS] KIP-939: Support Participation in 2PC

2024-02-27 Thread Artem Livshits
> applications' transactions to hang. > > Jun > > On Wed, Feb 21, 2024 at 4:34 PM Artem Livshits > wrote: > > > Hi Jun, > > > > > 20A. One option is to make the API initTransactions(boolean enable2PC). > > > > We could do that. I think there

Re: [DISCUSS] KIP-939: Support Participation in 2PC

2024-02-21 Thread Artem Livshits
> none The max time a currently-open transaction has been open > > Jun > > On Wed, Feb 21, 2024 at 11:25 AM Artem Livshits > wrote: > > > Hi Jun, > > > > > 20A. This only takes care of the abort case. The application still > needs > > to be chang

Re: [DISCUSS] KIP-939: Support Participation in 2PC

2024-02-21 Thread Artem Livshits
s can choose whether they want a > prepare RPC when not using the XA interface. How does that sound? > > - Rowland > > On Fri, Feb 16, 2024 at 7:15 PM Artem Livshits > wrote: > > > Hi Rowland, > > > > > I am not sure what you mean by guarantee, > > &

Re: [DISCUSS] KIP-939: Support Participation in 2PC

2024-02-21 Thread Artem Livshits
2. Ok. That's the kafka metric. In that case, the metric name has a group > and a name. There is no type and no package name. > > Jun > > > On Thu, Feb 15, 2024 at 8:23 PM Artem Livshits > wrote: > > > Hi Jun, > > > > Thank you for your questions. > > >

Re: [DISCUSS] KIP-939: Support Participation in 2PC

2024-02-16 Thread Artem Livshits
t; unblock affected partitions without administrative intervention or fast > restart of the client would be a worthwhile benefit. An explicit "prepare" > RPC will also be needed by the XA implementation, so I would like to see it > implemented for that reason. Otherwise, I will need

Re: [DISCUSS] KIP-939: Support Participation in 2PC

2024-02-15 Thread Artem Livshits
the type > attribute. > > 33. "If the value is 'true' then the corresponding field is set in the > InitProducerIdRequest and the KafkaProducer object is set into a state > which only allows calling .commitTransaction or .abortTransaction." > We should also allow .completeTransaction, rig

Re: [DISCUSS] KIP-939: Support Participation in 2PC

2024-02-06 Thread Artem Livshits
ion could be called. If the call is successful (all > messages successfully got flushed to all partitions) the transaction is > prepared." > If the producer calls send() in that state, what exception will the caller > receive? > > Jun > > > On Fri, Feb 2, 2024 at 3:

Re: [DISCUSS] KIP-939: Support Participation in 2PC

2024-02-05 Thread Artem Livshits
. However, the XA layer is going to > require certain capabilities from the layer below it, and one of those > capabilities is to be able to identify and report prepared transactions > during recovery. > > - Rowland > > On Mon, Feb 5, 2024 at 12:46 AM Artem Livshits > wrote: &

Re: [DISCUSS] KIP-939: Support Participation in 2PC

2024-02-04 Thread Artem Livshits
h, and that some users would prefer to have > speed in exchange for less system availability in some cases of client or > network failure. > > Let me know what you think. > > -Rowland > > On Fri, Jan 5, 2024 at 8:03 PM Artem Livshits > wrote: > > > Hi Rowland

Re: [DISCUSS] KIP-939: Support Participation in 2PC

2024-02-02 Thread Artem Livshits
est to use the next pid. So, the flow is different, right? > > 25. "We send out markers using the original ongoing transaction ProducerId > and ProducerEpoch" > We should use ProducerEpoch + 1 in the marker, right? > > Jun > > On Fri, Jan 26, 2024 at 8:35 PM Artem Livshits

Re: [DISCUSS] KIP-939: Support Participation in 2PC

2024-01-26 Thread Artem Livshits
ople implementing this new > client protocol to figure out when to use the current or the new producerId > in the EndTxnRequest. One potential way to improve this is to extend > EndTxnRequest with a new field like expectedNextProducerId. Then we can > always use the old produceId in the

Re: [DISCUSS] KIP-890 Server Side Defense

2024-01-22 Thread Artem Livshits
> Hmm -- we would fence the producer if the epoch is bumped and we get a lower epoch. Yes -- we are intentionally adding this to prevent fencing. I think Jun's point is that we can defer the fencing decision until transition into complete state (which I believe is what the current logic is

Re: [DISCUSS] KIP-1014: Managing Unstable Metadata Versions in Apache Kafka

2024-01-19 Thread Artem Livshits
n Fri, Jan 12, 2024, at 11:32, Artem Livshits wrote: > > I think using feature flags (whether we support a framework and tooling > for > > feature flags or just an ad-hoc XyzEnabled flag) can be an alternative to > > this KIP. I think the value of this KIP is that it's tr

Re: [DISCUSS] KIP-1014: Managing Unstable Metadata Versions in Apache Kafka

2024-01-12 Thread Artem Livshits
I think using feature flags (whether we support a framework and tooling for feature flags or just an ad-hoc XyzEnabled flag) can be an alternative to this KIP. I think the value of this KIP is that it's trying to propose a systemic approach for gating functionality that may take multiple releases

Re: [DISCUSS] KIP-1014: Managing Unstable Metadata Versions in Apache Kafka

2024-01-11 Thread Artem Livshits
Hi Proven, I'd say that we should do 2 & 2. The idea is that for small features that can be done and stabilized within a short period of time (with one or very few commits) that's exactly what happens -- people interested in testing in-progress feature could take unstable code from a patch (or

Re: [DISCUSS] KIP-939: Support Participation in 2PC

2024-01-05 Thread Artem Livshits
Hi Rowland, Thank you for the feedback. For the 2PC cases, the expectation is that the timeout on the client would be set to "effectively infinite", that would exceed all practical 2PC delays. Now I think that this flexibility is confusing and can be misused, I have updated the KIP to just say

Re: [DISCUSS] Kafka distributed transaction support

2024-01-03 Thread Artem Livshits
Hi Rowland, KIP-939 provides a foundation for using a two-phase commit protocol with Kafka (allows it to be a participant) that can be used to implement various concrete protocols, such as XA but not only XA. The benefit of supporting a foundational construct (and not just one concrete protocol

Re: [DISCUSS] KIP-939: Support Participation in 2PC

2023-12-18 Thread Artem Livshits
will need updating, but that is trivial. > > > > The final question I had was with respect to storing the new epoch. In > > KIP-890 part 2 (epoch bumps) I think we concluded that we don't need to > > store the epoch since we can interpret the previous epoch based on the > > pro

Re: [DISCUSS] KIP-939: Support Participation in 2PC

2023-12-18 Thread Artem Livshits
able to commit the > transaction. Is that the correct reasoning for why we need epoch here but > not the Prepare/Commit state. > > Thanks, > Justine > > On Wed, Nov 22, 2023 at 9:48 AM Artem Livshits > wrote: > > > Hi Justine, > > > > After thinking a bit a

[VOTE] KIP-939: Support Participation in 2PC

2023-12-01 Thread Artem Livshits
Hello, This is a voting thread for https://cwiki.apache.org/confluence/display/KAFKA/KIP-939%3A+Support+Participation+in+2PC . The KIP proposes extending Kafka transaction support (that already uses 2PC under the hood) to enable atomicity of dual writes to Kafka and an external database, and

Re: [DISCUSS] KIP-939: Support Participation in 2PC

2023-11-22 Thread Artem Livshits
downgrade in the Compatibility section - Added a rejected alternative -Artem On Fri, Oct 6, 2023 at 5:16 PM Artem Livshits wrote: > Hi Justine, > > Thank you for the questions. Currently (pre-KIP-939) we always bump the > epoch on InitProducerId and abort an ongoing transaction

Re: [DISCUSS] KIP-994: Minor Enhancements to ListTransactions and DescribeTransactions APIs

2023-11-16 Thread Artem Livshits
On Tue, Nov 7, 2023 at 11:04 PM Andrew Schofield < andrew_schofield_j...@outlook.com> wrote: > Hi Artem, > I think you make a very good point. This also looks to me like it deserves > a version bump for the request. > > Andrew > > > On 8 Nov 2023, at 04:12, Artem

Re: [DISCUSS] KIP-994: Minor Enhancements to ListTransactions and DescribeTransactions APIs

2023-11-07 Thread Artem Livshits
Hi Raman, Thank you for the KIP. I think using the tagged field in DescribeTransactionsResponse should be good -- if either the client or the server don't support it, it's not printed, which is reasonable behavior. For the ListTransactionsRequest, though, I think using the tagged field could

Re: [DISCUSS] KIP-939: Support Participation in 2PC

2023-10-06 Thread Artem Livshits
Hi Raman, Thank you for the questions. Given that the primary effect of setting enable2pc flag is disabling timeout, it makes sense to make enable2pc have similar behavior w.r.t. when it can be set. One clarification about the Ongoing case -- the current (pre-KIP-939) behavior is to abort

Re: [DISCUSS] KIP-939: Support Participation in 2PC

2023-10-06 Thread Artem Livshits
think this > means that we may skip epochs and the data itself will all have the same > epoch) > > I may have follow ups depending on the answer to this. :) > > Thanks, > Justine > > On Thu, Sep 7, 2023 at 9:51 PM Artem Livshits > wrote: > > > Hi Alex, >

Re: [DISCUSS] KIP-966: Eligible Leader Replicas

2023-10-03 Thread Artem Livshits
Hi Colin, I think in your example "do_unclean_recovery" would need to do different things depending on the strategy. do_unclean_recovery() { if (unclean.recovery.manager.enabled) { if (strategy == Aggressive) use UncleanRecoveryManager(waitLastKnownERL=false) // just inspect logs

Re: [DISCUSS] KIP-939: Support Participation in 2PC

2023-09-07 Thread Artem Livshits
s way, clients would know in advance > if 2PC is enabled on the brokers. > > Best, > Alex > > On Fri, Aug 25, 2023 at 9:40 AM Roger Hoover > wrote: > > > Other than supporting multiplexing transactional streams on a single > > producer, I don't see how to improve it.

Re: [DISCUSS] KIP-966: Eligible Leader Replicas

2023-09-07 Thread Artem Livshits
Hi Calvin, Thanks for the KIP. The new ELR protocol looks good to me. I have some questions about unclean recovery, specifically in "balanced" mode: 1. The KIP mentions that the controller would trigger unclear recovery when the leader is fenced, but my understanding is that when a leader is

Re: [DISCUSS] KIP-939: Support Participation in 2PC

2023-08-24 Thread Artem Livshits
to switch to something like a StatefulSet that gives each pod a > stable identity across restarts. On top of that pod identity which you can > use as a prefix, you then assign unique transactional ids to each > concurrency unit (thread/goroutine). > > On Wed, Aug 23, 2023 at 12:53 PM

Re: [DISCUSS] KIP-939: Support Participation in 2PC

2023-08-24 Thread Artem Livshits
ems. > > Let me know if you'd like to explore? > > Guy > > > On 2023/08/17 06:39:57 Artem Livshits wrote: > > Hello, > > > > This is a discussion thread for > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-939%3A+Support+Participation+in+2PC

Re: [DISCUSS] KIP-939: Support Participation in 2PC

2023-08-23 Thread Artem Livshits
to Kafka than the typical model of a single producer per > process. > > Otherwise, it seems you're left with single-threaded model per application > process? > > Thanks, > > Roger > > On Tue, Aug 22, 2023 at 5:11 PM Artem Livshits > wrote: > > > Hi Roger, Arjun

Re: [DISCUSS] KIP-939: Support Participation in 2PC

2023-08-22 Thread Artem Livshits
nt, independent > > txns happening in the same JVM so it seems like the granularity managing > > transactional ids and txn state needs to line up with granularity of the > DB > > locking. > > > > Does that make sense or am I misunderstanding? > > > > Thanks, > &

[DISCUSS] KIP-939: Support Participation in 2PC

2023-08-17 Thread Artem Livshits
Hello, This is a discussion thread for https://cwiki.apache.org/confluence/display/KAFKA/KIP-939%3A+Support+Participation+in+2PC . The KIP proposes extending Kafka transaction support (that already uses 2PC under the hood) to enable atomicity of dual writes to Kafka and an external database, and

[jira] [Created] (KAFKA-15370) Support Participation in 2PC (KIP-939)

2023-08-16 Thread Artem Livshits (Jira)
Artem Livshits created KAFKA-15370: -- Summary: Support Participation in 2PC (KIP-939) Key: KAFKA-15370 URL: https://issues.apache.org/jira/browse/KAFKA-15370 Project: Kafka Issue Type

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

2023-02-07 Thread Artem Livshits
Hi Calvin, Thank you for the KIP. I have a similar question -- we need to support rolling upgrades (when we have some old brokers and some new brokers), so there could be combinations of old leader - new follower, new leader - old follower, new leader - old controller, old leader - new

Re: [VOTE] KIP-890: Transactions Server Side Defense

2023-02-02 Thread Artem Livshits
(non-binding) +1. Looking forward to the implementation and fixing the issues that we've got. -Artem On Mon, Jan 23, 2023 at 2:25 PM Guozhang Wang wrote: > Thanks Justine, I have no further comments on the KIP. +1. > > On Tue, Jan 17, 2023 at 10:34 AM Jason Gustafson > wrote: > > > > +1.

Re: [DISCUSS] KIP-890 Server Side Defense

2023-01-20 Thread Artem Livshits
nd part -- I think it makes sense to have some sort of > > "sentinel" epoch to signal epoch is about to overflow (I think we sort of > > have this value in place in some ways) so we can codify it in the KIP. > I'll > > look into that and try to update soon. > > >

Re: [DISCUSS] KIP-890 Server Side Defense

2023-01-13 Thread Artem Livshits
t; way to direct this from the response, or if everything should be done on > the client side. Let me know if you have any thoughts here. > > Thanks, > Justine > > On Tue, Jan 10, 2023 at 4:06 PM Artem Livshits > wrote: > > > There are some workflows in the client t

Re: [DISCUSS] KIP-890 Server Side Defense

2023-01-10 Thread Artem Livshits
t;> > > > > > cases > >> > >> > > > > > > > like > >> > >> > > > > > > > > > this and the UNKNOWN_PRODUCER_ID error has a > pretty > >> > long > >> > >> > and > >> > >> > > > > > > > complic

Re: [DISCUSS] KIP-890 Server Side Defense

2022-11-30 Thread Artem Livshits
; but we would want to identify whether we missed the 0 sequence for older > > clients, no? > > > > 2) Upon returning from the transaction coordinator, we can set the > > transaction > > as ongoing on the leader by populating currentTxnFirstOffset > > thro

Re: [DISCUSS] KIP-881: Rack-aware Partition Assignment for Kafka Consumers

2022-11-30 Thread Artem Livshits
d locality, so aligning on > replica rack ids generally makes sense. > 3) We prioritize balanced assignment over locality in the consumer, so the > default assignors should work effectively regardless of broker's replica > selector. > > Does that make sense? > > > Thank you, &g

Re: [DISCUSS] KIP-881: Rack-aware Partition Assignment for Kafka Consumers

2022-11-28 Thread Artem Livshits
kAwareReplicaSelector in the broker. Did I misunderstand your concern? > > Regards, > > Rajini > > > On Tue, Nov 22, 2022 at 11:03 PM Artem Livshits > wrote: > > > Hi Rajini, > > > > Thank you for the KIP, the KIP looks good to match > RackAwareReplicaSelec

Re: [DISCUSS] KIP-890 Server Side Defense

2022-11-22 Thread Artem Livshits
Hi Justine, Thank you for the KIP. I have one question. 5) For new clients, we can once again return an error UNKNOWN_PRODUCER_ID I believe we had problems in the past with returning UNKNOWN_PRODUCER_ID because it was considered fatal and required client restart. It would be good to spell out

Re: [DISCUSS] KIP-881: Rack-aware Partition Assignment for Kafka Consumers

2022-11-22 Thread Artem Livshits
Hi Rajini, Thank you for the KIP, the KIP looks good to match RackAwareReplicaSelector behavior that is available out-of-box. Which should probably be good enough in practice. >From the design perspective, though, RackAwareReplicaSelector is just one possible plugin, in theory the broker could

Re: [DISCUSS] Apache Kafka 3.3.0 Release

2022-08-30 Thread Artem Livshits
Hi José , I found a potential regression in the new Sticky Partitioner logic, details are here https://issues.apache.org/jira/browse/KAFKA-14156. I've added a draft PR, will add unit tests soon. I think we should include the fix into 3.3.0. -Artem On Mon, Aug 29, 2022 at 1:17 PM Colin McCabe

[jira] [Created] (KAFKA-14156) Built-in partitioner may create suboptimal batches with large linger.ms

2022-08-10 Thread Artem Livshits (Jira)
Artem Livshits created KAFKA-14156: -- Summary: Built-in partitioner may create suboptimal batches with large linger.ms Key: KAFKA-14156 URL: https://issues.apache.org/jira/browse/KAFKA-14156 Project

[jira] [Created] (KAFKA-14087) Add jmh benchmark for producer with MockClient

2022-07-19 Thread Artem Livshits (Jira)
Artem Livshits created KAFKA-14087: -- Summary: Add jmh benchmark for producer with MockClient Key: KAFKA-14087 URL: https://issues.apache.org/jira/browse/KAFKA-14087 Project: Kafka Issue

[jira] [Created] (KAFKA-14086) Cleanup PlaintextConsumerTest.testInterceptors to not pass null record

2022-07-19 Thread Artem Livshits (Jira)
Artem Livshits created KAFKA-14086: -- Summary: Cleanup PlaintextConsumerTest.testInterceptors to not pass null record Key: KAFKA-14086 URL: https://issues.apache.org/jira/browse/KAFKA-14086 Project

[jira] [Created] (KAFKA-14085) Clean up usage of asserts in KafkaProducer

2022-07-19 Thread Artem Livshits (Jira)
Artem Livshits created KAFKA-14085: -- Summary: Clean up usage of asserts in KafkaProducer Key: KAFKA-14085 URL: https://issues.apache.org/jira/browse/KAFKA-14085 Project: Kafka Issue Type

Re: [VOTE] KIP-847

2022-07-18 Thread Artem Livshits
Thank you for the vote. I've got three +1s (Ismael, Jun, David), closing the vote now. -Artem On Wed, Jul 13, 2022 at 1:42 AM Ismael Juma wrote: > Thanks for the updates, +1 (binding) from me. > > Ismael > > On Fri, Jul 8, 2022 at 3:45 AM Artem Livshits >

[jira] [Created] (KAFKA-14083) Check if we don't need to refresh time in RecordAccumulator.append

2022-07-18 Thread Artem Livshits (Jira)
Artem Livshits created KAFKA-14083: -- Summary: Check if we don't need to refresh time in RecordAccumulator.append Key: KAFKA-14083 URL: https://issues.apache.org/jira/browse/KAFKA-14083 Project

Re: [DISCUSS] KIP-847: Add ProducerCount metrics

2022-07-11 Thread Artem Livshits
, Jul 5, 2022 at 5:57 PM Artem Livshits > wrote: > > > I've updated the KIP to clarify that the metric reflects the total amount > > of producer ids in all partitions maintained in the broker. > > > > -Artem > > > > On Thu, Jun 30, 2022 at 11:46 AM Jun R

Re: [VOTE] KIP-847

2022-07-07 Thread Artem Livshits
gt; > > > I am +1 (binding). > > > > A small nit: ProducerIdCount should be used in the motivation. > > > > Best, > > David > > > > On Thu, Jun 23, 2022 at 10:26 PM Artem Livshits > > wrote: > > > > > > Hello, &

Re: Transactions, delivery timeout and changing transactional producer behavior

2022-07-06 Thread Artem Livshits
Hi Daniel, What you say makes sense. Could you file a bug and put this info there so that it's easier to track? -Artem On Wed, Jul 6, 2022 at 8:34 AM Dániel Urbán wrote: > Hello everyone, > > I've been investigating some transaction related issues in a very > problematic cluster. Besides

Re: [DISCUSS] KIP-847: Add ProducerCount metrics

2022-07-05 Thread Artem Livshits
gt; > > > Thanks for the update. > > > LGTM. > > > > > > Luke > > > > > > On Thu, Jun 30, 2022 at 6:51 AM Artem Livshits > > > wrote: > > > > > > > Thank you for your feedback. I've updated the KIP to elaborate o

Re: [DISCUSS] KIP-847: Add ProducerCount metrics

2022-06-29 Thread Artem Livshits
> > > > Jun > > > > On Wed, Jun 22, 2022 at 1:08 AM David Jacot > > > wrote: > > > > > Hi Artem, > > > > > > The KIP LGTM. > > > > > > Thanks, > > > David > > > > > > On Tue, Jun 21, 20

[VOTE] KIP-847

2022-06-23 Thread Artem Livshits
Hello, I'd like to start a vote on KIP-847 https://cwiki.apache.org/confluence/display/KAFKA/KIP-847%3A+Add+ProducerCount+metrics -Artem

Re: [DISCUSS] KIP-847: Add ProducerCount metrics

2022-06-21 Thread Artem Livshits
If there is no other feedback I'm going to start voting in a couple days. -Artem On Fri, Jun 17, 2022 at 3:50 PM Artem Livshits wrote: > Thank you for your feedback. Updated the KIP and added the Rejected > Alternatives section. > > -Artem > > On Fri, Jun 17, 2022 at 1

Re: [DISCUSS] KIP-847: Add ProducerCount metrics

2022-06-17 Thread Artem Livshits
t case. > > Ismael > > On Fri, Jun 17, 2022 at 12:04 PM Artem Livshits > wrote: > > > Do you propose to have 2 metrics instead of one? Right now we don't > track > > if the producer id was transactional or idempotent and for metric > > collection we'd either have to

Re: [DISCUSS] KIP-847: Add ProducerCount metrics

2022-06-17 Thread Artem Livshits
t metrics. > > Ismael > > On Thu, Jun 16, 2022 at 2:27 PM Artem Livshits > wrote: > > > Hi Ismael, > > > > Thank you for your feedback. Yes, this is counting the number of > producer > > ids tracked by the partition and broker. Ano

Re: [DISCUSS] KIP-847: Add ProducerCount metrics

2022-06-16 Thread Artem Livshits
gt; IDs tracked by the broker? > > Ismael > > On Wed, Jun 15, 2022, 3:12 PM Artem Livshits .invalid> > wrote: > > > Hello, > > > > I'd like to start a discussion on the KIP-847: > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-847%3A+Add+ProducerCount+metrics > > . > > > > -Artem > > >

[DISCUSS] KIP-847: Add ProducerCount metrics

2022-06-15 Thread Artem Livshits
Hello, I'd like to start a discussion on the KIP-847: https://cwiki.apache.org/confluence/display/KAFKA/KIP-847%3A+Add+ProducerCount+metrics . -Artem

[jira] [Created] (KAFKA-13999) Add ProducerCount metrics (KIP-847)

2022-06-15 Thread Artem Livshits (Jira)
Artem Livshits created KAFKA-13999: -- Summary: Add ProducerCount metrics (KIP-847) Key: KAFKA-13999 URL: https://issues.apache.org/jira/browse/KAFKA-13999 Project: Kafka Issue Type

[jira] [Created] (KAFKA-13992) MockProducer shouldn't use DefaultPartitioner

2022-06-14 Thread Artem Livshits (Jira)
Artem Livshits created KAFKA-13992: -- Summary: MockProducer shouldn't use DefaultPartitioner Key: KAFKA-13992 URL: https://issues.apache.org/jira/browse/KAFKA-13992 Project: Kafka Issue Type

Re: [DISCUSS] KIP-841: Fenced replicas should not be allowed to join the ISR in KRaft

2022-05-20 Thread Artem Livshits
The KIP LGTM. My only question is why it's an issue with KRaft -- looks like ZK would have the same issue? -Artem On Fri, May 20, 2022 at 8:51 AM David Jacot wrote: > This KIP is pretty straight forward. I will start a vote on Monday > if no one objects. > > Best, > David > > On Wed, May 18,

[jira] [Created] (KAFKA-13885) Add new metrics for partitioner logic introduced in KIP-794

2022-05-06 Thread Artem Livshits (Jira)
Artem Livshits created KAFKA-13885: -- Summary: Add new metrics for partitioner logic introduced in KIP-794 Key: KAFKA-13885 URL: https://issues.apache.org/jira/browse/KAFKA-13885 Project: Kafka

Re: [VOTE] KIP-794: Strictly Uniform Sticky Partitioner

2022-05-05 Thread Artem Livshits
> > > Lucas > > > > On Wed, Mar 23, 2022 at 8:27 PM Luke Chen wrote: > > > > > Hi Artem, > > > > > > Thanks for the KIP and the patience during discussion! > > > +1 binding from me. > > > > > > Luke > > > >

[jira] [Created] (KAFKA-13880) DefaultStreamPartitioner may get "stuck" to one partition for unkeyed messages

2022-05-05 Thread Artem Livshits (Jira)
Artem Livshits created KAFKA-13880: -- Summary: DefaultStreamPartitioner may get "stuck" to one partition for unkeyed messages Key: KAFKA-13880 URL: https://issues.apache.org/jira/browse/K

[VOTE] KIP-794: Strictly Uniform Sticky Partitioner

2022-03-21 Thread Artem Livshits
Hi all, I'd like to start a vote on https://cwiki.apache.org/confluence/display/KAFKA/KIP-794%3A+Strictly+Uniform+Sticky+Partitioner . -Artem

Re: [DISCUSS] KIP-794: Strictly Uniform Sticky Partitioner

2022-03-17 Thread Artem Livshits
art voting in the next couple of days. -Artem On Mon, Mar 14, 2022 at 6:19 PM Artem Livshits wrote: > Hi Jun, > > 33. Sounds good. Updated the KIP. > > -Artem > > On Mon, Mar 14, 2022 at 5:45 PM Jun Rao wrote: > >> Hi, Artem, >> >> 33. We introduc

Re: [DISCUSS] KIP-794: Strictly Uniform Sticky Partitioner

2022-03-14 Thread Artem Livshits
onvincing usage, it's probably better to deprecate it so that we > could keep the API clean. > > Thanks, > > Jun > > On Mon, Mar 14, 2022 at 1:36 PM Artem Livshits > wrote: > > > Hi Jun, > > > > 33. That's an interesting point. Technically, onNewBatc

Re: [DISCUSS] KIP-794: Strictly Uniform Sticky Partitioner

2022-03-14 Thread Artem Livshits
ted side effect? > > Thanks, > > Jun > > On Thu, Mar 10, 2022 at 5:20 PM Artem Livshits > wrote: > > > Hi Jun, > > > > 32. Good point. Do you think it's ok to defer the metrics until we run > > some benchmarks so that we get a better idea of what

Re: [DISCUSS] KIP-794: Strictly Uniform Sticky Partitioner

2022-03-10 Thread Artem Livshits
need to add any new metric on the producer? For example, if > partitioner.availability.timeout.ms is > 0, it might be useful to know the > number of unavailable partitions. > > Thanks, > > Jun > > On Thu, Mar 10, 2022 at 12:46 PM Artem Livshits > wrote: > > > Hi

Re: [DISCUSS] KIP-794: Strictly Uniform Sticky Partitioner

2022-03-10 Thread Artem Livshits
gt; > must be used? By default, the built-in partitioner would be used > > unless > > > > the > > > > > user explicitly specify one. The downside is that the new default > > > > behavior > > > > > would not work

Re: [DISCUSS] KIP-794: Strictly Uniform Sticky Partitioner

2022-03-09 Thread Artem Livshits
so add a > > > warning if one of them is explicitly provided by the user to inform > them > > > that they should switch to the new built-in one. I am pretty sure that > > most > > > of the folks use the default configuration anyway. > > > > >

Re: [DISCUSS] KIP-794: Strictly Uniform Sticky Partitioner

2022-03-03 Thread Artem Livshits
meout.ms only apply to the two built-in > partitioners DefaultPartitioner and UniformStickyPartitioner, right? It > would be useful to document that in the KIP. > > Thanks, > > Jun > > On Thu, Mar 3, 2022 at 9:47 AM Artem Livshits > wrote: > > > Hi Jun, > > > >

Re: [DISCUSS] KIP-794: Strictly Uniform Sticky Partitioner

2022-03-03 Thread Artem Livshits
he effective batch size, does setting batch.size to > the effective batch size achieve the same result? > > 4. Thanks for the explanation. Makes sense to me. > > Thanks, > > Jun > > Thanks, > > Jun > > On Fri, Feb 25, 2022 at 8:26 PM Artem Livshits > wrote

Re: [DISCUSS] KIP-794: Strictly Uniform Sticky Partitioner

2022-02-25 Thread Artem Livshits
on.availability.timeout.ms > threshold. > > Thanks, > > Jun > > On Fri, Feb 18, 2022 at 5:14 PM Artem Livshits > wrote: > > > Hello Luke, Jun, > > > > Thank you for your feedback. I've added the Rejected Alternative section > > that may clarify so

Re: [DISCUSS] KIP-794: Strictly Uniform Sticky Partitioner

2022-02-18 Thread Artem Livshits
fficient. WDYT? > > > 4. I think the improved queuing logic should be good enough. I can't > get > > > the benefit of having `partition.availability.timeout.ms` config. In > > > short, you want to make the partitioner take the broker load into > > > consideration. We can just

Re: [DISCUSS] KIP-794: Strictly Uniform Sticky Partitioner

2022-02-15 Thread Artem Livshits
Hello, Please add your comments about the KIP. If there are no considerations, I'll put it up for vote in the next few days. -Artem On Mon, Feb 7, 2022 at 6:01 PM Artem Livshits wrote: > Hello, > > After trying a few prototypes, I've made some changes to the public > interface.

Re: [DISCUSS] KIP-794: Strictly Uniform Sticky Partitioner

2022-02-07 Thread Artem Livshits
Hello, After trying a few prototypes, I've made some changes to the public interface. Please see the updated document https://cwiki.apache.org/confluence/display/KAFKA/KIP-794%3A+Strictly+Uniform+Sticky+Partitioner . -Artem On Thu, Nov 4, 2021 at 10:37 AM Artem Livshits wrote: > He

[jira] [Resolved] (KAFKA-13540) UniformStickyPartitioner leads to uneven Kafka partitions

2021-12-13 Thread Artem Livshits (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13540?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Artem Livshits resolved KAFKA-13540. Resolution: Duplicate See also https://cwiki.apache.org/confluence/display/KAFKA/KIP-794

Re: [DISCUSS] KIP-782: Expandable batch size in producer

2021-12-08 Thread Artem Livshits
h.size or batch.max.size? > > Thanks, > > Jun > > On Wed, Dec 8, 2021 at 9:58 AM Artem Livshits > wrote: > > > Hi Jun, > > > > 10. My understanding is that MemoryRecords would under the covers be > > allocated in chunks, so logically it still would be o

Re: [DISCUSS] KIP-782: Expandable batch size in producer

2021-12-08 Thread Artem Livshits
uest.size, currently it's used for both the max record > size and the max request size, which is unintuitive. Perhaps we could > introduce a new config max.record.size that defaults to 1MB. We could then > increase max.request.size to sth like 10MB. > > Thanks, > > Jun &

Re: [DISCUSS] KIP-782: Expandable batch size in producer

2021-11-29 Thread Artem Livshits
tch.max.size" > is increased, there might be chances that the "ready" partitions need to > wait for next request to send to broker, because of the "max.request.size" > (default 1MB) limitation. "max.request.size" can also be considered to > inc

Re: [DISCUSS] KIP-782: Expandable batch size in producer

2021-11-23 Thread Artem Livshits
but maybe I can firstly decrease the > "batch.max.size" to 32KB, instead of aggressively 256KB in the KIP. That > should alleviate the problem. And still improve the throughput. What do you > think? > > Thank you. > Luke > > On Tue, Nov 23, 2021 at 9:04 AM Artem L

Re: [DISCUSS] KIP-782: Expandable batch size in producer

2021-11-22 Thread Artem Livshits
> I think this KIP would change the behaviour of producers when there are multiple partitions ready to be sent This is correct, the pattern changes and becomes more coarse-grained. But I don't think it changes fairness over the long run. I think it's a good idea to change drainIndex to be

Re: [DISCUSS] KIP-794: Strictly Uniform Sticky Partitioner

2021-11-19 Thread Artem Livshits
method that takes a callback that can be used to estimate record size. I've updated the KIP correspondingly https://cwiki.apache.org/confluence/display/KAFKA/KIP-794%3A+Strictly+Uniform+Sticky+Partitioner -Artem On Mon, Nov 8, 2021 at 5:42 PM Artem Livshits wrote: > Hi Luke, Justine, > >

Re: [DISCUSS] KIP-794: Strictly Uniform Sticky Partitioner

2021-11-08 Thread Artem Livshits
; description is confusing. And that's why I asked you if you have some > kind > > of "batch switch" here. > > > > 2. In motivation, you mentioned 1 drawback of current > > UniformStickyPartitioner is "the sticky partitioner doesn't create > batches > > as ef

Re: [DISCUSS] KIP-794: Strictly Uniform Sticky Partitioner

2021-11-05 Thread Artem Livshits
the same > example, to describe more detail about how you fix this issue with your > way. > > Thank you. > Luke > > On Fri, Nov 5, 2021 at 1:38 AM Artem Livshits > wrote: > > > Hello, > > > > This is the discussion thread for > > &g

[DISCUSS] KIP-794: Strictly Uniform Sticky Partitioner

2021-11-04 Thread Artem Livshits
Hello, This is the discussion thread for https://cwiki.apache.org/confluence/display/KAFKA/KIP-794%3A+Strictly+Uniform+Sticky+Partitioner . The proposal is a bug fix for https://issues.apache.org/jira/browse/KAFKA-10888, but it does include a client config change, therefore we have a KIP to

Wiki permission request

2021-11-02 Thread Artem Livshits
Hello, I'd like to be added to the contributors list, so that I can submit a KIP. My Jira ID is: alivshits Wiki ID: alivshits Thanks, -Artem

  1   2   >