you meant or I
> misunderstand?
> As for the KIP-932, I'll have a closer look.
>
> Ivan
>
>
> On Wed, May 15, 2024, at 20:14, Jun Rao wrote:
> > Hi, Ivan,
> >
> > Thanks for the KIP. +1
> >
> > Just a minor comment. Should we add metadata.recovery.strat
s for the nice KIP, it will allow other messaging use cases to be
> >> onboarded to Kafka.
> >>
> >> +1 from me.
> >>
> >> Satish.
> >>
> >> On Tue, 7 May 2024 at 03:41, Jun Rao wrote:
> >>>
> >>> Hi, Andrew,
> >>&
Hi, Ivan,
Thanks for the KIP. +1
Just a minor comment. Should we add metadata.recovery.strategy to the
Streams and the newly introduced ShareConsumer (KIP-932) too?
Jun
On Wed, May 8, 2024 at 11:35 AM Manikumar wrote:
> Thanks for the KIP.
>
> +1 (binding).
>
> On Wed, Apr 17, 2024 at 7:50
Hi, Andrew,
Thanks for the KIP. +1
Jun
On Mon, Mar 18, 2024 at 11:00 AM Edoardo Comar
wrote:
> Thanks Andrew,
>
> +1 (binding)
>
> Edo
>
> On Mon, 18 Mar 2024 at 16:32, Kenneth Eversole
> wrote:
> >
> > Hi Andrew
> >
> > + 1 (Non-Binding)
> >
> > This will be great addition to Kafka
> >
> >
remains because it adds support for share
> groups.
> This is needed to list share groups for the admin client and the
> command-line
> tools.
>
> Thanks,
> Andrew
>
> > On 6 May 2024, at 19:26, Jun Rao wrote:
> >
> > Hi, Andrew,
> >
> > Removing
custom group types.
> This all needs sorting out, I think.
>
> I propose to remove AdminClient.listGroups() from this KIP, and put
> it in the administration KIP.
>
> Let me know what you think.
>
> Thanks,
> Andrew
>
>
> > On 6 May 2024, at 18:04, Jun Rao wrot
livery, I’ll update the KIP with the final value.
>
> 164. KRaft only. All the RPCs are “broker” only. None of the code will
> be merged until after 3.8 has branched.
>
> Thanks,
> Andrew
>
> > On 4 May 2024, at 00:12, Jun Rao wrote:
> >
> > Hi, Andrew,
> &g
have tweaked the text so that the simple
> assignor will take into account existing assignment information when
> it has it, which would just minimise unnecessary churn of (b).
>
> 158. I’ve changed it to ReadShareGroupStateSummary.
>
> Thanks,
> Andrew
>
>
> > On
, yes. I see. Of course, I want the names as consistent and
> understandable too. I suggest renaming
> ReadShareGroupOffsetsState to ReadShareGroupStateSummary.
> I haven’t changed the KIP yet, so let me know if that’s OK.
>
> Thanks,
> Andrew
>
> > On 2 May 2024, at 22:18, Jun Rao wrote
y a leader change needs
> to force a ShareSnapshot. I’ve added leaderEpoch to the ShareUpdate.
>
> 158. Although ReadShareGroupOffsetsState is a bit of a mouthful,
> having “State” in the name makes it clear that this one the family of
> inter-broker RPCs served by the share coordinator. The
’s right.
>
> 147. The measurement is certainly from the point of view
> of the client, but it’s driven by sending and receiving heartbeats
> rather than whether the client triggered the rebalance itself.
> The client decides when it enters and leaves reconciliation
> of the assignment
Hi, Abhijeet,
Thanks for the KIP. +1
Jun
On Thu, Apr 25, 2024 at 10:30 PM Abhijeet Kumar
wrote:
> Hi All,
>
> I would like to start the vote for KIP-1023 - Follower fetch from tiered
> offset
>
> The KIP is here:
>
>
add this. Done.
>
> 149. There are several error codes for WriteShareGroupStateResponse:
>
> NOT_COORDINATOR - This is not the share coordinator you’re looking for.
> COORDINATOR_NOT_AVAILABLE - The SC can’t
> COORDINATOR_LOAD_IN_PROGRESS - The SC is replaying the topic.
> GROUP_ID_NOT_FOU
Hi, Abhijeet,
Thanks for the updated KIP. It looks good to me.
Jun
On Mon, Apr 22, 2024 at 12:08 PM Abhijeet Kumar
wrote:
> Hi Jun,
>
> Please find my comments inline.
>
>
> On Thu, Apr 18, 2024 at 3:26 AM Jun Rao wrote:
>
> > Hi, Abhijeet,
> >
> >
cordingly.
>
>
> I have made one more change to the KIP. The SharePartitionAssignor
> interface has been renamed to
> o.a.k.coordinator.group.assignor.ShareGroupPartitionAssignor and it
> now extends the PartitionAssignor interface. It’s essentially a marker
> of which partition assignors
e read, they’re likely to be very close.
>
> Later, for a cluster which is using a custom persister, the
> share-coordinator
> metrics would likely not be reported, and the persister would have its own
> metrics.
>
> 137.3: Correct. Fixed.
>
> 137.4: Yes, it does in
group
> configuration
> will only be permitted to name an assignor which is in this list. For now,
> there
> is no group configuration for assignor, so all groups get the one and only
> assignor in the list.
>
> 135. It’s the number of threads per broker. For a cluster with a s
ration to enable the feature in the mailing
> list with
> David Jacot also, so I anticipate a bit of change in this area still.
>
> Thanks,
> Andrew
>
> > On 15 Apr 2024, at 23:34, Jun Rao wrote:
> >
> > Hi, Andrew,
> >
> > Thanks for the up
ify this in my reply to Jun.
> > >
> > > > The follower needs to build the local data starting from the offset
> > > > Earliest-Pending-Upload-Offset. Hence it needs the offset and the
> > > > corresponding leader-epoch.
> > > > There are two ways to d
akes evolution and extension of the API much easier. Also, it matches
> the precedent set by ListConsumerGroupOffsetSpec.
>
> 113. ListConsumerGroupsResults.errors() is the same. I think you just have
> to look in the exception details and the same pattern is being followed
> here.
&g
Hi, Justine,
Thanks for the KIP. +1
Jun
On Wed, Apr 10, 2024 at 9:13 AM José Armando García Sancio
wrote:
> Hi Justine,
>
> +1 (binding)
>
> Thanks for the improvement.
> --
> -José
>
Justine Olshan
wrote:
> Updated. :)
> Thanks for the reviews
>
> Justine
>
> On Thu, Apr 11, 2024 at 11:01 AM Jun Rao wrote:
>
> > Hi, Justine,
> >
> > Thanks for the updated KIP.
> >
> > Perhaps it's better to name the new config
>
fig was never actually approved in any KIP. But we can say
> it is deprecated.
> I can change the config name.
>
> Justine
>
> On Thu, Apr 11, 2024 at 8:52 AM Jun Rao wrote:
>
> > Hi, Justine,
> >
> > Thanks for the updated KIP.
> >
> > Would u
KIP-1014 has been inactive recently. I can
> update my KIP and mention this change on that discussion thread.
>
> Justine
>
> On Tue, Apr 9, 2024 at 9:01 AM Jun Rao wrote:
>
> > Hi, Justine,
> >
> > A single config makes sense to me too. We just need to reach
Hi, Andrew,
Thanks for the reply. A few more comments.
41.
41.1 How does the partition leader obtain the group epoch to set
WriteShareGroupStateRequest.StateEpoch?
41.2 What's the benefit of having the group coordinator initialize the
state and the partition leader set the SPSO? It seems simpler
[
https://issues.apache.org/jira/browse/KAFKA-16485?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Jun Rao resolved KAFKA-16485.
-
Fix Version/s: 3.8.0
Resolution: Fixed
Merged the PR to trunk.
> Fix broker metrics to fol
makes sense, I will update the KIP.
>
> Justine
>
> On Mon, Apr 8, 2024 at 3:20 PM Jun Rao wrote:
>
> > Hi, Justine,
> >
> > Thanks for the updated KIP.
> >
> > One more question related to KIP-1014. It introduced a new
> > config unstabl
d
> that
> >> > transaction version is simpler and conveys more than just protocol
> >> changes
> >> > (flexible records for example)
> >> >
> >> > I will update the KIP as well as KIP-890
> >> >
> >> > Thanks,
> >>
Hi, Abhijeet,
Thanks for the KIP. Left a few comments.
1. "A drawback of using the last-tiered-offset is that this new follower
would possess only a limited number of locally stored segments. Should it
ascend to the role of leader, there is a risk of needing to fetch these
segments from the
Hi, Andrew,
Thanks for the updated KIP. A few more comments.
40. It would be useful to document the flow for serving the initial
ShareGroupHeartbeatRequest. The coordinator needs to (1) write the
ConsumerGroupMetadata record, the ConsumerGroupMemberMetadata record, the
initial
eed to be populated for TV 1. We don't
> have the same need to retain this since it is written directly to the
> transaction log in InitProducerId. It is only needed for KIP-890 part 2 /
> TV 2.
>
> 22. We can do that.
>
> Justine
>
> On Tue, Apr 2, 2024 at 10:41 AM Jun
s epoch but we explained in this KIP
> how we can figure this out.
>
> Justine
>
>
>
> On Mon, Apr 1, 2024 at 3:56 PM Jun Rao wrote:
>
> > Hi, Justine,
> >
> > Thanks for the updated KIP. A couple of more comments.
> >
> > 20. Could we show
at. I can include this context in
> the
> > KIP.
> >
> > I'm happy to modify his 1 and 2 to 0 and 1.
> >
> > Justine
> >
> > On Thu, Mar 28, 2024 at 10:57 AM Jun Rao
> wrote:
> >
> >> Hi, David,
> >>
> >&g
Hi, Jose,
Thanks for the KIP. +1
Jun
On Fri, Mar 29, 2024 at 9:55 AM José Armando García Sancio
wrote:
> Hi all,
>
> I would like to call a vote to adopt KIP-853.
>
> KIP: https://cwiki.apache.org/confluence/x/nyH1D
> Discussion thread:
>
Hi, Jose,
Thanks for the explanation. Other than depending on KIP-1022 to be
approved, the KIP looks good to me now.
Jun
On Thu, Mar 28, 2024 at 2:56 PM José Armando García Sancio
wrote:
> Hi Jun,
>
> See my comments below.
>
> On Thu, Mar 28, 2024 at 11:09 AM Jun Rao wro
reporting the same
metric. For example, if they don't report the same value (e.g. because one
broker is catching up), how does a user know which value is correct?
Thanks,
Jun
On Thu, Mar 28, 2024 at 10:56 AM José Armando García Sancio
wrote:
> Hi Jun,
>
> On Thu, Mar 28, 2024 at 10:
other use
> > case is the Queues KIP. I think that we should also use this new flag to
> > gate it.
> >
> > Best,
> > David
> >
> > On Thu, Mar 28, 2024 at 1:14 AM Jun Rao
> wrote:
> >
> >> Hi, Justine,
> >>
> >> Thanks for
d, Mar 27, 2024 at 2:26 PM Jun Rao wrote:
> > 55.1 How does the broker and non-leader controller know the pending
> voters?
>
> They are in the log. Pending voter sets are VotersRecords between the
> HWM and the LEO. The leader will make sure that there is at most one
> V
hanging the IV strings, but I wasn't sure if there would
> be some disagreement with the decision. Not sure if that breaks
> compatibility etc. Happy to hear everyone's thoughts.
>
> Justine
>
> On Wed, Mar 27, 2024 at 3:36 PM Jun Rao wrote:
>
> > Hi, Justine,
> >
> > Than
Hi, Justine,
Thanks for the reply.
Having "kafka-feature dependencies" seems enough to me. We don't need to
include the dependencies in the output of "kafka-feature describe".
We only support "dependencies" in kafka-feature, not kafka-storage. We
probably should do the same for
Hi, Jose,
Thanks for the reply.
55.1 How does the broker and non-leader controller know the pending voters?
Jun
On Wed, Mar 27, 2024 at 1:03 PM José Armando García Sancio
wrote:
> Hi Jun,
>
> Thanks for the feedback. See my comments below.
>
> On Mon, Mar 25, 2024 at 2:21 PM
nd finalized versions?
>
> Feature: transaction.protocol.version SupportedMinVersion: 0
> SupportedMaxVersion: 2 FinalizedVersionLevel: 1 Epoch: 3 Dependencies:
> metadata.version=4
>
> On Wed, Mar 27, 2024 at 11:14 AM Jun Rao wrote:
>
> > Hi, Justine,
> >
> > Ye
ion dependencies)
>
> Justine
>
> On Wed, Mar 27, 2024 at 10:28 AM Jun Rao wrote:
>
> > Hi, Colin,
> >
> > Thanks for the comments. It's fine if we want to keep the --metadata flag
> > if it's useful. Then we should add the same flag for kafka-storag
Hi, Colin,
Thanks for the comments. It's fine if we want to keep the --metadata flag
if it's useful. Then we should add the same flag for kafka-storage for
consistency, right?
Hi, Justine,
Thanks for the reply.
How will a user know about the dependencies among features? Should we
expose them
o the storage tool as it is
> not necessary. The reason it is not removed is purely for backwards
> compatibility. Colin had strong feelings about not removing any flags.
>
> Justine
>
> On Mon, Mar 25, 2024 at 5:01 PM Jun Rao wrote:
>
> > Hi, Justine,
> >
> > Thanks
Hi, Justine,
Thanks for the updated KIP. A few more comments.
10. Could we describe what RPCs group.coordinator.version controls?
12. --metadata METADATA is not removed from kafka-features. Do we have a
justification for this? If so, should we add that to kafka-storage to be
consistent?
14.
and tags.
57. Could we remove --release-version 3.8 in the upgrade example?
Jun
On Mon, Mar 25, 2024 at 11:54 AM José Armando García Sancio
wrote:
> Hi Jun,
>
> See my comments below.
>
> On Fri, Mar 22, 2024 at 1:30 PM Jun Rao wrote:
> > 54. Admin.addMetadataVoter: I
[
https://issues.apache.org/jira/browse/KAFKA-15950?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Jun Rao resolved KAFKA-15950.
-
Fix Version/s: 3.8.0
Resolution: Fixed
merged the PR to trunk.
> Serialize broker heartb
Hi, Jose,
Thanks for the reply. A few more comments.
54. Admin.addMetadataVoter: It seems that Endpoint shouldn't include
securityProtocol since it's not in DescribeQuorumResponse.
55. Metrics:
55.1 It would be useful to be clear whether they are reported by the
controller leader, all
well.
53.5 the the current
extra the
53.6 it will be discover
discover => discovered
53.7 it would beneficial
beneficial => be beneficial
Jun
On Mon, Mar 11, 2024 at 10:39 AM José Armando García Sancio
wrote:
> Hi Jun
>
> Thanks for the feedback. See my comments below.
Hi, Abhijeet,
Thanks for the KIP. +1
Jun
On Fri, Mar 8, 2024 at 3:44 AM Abhijeet Kumar
wrote:
> Hi All,
>
> I would like to start the vote for KIP-956 - Tiered Storage Quotas
>
> The KIP is here:
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-956+Tiered+Storage+Quotas
>
> Regards.
in Vote and
BeginQuorumEpoch requests, but not in EndQuorumEpoch, Fetch and
FetchSnapshot. Could you explain how they are used?
Jun
On Wed, Mar 6, 2024 at 8:53 AM José Armando García Sancio
wrote:
> Hi Jun,
>
> See my comments below.
>
> On Tue, Mar 5, 2024 at 2:57 PM Jun Rao wrote:
&
ou explain whether it is truly needed? Before this KIP,
KRaft already supports replacing a disk on the voter node, right?
Jun
On Mon, Mar 4, 2024 at 2:55 PM José Armando García Sancio
wrote:
> Hi Jun,
>
> Thanks for the feedback. See my comments below.
>
> On Fri, Mar 1, 2024 at 11:36
rottle-time-max
> metrics we have for Kafak Consumers?
> The Avg and Max are computed over the (sliding) window as defined by the
> configuration metrics.sample.window.ms on the server.
>
> (Also, I will update the config and metric names to be consistent)
>
> Regards.
>
> On Thu
upported the tool will throw an error. We
> can also issue a warning if the given command (if supported by the cluster)
> will downgrade a feature. One option is also to require a yes/no prompt or
> flag to allow downgrades. The downgrade tool would allow the same.
> Alternatively we coul
ta.log.dir?
Jun
On Fri, Mar 1, 2024 at 10:51 AM José Armando García Sancio
wrote:
> Hi Jun,
>
> Thanks for the feedback. See my comments below.
>
> On Tue, Feb 27, 2024 at 11:27 AM Jun Rao wrote:
> > 30. Who controls RPCs like Fetch, FetchSnapshot, DescribeQuorum RPC? The
Hi, Justine,
Thanks for the KIP. A few comments below.
10. Currently, MV controls records, inter-broker RPCs and code logic. With
more features, would each of those be controlled by a separate feature or
multiple features. For example, is the new transaction record format
controlled only by MV
and a large enough
> timeout could still produce the undesirable effects for the cloud
> operations (so we kind of get worst of both options -- we don't provide
> guarantees and still have impact on operations).
>
> -Artem
>
> On Fri, Feb 23, 2024 at 8:55 AM Jun Rao wrote:
>
>
type=RemoteLogManager, name=RemoteCopyThrottleTime -> The
> duration of time required at a given moment to bring the observed remote
> copy rate within the allowed limit, by preventing further copies.
>
> Regards.
>
> On Wed, Feb 28, 2024 at 12:28 AM Jun Rao wrote:
>
> > Hi,
ed ApiKeys or kraft.version feature in
ApiVersions response for deciding whether to send AddVoter requests?
Jun
On Tue, Feb 27, 2024 at 8:20 AM José Armando García Sancio
wrote:
> Hi Jun,
>
> Thanks for the comments. See my comments below.
>
> On Mon, Feb 26, 2024 at 4:48 PM Jun Rao
I will update the KIP based on the
> discussion)
>
> Regards,
> Abhijeet.
>
> On Tue, Feb 27, 2024 at 2:49 AM Jun Rao wrote:
>
> > Hi, Abhijeet,
> >
> > Sorry for the late reply. It seems that you haven't updated the KIP based
> > on the discussion? One
i, Feb 23, 2024 at 1:39 PM Jun Rao wrote:
> > 15.1 "In this configuration, the local replica needs to use
> > CONTROLLER_PLAINTEXT because that is what is supported by all of the
> > voters."
> >Hmm, my understanding is that what listener to use can be made
&
ou explain the
usage of this additional quota?
Thanks,
Jun
On Mon, Feb 12, 2024 at 11:08 AM Abhijeet Kumar
wrote:
> Comments inline
>
> On Wed, Dec 6, 2023 at 1:12 AM Jun Rao wrote:
>
> > Hi, Abhijeet,
> >
> > Thanks for the KIP. A few comments.
> >
> >
Hi, Jose,
Thanks for the reply.
15. VotersRecord: Ok. I got the need for including the listener name in the
endpoint. Currently, controller.quorum.voters can only specify one endpoint
per voter. So, we can only support one listener. It's fine if we want to
extend that in this KIP.
15.1 "In this
tage of.
>
> If we enabled keepPreparedTxn=true even without 2PC, then we could enable
> case 3b without the need to fall back to reflection, so we could get rid of
> reflection-based logic and just have a single implementation based on
> KIP-939.
>
> > 32. My sugges
blic APIs
> provided by KIP-939 it'll break.
>
> > 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.
>
> Is this a suggestion to change or confirmation that the current logic is
> ok? I just copied a
Hi, Mickael,
Thanks for the updated KIP.
There is a typo. The KIP says that it adds a new option
compression.snappy.level,
but later says that Snappy is excluded.
Otherwise, the changes look good to me.
Jun
On Wed, Feb 7, 2024 at 6:40 AM Mickael Maison
wrote:
> Hi Divij,
>
> Thanks for
Hi, Apoorv,
Thanks for the KIP. +1
Jun
On Mon, Feb 19, 2024 at 2:32 PM Apoorv Mittal
wrote:
> Hi,
> I’d like to start the voting for KIP-1019: Expose method to determine
> Metric Measurability.
>
>
>
/common/metrics/KafkaMetric.java#L36
> >
> for
> Measurable and Gauge hence went ahead only exposing what is currently
> needed.
>
> Regards,
> Apoorv Mittal
> +44 7721681581
>
>
> On Tue, Feb 20, 2024 at 10:58 PM Jun Rao wrote:
>
> > Hi, Apoorv,
> >
Hi, Apoorv,
Thanks for the KIP.
Could we document how we plan to use the new isMeasurable() method? For
example, gauge is another type of metric. Do we plan to add an isGauge()
method too? If so, is it better to add a separate method for each metric
type or is it better to have a single method
-coordinator-metrics,name=partition-load-time-max.
>
> > 33. "If the value is 'true' then the corresponding field is set
>
> That's correct. Updated the KIP.
>
> -Artem
>
> On Wed, Feb 7, 2024 at 10:06 AM Jun Rao wrote:
>
> > Hi, Artem,
> >
>
for the feedback. Excuse the delay, it took me a while to
> properly address your detailed feedback. See my comments below.
>
> I am going to update the KIP as outlined in this email. I will send
> another email when I have made all of the changes.
>
> On Fri, Feb 2, 2024 at 10
rather than each record individually.
> I could have gone with a callback with a record-based interface. Would
> that be preferable, do you think?
> For one thing, that does give more flexibility for optimisations such as
> fetch pipelining in the future.
>
> 26. The metadata is unused. Th
Hi, Andrew,
Thanks for the KIP. A few comments below.
10. ShareFetchResponse: To consume transactional data, currently
FetchResponse includes the AbortedTransactions fields for the client to
properly skip aborted records. ShareFetchResponse doesn't include that. How
do we prevent the consumer
es and transition the
> producer
>
> Updated the KIP to clarify that IllegalStateException will be thrown.
>
> -Artem
>
>
> On Mon, Feb 5, 2024 at 2:22 PM Jun Rao wrote:
>
> > Hi, Artem,
> >
> > Thanks for the reply.
> >
> > 20. For Flink usage, it se
d we could consider work
> arounds similar to the AddPartitionsToTxn call.
>
> Justine
>
> On Mon, Feb 5, 2024 at 4:56 PM Jun Rao wrote:
>
> > Hi, Justine,
> >
> > Which PRC/record protocols will TV guard? Going forward, will those
> > PRC/record pro
he finalizedFeatures should just reflect the feature
> downgrade we made.
>
> I also plan to write a new KIP for managing the disk format and upgrade
> tool as we will need new flags to support these features. That should help
> clarify some things.
>
> Justine
>
> On Mon, Feb
operations we use the latest "operational" pid and epoch
> known to the client, this way we guarantee that we can fence zombie / split
> brain clients by disrupting the "latest known" pid + epoch progression.
>
> > 25. "We send out markers using the origi
> done.
>
> Thanks,
> Justine
>
> On Fri, Feb 2, 2024 at 11:12 AM Jun Rao wrote:
>
> > Hi, Justine,
> >
> > Thanks for the update.
> >
> > If we ever downgrade the transaction feature, any feature depending on
> > changes on top of those
> COMPLETE
> > producerId: x or z if y overflowed
> > previous/lastProducerId (tagged field): x
> > nextProducerId (tagged field): empty
> > producerEpoch: y + 1 or 0 if we overflowed
> >
> > Thanks again,
> > Justine
> >
> > On Mon, Jan 22, 20
Hi, Jose,
Thanks for the KIP. A few comments below.
10. kraft.version: Functionality wise, this seems very similar to
metadata.version, which is to make sure that all brokers/controllers are on
a supported version before enabling a new feature. Could you explain why we
need a new one instead of
[
https://issues.apache.org/jira/browse/KAFKA-16186?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Jun Rao resolved KAFKA-16186.
-
Fix Version/s: 3.8.0
Resolution: Fixed
merged the PR to trunk
> Implement broker metr
is actually going to be the same way as it is now -- the "main"
> producer id + epoch needs to be used in all operations to prevent fencing
> (it's sort of a common "header" in all RPC calls that follow the same
> rules). The ongoing txn info is just additional
; On Tue, Jan 23, 2024, at 11:21, Jun Rao wrote:
> > Hi, Proven,
> >
> > Thanks for the KIP.
> >
> > I am not sure about the reordering approach proposed in the KIP. Let's
> say
> > in a release we have features X and Y, depending on MV IV1 and IV2,
>
h
> stays the same, no matter how many times the InitProducerId is called
> before the transaction is completed. Eventually the epoch may overflow,
> and then a new producer id would be allocated, but the ongoing transaction
> producer id would stay the same.
>
> I've added a coupl
[
https://issues.apache.org/jira/browse/KAFKA-15813?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Jun Rao resolved KAFKA-15813.
-
Fix Version/s: 3.8.0
Resolution: Fixed
Merged to the PR to trunk.
> Improve implementat
Hi, Proven,
Thanks for the KIP.
I am not sure about the reordering approach proposed in the KIP. Let's say
in a release we have features X and Y, depending on MV IV1 and IV2,
respectively. At the release time, feature Y is ready, but X is not. I
guess the proposal is to move IV1 to a new MV IV3?
pported version errors correctly if we receive them in
> edge cases like upgrades/downgrades.
>
> Justine
>
> On Mon, Jan 22, 2024 at 11:00 AM Jun Rao wrote:
>
> > Hi, Justine,
> >
> > Thanks for the reply.
> >
> > 101.3 I realized that I actually have
[
https://issues.apache.org/jira/browse/KAFKA-16137?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Jun Rao resolved KAFKA-16137.
-
Fix Version/s: 3.8.0
Resolution: Fixed
merged the PR to trunk
still need it. But I am still finalizing the design. I will update
> the KIP once I get the information finalized. Sorry for the delays.
>
> Justine
>
> On Fri, Jan 19, 2024 at 10:50 AM Jun Rao wrote:
>
> > Hi, Justine,
> >
> > Thanks for the reply.
> >
>
> the design for the work to remove the extra add partitions call and I right
> now the design bumps MV. I have yet to update that section as I finalize
> the design so please stay tuned. Was there anything else you thought needed
> MV bump?
>
> Justine
>
> On Thu, Jan 1
[
https://issues.apache.org/jira/browse/KAFKA-15811?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Jun Rao resolved KAFKA-15811.
-
Fix Version/s: 3.8.0
Resolution: Fixed
merged the PR to trunk.
> implement capturing cli
placing the record written on
> InitProducerId. Is this correct?
>
> Thanks,
> Justine
>
> On Tue, Jan 16, 2024 at 5:14 PM Jun Rao wrote:
>
> > Hi, Justine,
> >
> > Thanks for the explanation. I understand the intention now. In the
> overflow
> > ca
ario above would help
> here too.
>
> To be clear, my compatibility story is meant to support downgrades server
> side in keeping the transactional correctness. Keeping the client from
> fencing itself is not the priority.
>
> Hope this helps. I can also add text in the KIP about InitP
restarted and it is ok for it to
> have a new producer ID too).
>
> Justine
>
> On Fri, Jan 12, 2024 at 11:42 AM Jun Rao wrote:
>
> > Hi, Justine,
> >
> > Thanks for the reply.
> >
> > 101.4 "If the marker is written by the new client, we
ide. If the server downgrades, we are still compatible.
> This addresses both the prepare and complete state downgrades.
>
> Justine
>
> On Fri, Jan 12, 2024 at 10:21 AM Jun Rao wrote:
>
> > Hi, Justine,
> >
> > Thanks for the reply. Sorry for the delay. I hav
ds as we do in the transactional state (as we need to
> do this for compatibility -- if we downgrade, we will only have the
> non-tagged fields) It will be the old producer ID and max epoch.
>
> Hope this helps. Let me know if you have further questions.
>
> Justine
>
> On Wed, Dec 20, 2023
Hi, Raman,
Thanks for the KIP. +1 from me.
Jun
On Tue, Dec 26, 2023 at 11:32 AM Raman Verma
wrote:
> I would like to start a Vote on KIP-994
>
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-994%3A+Minor+Enhancements+to+ListTransactions+and+DescribeTransactions+APIs
>
Hi, Artem,
Thanks for the KIP. A few comments below.
10. For the two new fields in Enable2Pc and KeepPreparedTxn in
InitProducerId, it would be useful to document a bit more detail on what
values are set under what cases. For example, are all four combinations
valid?
11.
Hi, Alyssa,
Thanks for the KIP. +1
Jun
On Fri, Dec 8, 2023 at 10:52 AM José Armando García Sancio
wrote:
> +1.
>
> Thanks for the KIP. Looking forward to the implementation!
>
> --
> -José
>
1 - 100 of 5139 matches
Mail list logo