Artem Livshits created KAFKA-19367:
--
Summary: InitProducerId with TV2 double-increments epoch if
ongoing transaction is aborted
Key: KAFKA-19367
URL: https://issues.apache.org/jira/browse/KAFKA-19367
Hi Calvin,
Thanks for the KIP.
[AL1]
> When the cluster does not support the new version, the admin client will
fail to build the ListTransaction request with version 2.
UnsupportedVersionException will be thrown.
It looks like this will make it impossible to use newer clients with older
brokers
Hi Kaushik,
Thank you for the KIP! I think it'll make writing transactional
application easier and less error prone. I have a couple comments:
AL1. The keep proposes changing the semantics
of UnknownProducerIdException. Currently, this error is never returned by
the broker so we cannot valida
solution for broker side errors or not -- so maybe it's ok to
> > accept this and drop/ignore the broker side error question for now.
> >
> >
> >
> > A small follow up thought/question: instead of using a boolean, would we
> > actually want to make it a var-a
exciting.
> > > +1 non-binding
> > >
> > > Best
> > > Omnia
> > >
> > > On 1 Dec 2023, at 19:06, Artem Livshits .INVALID>
> > > wrote:
> > >
> > > Hello,
> > >
> > > This is a voting thread for
>
> > > producer if some records are not critical-path
> >
> > We (1) already have a "too many connections" problem with KS so using
> > move clients is something we try to avoid (and we actually hope to
> > reduce the number of client and connection mid
re
> > responsible for the data until commitTransaction() has succeeded.
> > With this proposed change that makes the producer tolerate
> > too-large-exceptions, applications are still responsible for storing
> their
> > data until commitTransaction() has succeeded, becaus
Hi Greg,
What you say makes a lot of sense. I just wanted to clarify a couple of
subtle points.
AL1. There is a functional reason to handle errors that happen on send
(oginate in the producer logic in the client) vs. errors that are returned
from the broker. The problem is that RecordTooLargeEx
way to clear the error state of the
> > >> producer. We
> > >>>>>>>> could catch a `RecordTooLargeException` from `send()`, call the
> > >> handler
> > >>>>>> and
> > >>>>>>>> le
lead to broken behavior as the application won't be able to
> stop a commit from proceeding even on fatal errors.
>
> Is this with respect to the case a request is still inflight when we call
> commitTransaction? I thought we still wait for requests (and their errors)
> to come in a
; >>> Chris
> >>>
> >>> On Wed, Jun 19, 2024 at 10:32 PM Matthias J. Sax
> wrote:
> >>>
> >>>> Thanks for the KIP Alieh. I actually like the KIP as-is, but think
> >>>> Arthem raises very good points...
> >>>>
Hi Alieh,
Thank you for the KIP. I have a couple of suggestions:
AL1. We should throw an error from flush after we clear it. This would
make it so that both "send + commit" and "send + flush + commit" (the
latter looks like just a more verbose way to express the former, and it
would be intuiti
t; handler
> > > is an very advanced feature anyway, and if it's implemented in a bad
> > > way, well, it's a user error -- we cannot protect users from
> everything.
> > > To me, a handler like this, is to some extend "business logic" and if a
>
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
UnknownTopicOrPartiti
[
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
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
records,
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
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,
>
>
7;t provide the
> same guarantees as 2PC either.
>
> To me, if 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
>
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 for
, but not too long for non-behaving
> 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).
> >
>
action-open-time-max Max transaction-coordinator-metrics
> 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
ere clients 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,
&
n to
> circumvent that.
>
> 32. 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,
> >
> >
> 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
tion, but doesn't have 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."
> W
ompleteTransaction 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, Fe
ayered approach. 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
of the extra RPC call is too much, 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
>
the
> endTxn request 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:
t produceId to commit an old txn works, but
> can be confusing. It's going to be hard for people 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
> 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 doing)
e:
> On 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 th
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
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 pr
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 t
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
hink the tags 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
correct epoch to be 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,
> >
> > Af
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 help
o downgrade
- Added a note about 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 InitProducerI
Artem
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,
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 lea
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 ongoin
0 where we also bump the epoch on every transaction. (I 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
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 fr
ommit.enable? This 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'
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 fe
> may need 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, 2
/ systems.
>
> 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+Partic
producers and resultant
> connections 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 Livshit
tiple, concurrent, 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?
> >
>
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
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
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 controller
(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. Than
; >
> > As for the second 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 a
ill wondering if there is a
> 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
ly it makes sense to simplify
> error
> >> > >> handling
> >> > >> > > in
> >> > >> > > > > > cases
> >> > >> > > > > > > > like
> >> > >> > > > > > > > > > this and the UNKNOWN_PRODUCER_ID error has a
>
y changes we are making. Maybe I'm missing something,
> > 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 on
or improved 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?
>
>
>
RackAwareReplicaSelector 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
> RackAwareRepli
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
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 u
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
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
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
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
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
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
> wrote:
>
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
Pr
, 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
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,
&
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 find
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
;
> > Thanks,
> >
> > Jun
> >
> > On Wed, Jun 22, 2022 at 1:08 AM David Jacot >
> > wrote:
> >
> > > Hi Artem,
> > >
> > > The KIP LGTM.
> > >
> > > Thanks,
> > > David
> > >
> > >
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
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
unt in that 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
nalProducerCount 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 a
> 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
> >
>
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
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
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
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, 2
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
inding from me.
> >
> > 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.
> > >
> > > Luk
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
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
s, I'll start 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.
any convincing 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,
expected 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 ide
o we 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:
>
> &g
; partitioner
> > > > > 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
> >
uld also 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.
> > >
lity.timeout.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,
> >
&g
t; the user somehow knows the 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
ed on a partition.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 we can directly create new batch for next
> records.
> > > This way should be able to make it more efficient. 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.
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
>
1 - 100 of 114 matches
Mail list logo