Re: Apache Kafka 3.6.0 release

2023-07-27 Thread Satish Duggana
Hi All,
Whoever has KIP entries in the 3.6.0 release plan. Please update it
with the latest status by tomorrow(end of the day 29th Jul UTC ).

Thanks
Satish.

On Fri, 28 Jul 2023 at 12:01, Satish Duggana  wrote:
>
> Thanks Ismael and Divij for the suggestions.
>
> One way was to follow the earlier guidelines that we set for any early
> access release. It looks Ismael already mentioned the example of
> KRaft.
>
> KIP-405 mentions upgrade/downgrade and limitations sections. We can
> clarify that in the release notes for users on how this feature can be
> used for early access.
>
> Divij, We do not want users to enable this feature on production
> environments in early access release. Let us work together on the
> followups Ismael suggested.
>
> ~Satish.
>
> On Fri, 28 Jul 2023 at 02:24, Divij Vaidya  wrote:
> >
> > Those are great suggestions, thank you. We will continue this discussion
> > forward in a separate KIP for release plan for Tiered Storage.
> >
> > On Thu 27. Jul 2023 at 21:46, Ismael Juma  wrote:
> >
> > > Hi Divij,
> > >
> > > I think the points you bring up for discussion are all good. My main
> > > feedback is that they should be discussed in the context of KIPs vs the
> > > release template. That's why we have a backwards compatibility section for
> > > every KIP, it's precisely to ensure we think carefully about some of the
> > > points you're bringing up. When it comes to defining the meaning of early
> > > access, we have two options:
> > >
> > > 1. Have a KIP specifically for tiered storage.
> > > 2. Have a KIP to define general guidelines for what early access means.
> > >
> > > Does this make sense?
> > >
> > > Ismael
> > >
> > > On Thu, Jul 27, 2023 at 6:38 PM Divij Vaidya 
> > > wrote:
> > >
> > > > Thank you for the response, Ismael.
> > > >
> > > > 1. Specifically in context of 3.6, I wanted this compatibility
> > > > guarantee point to encourage a discussion on
> > > >
> > > >
> > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-952%3A+Regenerate+segment-aligned+producer+snapshots+when+upgrading+to+a+Kafka+version+supporting+Tiered+Storage
> > > > .
> > > > Due to lack of producer snapshots in <2.8 versions, a customer may not
> > > > be able to upgrade to 3.6 and use TS on a topic which was created when
> > > > the cluster was on <2.8 version (see motivation for details). We can
> > > > discuss and agree that it does not break compatibility, which is fine.
> > > > But I want to ensure that we have a discussion soon on this to reach a
> > > > conclusion.
> > > >
> > > > 2. I will start a KIP on this for further discussion.
> > > >
> > > > 3. In the context of 3.6, this would mean that there should be
> > > > no-regression, if a user does "not" turn-on remote storage (early
> > > > access feature) at a cluster level. We have some known cases (such as
> > > > https://issues.apache.org/jira/browse/KAFKA-15189) which violate this
> > > > compatibility requirement. Having this guarantee mentioned in the
> > > > release plan will ensure that we are all in agreement with which cases
> > > > are truly blockers and which aren't.
> > > >
> > > > 4. Fair, instead of a general goal, let me put it specifically in the
> > > > context of 3.6. Let me know if this is not the right forum for this
> > > > discussion.
> > > > Once a user "turns on" tiered storage (TS) at a cluster level, I am
> > > > proposing that they should have the ability to turn it off as well at
> > > > a cluster level. Since this is a topic level feature, folks may not
> > > > spin up a separate cluster to try this feature, hence, we need to
> > > > ensure that we provide them with the ability to try tiered storage for
> > > > a topic which could be deleted and featured turned-off, so that rest
> > > > of the production cases are not impacted.
> > > >
> > > > 5. Agree on not making public interface change as a requirement but we
> > > > should define what "early access" means in that case. Users may not be
> > > > aware that "early access" public APIs may change (unless I am missing
> > > > some documentation somewhere completely, in which case I apologize for
> > > > bringing this naive point).
> > > >
> > > > --
> > > > Divij Vaidya
> > > >
> > > > On Thu, Jul 27, 2023 at 2:27 PM Ismael Juma  wrote:
> > > > >
> > > > > Hi Divij,
> > > > >
> > > > > Some of these are launch checklist items (not really goals) and some
> > > are
> > > > > compatibility guarantees. More below.
> > > > >
> > > > > On Thu, Jul 27, 2023, 12:10 PM Divij Vaidya 
> > > > wrote:
> > > > >
> > > > > > Hey Satish
> > > > > >
> > > > > > Could we consider adding "launch goals" in the release plan. While
> > > > > > some of these may be implicit, it would be nice to list them down in
> > > > > > the release plan. For this release, our launch requirements would 
> > > > > > be:
> > > > > > 1. Users should be able to upgrade from any prior Kafka version to
> > > this
> > > > > > version.
> > > > > >
> > > > >
> > > > > This is part of the compatibili

Re: Apache Kafka 3.6.0 release

2023-07-27 Thread Satish Duggana
Thanks Ismael and Divij for the suggestions.

One way was to follow the earlier guidelines that we set for any early
access release. It looks Ismael already mentioned the example of
KRaft.

KIP-405 mentions upgrade/downgrade and limitations sections. We can
clarify that in the release notes for users on how this feature can be
used for early access.

Divij, We do not want users to enable this feature on production
environments in early access release. Let us work together on the
followups Ismael suggested.

~Satish.

On Fri, 28 Jul 2023 at 02:24, Divij Vaidya  wrote:
>
> Those are great suggestions, thank you. We will continue this discussion
> forward in a separate KIP for release plan for Tiered Storage.
>
> On Thu 27. Jul 2023 at 21:46, Ismael Juma  wrote:
>
> > Hi Divij,
> >
> > I think the points you bring up for discussion are all good. My main
> > feedback is that they should be discussed in the context of KIPs vs the
> > release template. That's why we have a backwards compatibility section for
> > every KIP, it's precisely to ensure we think carefully about some of the
> > points you're bringing up. When it comes to defining the meaning of early
> > access, we have two options:
> >
> > 1. Have a KIP specifically for tiered storage.
> > 2. Have a KIP to define general guidelines for what early access means.
> >
> > Does this make sense?
> >
> > Ismael
> >
> > On Thu, Jul 27, 2023 at 6:38 PM Divij Vaidya 
> > wrote:
> >
> > > Thank you for the response, Ismael.
> > >
> > > 1. Specifically in context of 3.6, I wanted this compatibility
> > > guarantee point to encourage a discussion on
> > >
> > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-952%3A+Regenerate+segment-aligned+producer+snapshots+when+upgrading+to+a+Kafka+version+supporting+Tiered+Storage
> > > .
> > > Due to lack of producer snapshots in <2.8 versions, a customer may not
> > > be able to upgrade to 3.6 and use TS on a topic which was created when
> > > the cluster was on <2.8 version (see motivation for details). We can
> > > discuss and agree that it does not break compatibility, which is fine.
> > > But I want to ensure that we have a discussion soon on this to reach a
> > > conclusion.
> > >
> > > 2. I will start a KIP on this for further discussion.
> > >
> > > 3. In the context of 3.6, this would mean that there should be
> > > no-regression, if a user does "not" turn-on remote storage (early
> > > access feature) at a cluster level. We have some known cases (such as
> > > https://issues.apache.org/jira/browse/KAFKA-15189) which violate this
> > > compatibility requirement. Having this guarantee mentioned in the
> > > release plan will ensure that we are all in agreement with which cases
> > > are truly blockers and which aren't.
> > >
> > > 4. Fair, instead of a general goal, let me put it specifically in the
> > > context of 3.6. Let me know if this is not the right forum for this
> > > discussion.
> > > Once a user "turns on" tiered storage (TS) at a cluster level, I am
> > > proposing that they should have the ability to turn it off as well at
> > > a cluster level. Since this is a topic level feature, folks may not
> > > spin up a separate cluster to try this feature, hence, we need to
> > > ensure that we provide them with the ability to try tiered storage for
> > > a topic which could be deleted and featured turned-off, so that rest
> > > of the production cases are not impacted.
> > >
> > > 5. Agree on not making public interface change as a requirement but we
> > > should define what "early access" means in that case. Users may not be
> > > aware that "early access" public APIs may change (unless I am missing
> > > some documentation somewhere completely, in which case I apologize for
> > > bringing this naive point).
> > >
> > > --
> > > Divij Vaidya
> > >
> > > On Thu, Jul 27, 2023 at 2:27 PM Ismael Juma  wrote:
> > > >
> > > > Hi Divij,
> > > >
> > > > Some of these are launch checklist items (not really goals) and some
> > are
> > > > compatibility guarantees. More below.
> > > >
> > > > On Thu, Jul 27, 2023, 12:10 PM Divij Vaidya 
> > > wrote:
> > > >
> > > > > Hey Satish
> > > > >
> > > > > Could we consider adding "launch goals" in the release plan. While
> > > > > some of these may be implicit, it would be nice to list them down in
> > > > > the release plan. For this release, our launch requirements would be:
> > > > > 1. Users should be able to upgrade from any prior Kafka version to
> > this
> > > > > version.
> > > > >
> > > >
> > > > This is part of the compatibility guarantees. The upgrade notes mention
> > > > this already. If there is a change in a given release, it should
> > > definitely
> > > > be highlighted.
> > > >
> > > > 2. On release, this version (or it's dependencies) would not have any
> > > > > known MEDIUM/HIGH CVE.
> > > > >
> > > >
> > > > This is a new policy and the details should be discussed. In
> > particular,
> > > > the threshold (medium or high).
> > > >
> > > > 3. Pres

[jira] [Created] (KAFKA-15265) Remote copy/fetch quotas for tiered storage.

2023-07-27 Thread Satish Duggana (Jira)
Satish Duggana created KAFKA-15265:
--

 Summary: Remote copy/fetch quotas for tiered storage.
 Key: KAFKA-15265
 URL: https://issues.apache.org/jira/browse/KAFKA-15265
 Project: Kafka
  Issue Type: Improvement
  Components: core
Reporter: Satish Duggana
Assignee: Abhijeet Kumar






--
This message was sent by Atlassian Jira
(v8.20.10#820010)


Re: [DISCUSS] KIP-953: partition method to be overloaded to accept headers as well.

2023-07-27 Thread Sagar
Hey Andrew,

Thanks for the review. Since I had reviewed the KIP I thought I would also
respond. Of course Jack has the final say on this since he wrote the KIP.

1) This is an interesting point and I hadn't considered it. The
comparison with KIP-848 is a valid one but even within that KIP, it allows
client side partitioning for power users like Streams. So while we would
want to move away from client side partitioner as much as possible, we
still shouldn't do away completely with Client side partitioning and end up
being in a state of inflexibility for different kinds of usecases. This is
my opinion though and you have more context on Clients, so would like to
know your thoughts on this.

2) Regarding this, I assumed that since the headers are already part of the
consumer records they should have access to the headers and if there is a
contract b/w the applications producing and the application consuming, that
decisioning should be transparent. Was my assumption incorrect? But as you
rightly pointed out header based partitioning with keys is going to lead to
surprising results. Assuming there is merit in this proposal, do you think
we should ignore the keys in this case (similar to the effect of
setting *partitioner.ignore.keys
*config to false) and document it appropriately?

Let me know what you think.

Thanks!
Sagar.


On Thu, Jul 27, 2023 at 9:41 PM Andrew Schofield <
andrew_schofield_j...@outlook.com> wrote:

> Hi Jack,
> Thanks for the KIP. I have a few concerns about the idea.
>
> 1) I think that while a client-side partitioner seems like a neat idea and
> it’s an established part of Kafka,
> it’s one of the things which makes Kafka clients quite complicated. Just
> as KIP-848 is moving from
> client-side assignors to server-side assignors, I wonder whether really we
> should be looking to make
> partitioning a server-side capability too over time. So, I’m not convinced
> that making the Partitioner
> interface richer is moving in the right direction.
>
> 2) For records with a key, the partitioner usually calculates the
> partition from the key. This means
> that records with the same key end up on the same partition. Many
> applications expect this to give ordering.
> Log compaction expects this. There are situations in which records have to
> be repartitioned, such as
> sometimes happens with Kafka Streams. I think that a header-based
> partitioner for records which have
> keys is going to be surprising and only going to have limited
> applicability as a result.
>
> The tricky part about clever partitioning is that downstream systems have
> no idea how the partition
> number was arrived at, so they do not truly understand how the ordering
> was derived. I do think that
> perhaps there’s value to being able to influence the partitioning using
> the headers, but I wonder if actually
> transforming the headers into an “ordering context” that then flows with
> the record as it moves through
> the system would be a stronger solution. Today, the key is the ordering
> context. Maybe it should be a
> concept in its own right and the Producer could configure a converter from
> headers to ordering context.
> That is of course a much bigger change.
>
> In one of the examples you mention in the KIP, you mentioned using a
> header to control priority. I guess the
> idea is to preferentially process records off specific partitions so they
> can overtake lower priority records.
> I suggest just sending the records explicitly to specific partitions to
> achieve this.
>
> Sorry for the essay, but you did ask for people to share their thoughts :)
>
> Just my opinion. Let’s see what others think.
>
> Thanks,
> Andrew
>
> > On 25 Jul 2023, at 14:58, Jack Tomy  wrote:
> >
> > Hey @Sagar
> >
> > Thanks again for the review.
> > 1. "a null headers value is equivalent to invoking the older partition
> > method", this is not true. If someone makes an implementation and the
> > headers come as null, still the new implementation will take effect.
> > Instead I have added : "Not overriding this method in the Partitioner
> > interface has the same behaviour as using the existing method."
> > 2. Corrected.
> >
> > Hey @Sagar and everyone,
> > Please have a look at the new version and share your thoughts.
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=263424937
> > Like Sagar mentioned, I would also request more people who have more
> > context on clients to chime in.
> >
> >
> > On Tue, Jul 25, 2023 at 2:58 PM Sagar  wrote:
> >
> >> Hi Jack,
> >>
> >> Thanks I have a couple of final comments and then I am good.
> >>
> >> 1) Can you elaborate on the Javadocs of the partition headers argument
> to
> >> specify that a null headers value is equivalent to invoking the older
> >> partition method? It is apparent but generally good to call out.
> >> 2) In the Compatibility section, you have mentioned backward
> comparable. I
> >> believe it should be *backward compatible change.*
> >>
> >> I don't have other 

Jenkins build is unstable: Kafka » Kafka Branch Builder » trunk #2050

2023-07-27 Thread Apache Jenkins Server
See 




Re: [VOTE] KIP-919: Allow AdminClient to Talk Directly with the KRaft Controller Quorum and add Controller Registration

2023-07-27 Thread Ismael Juma
Sounds good to me.

Ismael

On Thu, Jul 27, 2023, 6:32 PM Colin McCabe  wrote:

> I would incline towards plural, but I honestly don't feel that strongly
> about it. I will just change it to --bootstrap-controller for now, to match
> --bootstrap-server. Perhaps we can discuss this further in a later KIP, if
> people are inclined...
>
> Colin
>
>
> On Thu, Jul 27, 2023, at 07:13, Ron Dagostino wrote:
> >> support plural for both and the singular one is just an alias for
> compatibility
> >
> > I like that approach.  I can never remember what works, so I would
> > prefer to just use what I think works and then have it work.
> >
> > Ron
> >
> > On Thu, Jul 27, 2023 at 8:33 AM Ismael Juma  wrote:
> >>
> >> I think singular was used because that's the common case for the cli
> tools.
> >> To be honest, it's actually a bit more confusing to have both singular
> and
> >> plural for the cli tools and you have to remember the exact version for
> >> each one. We should either support plural for both and the singular one
> is
> >> just an alias for compatibility or stick with singular for both.
> >>
> >> Ismael
> >>
> >> On Thu, Jul 27, 2023, 12:03 AM Colin McCabe  wrote:
> >>
> >> > On Wed, Jul 26, 2023, at 07:09, Ron Dagostino wrote:
> >> > > Thanks, Colin.  +1 (binding) from me.
> >> > >
> >> > > I will note that Ziming mentioned in the DISCUSS thread that "There
> is
> >> > > a mistake that we use `--bootstrap-server` instead of
> >> > > `--bootstrap-server(s)`, so should we also change the new argument
> >> > > `--bootstrap-controller` (no s).".  I agree that this is an
> >> > > unfortunate historical artifact.  Furthermore, the config property
> is
> >> > > bootstrap.servers, which does not match the command line argument
> >> > > --bootstrap-server.  I don't know what we should do here -- I am not
> >> > > sure that we should propagate it.  We could continue the same
> mismatch
> >> > > and use --bootstrap-controller, but I feel like using
> >> > > --bootstrap-controllers makes more sense.  Anyway, this is more an
> >> > > ergonomic issue than anything else.  I am still +1 binding, but we
> >> > > should arrive at an explicit decision here.
> >> >
> >> > Hi Ron,
> >> >
> >> > Thanks for the review and vote.
> >> >
> >> > The current KIP-919 text has "--bootstrap-controllers" for
> command-line
> >> > and "bootstrap.controllers" for config.
> >> >
> >> > As far as I know, --bootstrap-server is singular rather than plural
> just
> >> > for historical reasons so I figured the new flag can just be
> plural. As
> >> > you mentioned, the configs have always been plural, so it's more
> consistent
> >> > for the flags to match the configs.
> >> >
> >> > (Though obviously that ship has sailed on --bootstrap-server, we're
> not
> >> > gonna change it now ;)
> >> >
> >> > >
> >> > > Nits:
> >> > > s/boostrap-servers/bootstrap-servers/
> >> > > s/DescribeClusterResquest/DescribeClusterRequest/
> >> > > s/ControllerRegistrationRecord/RegisterControllerRecord/g
> >> > >
> >> >
> >> > Thanks. I fixed these typos and changed ControllerRegistrationRecord
> ->
> >> > RegisterControllerRecord as suggested.
> >> >
> >> > best,
> >> > Colin
> >> >
> >> >
> >> > > Ron
> >> > >
> >> > > On Wed, Jul 26, 2023 at 10:06 AM David Arthur
> >> > >  wrote:
> >> > >>
> >> > >> Thanks for driving this KIP, Colin!
> >> > >>
> >> > >> +1 binding
> >> > >>
> >> > >> -David
> >> > >>
> >> > >> On Wed, Jul 26, 2023 at 8:58 AM Divij Vaidya <
> divijvaidy...@gmail.com>
> >> > >> wrote:
> >> > >>
> >> > >> > +1 (binding)
> >> > >> >
> >> > >> > --
> >> > >> > Divij Vaidya
> >> > >> >
> >> > >> >
> >> > >> > On Wed, Jul 26, 2023 at 2:56 PM ziming deng <
> dengziming1...@gmail.com
> >> > >
> >> > >> > wrote:
> >> > >> > >
> >> > >> > > +1 (binding) from me.
> >> > >> > >
> >> > >> > > Thanks for the KIP!
> >> > >> > >
> >> > >> > > --
> >> > >> > > Ziming
> >> > >> > >
> >> > >> > > > On Jul 26, 2023, at 20:18, Luke Chen 
> wrote:
> >> > >> > > >
> >> > >> > > > +1 (binding) from me.
> >> > >> > > >
> >> > >> > > > Thanks for the KIP!
> >> > >> > > >
> >> > >> > > > Luke
> >> > >> > > >
> >> > >> > > > On Tue, Jul 25, 2023 at 1:24 AM Colin McCabe <
> cmcc...@apache.org>
> >> > >> > wrote:
> >> > >> > > >
> >> > >> > > >> Hi all,
> >> > >> > > >>
> >> > >> > > >> I'd like to start the vote for KIP-919: Allow AdminClient
> to Talk
> >> > >> > Directly
> >> > >> > > >> with the KRaft Controller Quorum and add Controller
> Registration.
> >> > >> > > >>
> >> > >> > > >> The KIP is here:
> https://cwiki.apache.org/confluence/x/Owo0Dw
> >> > >> > > >>
> >> > >> > > >> Thanks to everyone who reviewed the proposal.
> >> > >> > > >>
> >> > >> > > >> best,
> >> > >> > > >> Colin
> >> > >> > > >>
> >> > >> > >
> >> > >> >
> >> > >>
> >> > >>
> >> > >> --
> >> > >> -David
> >> >
>


[jira] [Created] (KAFKA-15264) Compared with 1.1.0zk, the peak throughput of 3.5.1kraft is very jitter

2023-07-27 Thread jianbin.chen (Jira)
jianbin.chen created KAFKA-15264:


 Summary: Compared with 1.1.0zk, the peak throughput of 3.5.1kraft 
is very jitter
 Key: KAFKA-15264
 URL: https://issues.apache.org/jira/browse/KAFKA-15264
 Project: Kafka
  Issue Type: Bug
Reporter: jianbin.chen


I was preparing to upgrade from 1.1.0 to 3.5.1's kraft mode (new cluster 
deployment), and when I recently compared and tested, I found that when using 
the following stress test command, the throughput gap is obvious

 
{code:java}
./kafka-producer-perf-test.sh --topic test321 --num-records 3000 
--record-size 1024 --throughput -1 --producer-props bootstrap.servers=xxx: 
acks=1
419813 records sent, 83962.6 records/sec (81.99 MB/sec), 241.1 ms avg latency, 
588.0 ms max latency.
555300 records sent, 111015.6 records/sec (108.41 MB/sec), 275.1 ms avg 
latency, 460.0 ms max latency.
552795 records sent, 110536.9 records/sec (107.95 MB/sec), 265.9 ms avg 
latency, 1120.0 ms max latency.
552600 records sent, 110520.0 records/sec (107.93 MB/sec), 284.5 ms avg 
latency, 1097.0 ms max latency.
538500 records sent, 107656.9 records/sec (105.13 MB/sec), 277.5 ms avg 
latency, 610.0 ms max latency.
511545 records sent, 102309.0 records/sec (99.91 MB/sec), 304.1 ms avg latency, 
1892.0 ms max latency.
511890 records sent, 102337.1 records/sec (99.94 MB/sec), 288.4 ms avg latency, 
3000.0 ms max latency.
519165 records sent, 103812.2 records/sec (101.38 MB/sec), 262.1 ms avg 
latency, 1781.0 ms max latency.
513555 records sent, 102669.9 records/sec (100.26 MB/sec), 338.2 ms avg 
latency, 2590.0 ms max latency.
463329 records sent, 92665.8 records/sec (90.49 MB/sec), 276.8 ms avg latency, 
1463.0 ms max latency.
494248 records sent, 98849.6 records/sec (96.53 MB/sec), 327.2 ms avg latency, 
2362.0 ms max latency.
506272 records sent, 101254.4 records/sec (98.88 MB/sec), 322.1 ms avg latency, 
2986.0 ms max latency.
393758 records sent, 78735.9 records/sec (76.89 MB/sec), 387.0 ms avg latency, 
2958.0 ms max latency.
426435 records sent, 85252.9 records/sec (83.25 MB/sec), 363.3 ms avg latency, 
1959.0 ms max latency.
412560 records sent, 82298.0 records/sec (80.37 MB/sec), 374.1 ms avg latency, 
1995.0 ms max latency.
370137 records sent, 73997.8 records/sec (72.26 MB/sec), 396.8 ms avg latency, 
1496.0 ms max latency.
391781 records sent, 78340.5 records/sec (76.50 MB/sec), 410.7 ms avg latency, 
2446.0 ms max latency.
355901 records sent, 71166.0 records/sec (69.50 MB/sec), 397.5 ms avg latency, 
2715.0 ms max latency.
385410 records sent, 77082.0 records/sec (75.28 MB/sec), 417.5 ms avg latency, 
2702.0 ms max latency.
381160 records sent, 76232.0 records/sec (74.45 MB/sec), 407.7 ms avg latency, 
1846.0 ms max latency.
67 records sent, 0.1 records/sec (65.10 MB/sec), 456.2 ms avg latency, 
1414.0 ms max latency.
376251 records sent, 75175.0 records/sec (73.41 MB/sec), 401.9 ms avg latency, 
1897.0 ms max latency.
354434 records sent, 70886.8 records/sec (69.23 MB/sec), 425.8 ms avg latency, 
1601.0 ms max latency.
353795 records sent, 70744.9 records/sec (69.09 MB/sec), 411.7 ms avg latency, 
1563.0 ms max latency.
321993 records sent, 64360.0 records/sec (62.85 MB/sec), 447.3 ms avg latency, 
1975.0 ms max latency.
404075 records sent, 80750.4 records/sec (78.86 MB/sec), 408.4 ms avg latency, 
1753.0 ms max latency.
384526 records sent, 76905.2 records/sec (75.10 MB/sec), 406.0 ms avg latency, 
1833.0 ms max latency.
387652 records sent, 77483.9 records/sec (75.67 MB/sec), 397.3 ms avg latency, 
1927.0 ms max latency.
343286 records sent, 68629.7 records/sec (67.02 MB/sec), 455.6 ms avg latency, 
1685.0 ms max latency.
00 records sent, 66646.7 records/sec (65.08 MB/sec), 456.6 ms avg latency, 
2146.0 ms max latency.
361191 records sent, 72238.2 records/sec (70.55 MB/sec), 409.4 ms avg latency, 
2125.0 ms max latency.
357525 records sent, 71490.7 records/sec (69.82 MB/sec), 436.0 ms avg latency, 
1502.0 ms max latency.
340238 records sent, 68047.6 records/sec (66.45 MB/sec), 427.9 ms avg latency, 
1932.0 ms max latency.
390016 records sent, 77956.4 records/sec (76.13 MB/sec), 418.5 ms avg latency, 
1807.0 ms max latency.
352830 records sent, 70523.7 records/sec (68.87 MB/sec), 439.4 ms avg latency, 
1892.0 ms max latency.
354526 records sent, 70905.2 records/sec (69.24 MB/sec), 429.6 ms avg latency, 
2128.0 ms max latency.
356670 records sent, 71305.5 records/sec (69.63 MB/sec), 408.9 ms avg latency, 
1329.0 ms max latency.
309204 records sent, 60687.7 records/sec (59.27 MB/sec), 438.6 ms avg latency, 
2566.0 ms max latency.
366715 records sent, 72316.1 records/sec (70.62 MB/sec), 474.5 ms avg latency, 
2169.0 ms max latency.
375174 records sent, 75034.8 records/sec (73.28 MB/sec), 429.9 ms avg latency, 
1722.0 ms max latency.
359400 records sent, 70346.4 records/sec (68.70 MB/sec), 432.1 ms avg latency, 
1961.0 ms max latency.
312276 rec

Build failed in Jenkins: Kafka » Kafka Branch Builder » trunk #2049

2023-07-27 Thread Apache Jenkins Server
See 


Changes:


--
[...truncated 296882 lines...]

Gradle Test Run :streams:integrationTest > Gradle Test Executor 182 > 
TableTableJoinIntegrationTest > [caching enabled = false] > 
testInnerLeft[caching enabled = false] STARTED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 182 > 
TableTableJoinIntegrationTest > [caching enabled = false] > 
testInnerLeft[caching enabled = false] PASSED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 182 > 
TableTableJoinIntegrationTest > [caching enabled = false] > 
testOuterInner[caching enabled = false] STARTED

> Task :core:integrationTest

Gradle Test Run :core:integrationTest > Gradle Test Executor 178 > 
AllocateProducerIdsRequestTest > testAllocateProducersIdSentToNonController() > 
[1] Type=Raft-Isolated, Name=testAllocateProducersIdSentToNonController, 
MetadataVersion=3.6-IV0, Security=PLAINTEXT PASSED

> Task :streams:integrationTest

Gradle Test Run :streams:integrationTest > Gradle Test Executor 182 > 
TableTableJoinIntegrationTest > [caching enabled = false] > 
testOuterInner[caching enabled = false] PASSED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 182 > 
TableTableJoinIntegrationTest > [caching enabled = false] > 
testOuterOuter[caching enabled = false] STARTED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 182 > 
TableTableJoinIntegrationTest > [caching enabled = false] > 
testOuterOuter[caching enabled = false] PASSED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 182 > 
TableTableJoinIntegrationTest > [caching enabled = false] > 
testInnerWithRightVersionedOnly[caching enabled = false] STARTED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 182 > 
TableTableJoinIntegrationTest > [caching enabled = false] > 
testInnerWithRightVersionedOnly[caching enabled = false] PASSED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 182 > 
TableTableJoinIntegrationTest > [caching enabled = false] > 
testLeftWithLeftVersionedOnly[caching enabled = false] STARTED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 182 > 
TableTableJoinIntegrationTest > [caching enabled = false] > 
testLeftWithLeftVersionedOnly[caching enabled = false] PASSED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 182 > 
TableTableJoinIntegrationTest > [caching enabled = false] > 
testInnerWithLeftVersionedOnly[caching enabled = false] STARTED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 182 > 
TableTableJoinIntegrationTest > [caching enabled = false] > 
testInnerWithLeftVersionedOnly[caching enabled = false] PASSED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 182 > 
TaskAssignorIntegrationTest > shouldProperlyConfigureTheAssignor STARTED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 182 > 
TaskAssignorIntegrationTest > shouldProperlyConfigureTheAssignor PASSED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 182 > 
TaskMetadataIntegrationTest > shouldReportCorrectEndOffsetInformation STARTED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 182 > 
TaskMetadataIntegrationTest > shouldReportCorrectEndOffsetInformation PASSED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 182 > 
TaskMetadataIntegrationTest > shouldReportCorrectCommittedOffsetInformation 
STARTED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 182 > 
TaskMetadataIntegrationTest > shouldReportCorrectCommittedOffsetInformation 
PASSED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 182 > 
EmitOnChangeIntegrationTest > shouldEmitSameRecordAfterFailover() STARTED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 182 > 
EmitOnChangeIntegrationTest > shouldEmitSameRecordAfterFailover() PASSED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 182 > 
HighAvailabilityTaskAssignorIntegrationTest > 
shouldScaleOutWithWarmupTasksAndPersistentStores(TestInfo) STARTED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 182 > 
HighAvailabilityTaskAssignorIntegrationTest > 
shouldScaleOutWithWarmupTasksAndPersistentStores(TestInfo) PASSED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 182 > 
HighAvailabilityTaskAssignorIntegrationTest > 
shouldScaleOutWithWarmupTasksAndInMemoryStores(TestInfo) STARTED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 182 > 
HighAvailabilityTaskAssignorIntegrationTest > 
shouldScaleOutWithWarmupTasksAndInMemoryStores(TestInfo) PASSED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 182 > 
KStreamAggregationDedupIntegrationTest > shouldReduce(TestInfo) STARTED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 182 > 
KStreamAggregationDedupIntegrationTest > shouldReduce(TestInfo) PASSED

Gradle Te

Jenkins build is unstable: Kafka » Kafka Branch Builder » trunk #2048

2023-07-27 Thread Apache Jenkins Server
See 




Re: Apache Kafka 3.6.0 release

2023-07-27 Thread Divij Vaidya
Those are great suggestions, thank you. We will continue this discussion
forward in a separate KIP for release plan for Tiered Storage.

On Thu 27. Jul 2023 at 21:46, Ismael Juma  wrote:

> Hi Divij,
>
> I think the points you bring up for discussion are all good. My main
> feedback is that they should be discussed in the context of KIPs vs the
> release template. That's why we have a backwards compatibility section for
> every KIP, it's precisely to ensure we think carefully about some of the
> points you're bringing up. When it comes to defining the meaning of early
> access, we have two options:
>
> 1. Have a KIP specifically for tiered storage.
> 2. Have a KIP to define general guidelines for what early access means.
>
> Does this make sense?
>
> Ismael
>
> On Thu, Jul 27, 2023 at 6:38 PM Divij Vaidya 
> wrote:
>
> > Thank you for the response, Ismael.
> >
> > 1. Specifically in context of 3.6, I wanted this compatibility
> > guarantee point to encourage a discussion on
> >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-952%3A+Regenerate+segment-aligned+producer+snapshots+when+upgrading+to+a+Kafka+version+supporting+Tiered+Storage
> > .
> > Due to lack of producer snapshots in <2.8 versions, a customer may not
> > be able to upgrade to 3.6 and use TS on a topic which was created when
> > the cluster was on <2.8 version (see motivation for details). We can
> > discuss and agree that it does not break compatibility, which is fine.
> > But I want to ensure that we have a discussion soon on this to reach a
> > conclusion.
> >
> > 2. I will start a KIP on this for further discussion.
> >
> > 3. In the context of 3.6, this would mean that there should be
> > no-regression, if a user does "not" turn-on remote storage (early
> > access feature) at a cluster level. We have some known cases (such as
> > https://issues.apache.org/jira/browse/KAFKA-15189) which violate this
> > compatibility requirement. Having this guarantee mentioned in the
> > release plan will ensure that we are all in agreement with which cases
> > are truly blockers and which aren't.
> >
> > 4. Fair, instead of a general goal, let me put it specifically in the
> > context of 3.6. Let me know if this is not the right forum for this
> > discussion.
> > Once a user "turns on" tiered storage (TS) at a cluster level, I am
> > proposing that they should have the ability to turn it off as well at
> > a cluster level. Since this is a topic level feature, folks may not
> > spin up a separate cluster to try this feature, hence, we need to
> > ensure that we provide them with the ability to try tiered storage for
> > a topic which could be deleted and featured turned-off, so that rest
> > of the production cases are not impacted.
> >
> > 5. Agree on not making public interface change as a requirement but we
> > should define what "early access" means in that case. Users may not be
> > aware that "early access" public APIs may change (unless I am missing
> > some documentation somewhere completely, in which case I apologize for
> > bringing this naive point).
> >
> > --
> > Divij Vaidya
> >
> > On Thu, Jul 27, 2023 at 2:27 PM Ismael Juma  wrote:
> > >
> > > Hi Divij,
> > >
> > > Some of these are launch checklist items (not really goals) and some
> are
> > > compatibility guarantees. More below.
> > >
> > > On Thu, Jul 27, 2023, 12:10 PM Divij Vaidya 
> > wrote:
> > >
> > > > Hey Satish
> > > >
> > > > Could we consider adding "launch goals" in the release plan. While
> > > > some of these may be implicit, it would be nice to list them down in
> > > > the release plan. For this release, our launch requirements would be:
> > > > 1. Users should be able to upgrade from any prior Kafka version to
> this
> > > > version.
> > > >
> > >
> > > This is part of the compatibility guarantees. The upgrade notes mention
> > > this already. If there is a change in a given release, it should
> > definitely
> > > be highlighted.
> > >
> > > 2. On release, this version (or it's dependencies) would not have any
> > > > known MEDIUM/HIGH CVE.
> > > >
> > >
> > > This is a new policy and the details should be discussed. In
> particular,
> > > the threshold (medium or high).
> > >
> > > 3. Presence of any "early access"/"beta" feature should not impact
> > > > other production features when it is not enabled.
> > > >
> > >
> > > This is a general guideline for early access features and not specific
> to
> > > this release. It would be good to have a page that talks about these
> > things.
> > >
> > > 4. Once enabled, users should have an option to disable any "early
> > > > access"/"beta" feature and resume normal production features, i.e.
> > > > impact of beta features should be reversible.
> > > >
> > >
> > > This needs discussion and I don't think it's reasonable as a general
> > rule.
> > > For example, Kraft early access wasn't reversible and it was not
> feasible
> > > for it to be.
> > >
> > > 5. KIP-405 will be available in "early access"/"beta" mode. 

Re: Apache Kafka 3.6.0 release

2023-07-27 Thread Ismael Juma
Hi Divij,

I think the points you bring up for discussion are all good. My main
feedback is that they should be discussed in the context of KIPs vs the
release template. That's why we have a backwards compatibility section for
every KIP, it's precisely to ensure we think carefully about some of the
points you're bringing up. When it comes to defining the meaning of early
access, we have two options:

1. Have a KIP specifically for tiered storage.
2. Have a KIP to define general guidelines for what early access means.

Does this make sense?

Ismael

On Thu, Jul 27, 2023 at 6:38 PM Divij Vaidya 
wrote:

> Thank you for the response, Ismael.
>
> 1. Specifically in context of 3.6, I wanted this compatibility
> guarantee point to encourage a discussion on
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-952%3A+Regenerate+segment-aligned+producer+snapshots+when+upgrading+to+a+Kafka+version+supporting+Tiered+Storage
> .
> Due to lack of producer snapshots in <2.8 versions, a customer may not
> be able to upgrade to 3.6 and use TS on a topic which was created when
> the cluster was on <2.8 version (see motivation for details). We can
> discuss and agree that it does not break compatibility, which is fine.
> But I want to ensure that we have a discussion soon on this to reach a
> conclusion.
>
> 2. I will start a KIP on this for further discussion.
>
> 3. In the context of 3.6, this would mean that there should be
> no-regression, if a user does "not" turn-on remote storage (early
> access feature) at a cluster level. We have some known cases (such as
> https://issues.apache.org/jira/browse/KAFKA-15189) which violate this
> compatibility requirement. Having this guarantee mentioned in the
> release plan will ensure that we are all in agreement with which cases
> are truly blockers and which aren't.
>
> 4. Fair, instead of a general goal, let me put it specifically in the
> context of 3.6. Let me know if this is not the right forum for this
> discussion.
> Once a user "turns on" tiered storage (TS) at a cluster level, I am
> proposing that they should have the ability to turn it off as well at
> a cluster level. Since this is a topic level feature, folks may not
> spin up a separate cluster to try this feature, hence, we need to
> ensure that we provide them with the ability to try tiered storage for
> a topic which could be deleted and featured turned-off, so that rest
> of the production cases are not impacted.
>
> 5. Agree on not making public interface change as a requirement but we
> should define what "early access" means in that case. Users may not be
> aware that "early access" public APIs may change (unless I am missing
> some documentation somewhere completely, in which case I apologize for
> bringing this naive point).
>
> --
> Divij Vaidya
>
> On Thu, Jul 27, 2023 at 2:27 PM Ismael Juma  wrote:
> >
> > Hi Divij,
> >
> > Some of these are launch checklist items (not really goals) and some are
> > compatibility guarantees. More below.
> >
> > On Thu, Jul 27, 2023, 12:10 PM Divij Vaidya 
> wrote:
> >
> > > Hey Satish
> > >
> > > Could we consider adding "launch goals" in the release plan. While
> > > some of these may be implicit, it would be nice to list them down in
> > > the release plan. For this release, our launch requirements would be:
> > > 1. Users should be able to upgrade from any prior Kafka version to this
> > > version.
> > >
> >
> > This is part of the compatibility guarantees. The upgrade notes mention
> > this already. If there is a change in a given release, it should
> definitely
> > be highlighted.
> >
> > 2. On release, this version (or it's dependencies) would not have any
> > > known MEDIUM/HIGH CVE.
> > >
> >
> > This is a new policy and the details should be discussed. In particular,
> > the threshold (medium or high).
> >
> > 3. Presence of any "early access"/"beta" feature should not impact
> > > other production features when it is not enabled.
> > >
> >
> > This is a general guideline for early access features and not specific to
> > this release. It would be good to have a page that talks about these
> things.
> >
> > 4. Once enabled, users should have an option to disable any "early
> > > access"/"beta" feature and resume normal production features, i.e.
> > > impact of beta features should be reversible.
> > >
> >
> > This needs discussion and I don't think it's reasonable as a general
> rule.
> > For example, Kraft early access wasn't reversible and it was not feasible
> > for it to be.
> >
> > 5. KIP-405 will be available in "early access"/"beta" mode. Early
> > > access/beta means that the public facing interfaces won't change in
> > > future but the implementation is not recommended to be used in
> > > production.
> >
> >
> > I don't think it's ok to make this a requirement. Early access is a way
> to
> > get early feedback and all types of changes should be on the table. They
> > would be discussed via KIPs as usual. I believe there were some
> > incompatible ch

Re: [DISCUSS] KIP-954: expand default DSL store configuration to custom types

2023-07-27 Thread Guozhang Wang
Yes, that sounds right to me. Thanks Sophie.

On Thu, Jul 27, 2023 at 12:35 PM Sophie Blee-Goldman
 wrote:
>
> A2: Guozhang, just to close the book on the ListValue store thing, I fully
> agree it seems like overreach
> to expose/force this on users, especially if it's fully internal today. But
> just to make sure we're on the same
> page here, you're still ok with this KIP fixing the API gap that exists
> today, in which these stores cannot be
> customized by the user at all?
>
> In other words, after this KIP, the new behavior for the ListValue store in
> a stream join will be:
>
> S1: First, check if the user passed in a `DSLStoreSuppliers` (or whatever
> the name will be) to the
>StreamJoined config object, and use that to obtain the
> KVStoreSupplier for this ListValue store
>
> S2: If none was provided, check if the user has set a default
> DSLStoreSuppliers via the new config,
>and use that to get the KVStoreSupplier if so
>
> S3: If neither is set, fall back to the original logic as it is today,
> which is to pass in a KVStoreSupplier
>that is hard-coded to be either RocksDB or InMemory, based on what
> is returned for the #persistent
>API by the StreamJoined's WindowStoreSupplier
>
> Does that sound right? We can clarify this further in the KIP if need be
>
> On Thu, Jul 27, 2023 at 10:48 AM Guozhang Wang 
> wrote:
>
> > Hi all,
> >
> > Like Almog's secretary as well! Just following up on that index:
> >
> > A2: I'm also happy without introducing versioned KV in this KIP as I
> > would envision it to be introduced as new params into the
> > KeyValuePluginParams in the future. And just to clarify on Sophie's
> > previous comment, I think ListStore should not be exposed in this API
> > until we see it as a common usage and hence would want to (again, we
> > need to think very carefully since it would potentially ask all
> > implementers to adopt) move it from the internal category to the
> > public interface category. As for now, I think only having kv / window
> > / session as public store types is fine.
> >
> > A3: Seems I was not making myself very clear at the beginning :P The
> > major thing that I'd actually like to avoid having two configs
> > co-exist for the same function since it will be a confusing learning
> > curve for users, and hence what I was proposing is to just have the
> > newly introduced interface but not introducing a new config, and I
> > realized now that it is actually more aligned with the CUSTOM idea
> > where the ordering would be looking at config first, and then the
> > interface. I blushed as I read Almog likes it.. After thinking about
> > it twice, I'm now a bit leaning towards just deprecating the old
> > config with the new API+config as well.
> >
> > A5: Among the names we have been discussed so far:
> >
> > DslStorePlugin
> > StoreTypeSpec
> > StoreImplSpec
> > DslStoreSuppliers
> >
> > I am in favor of DslStoreSuppliers as well as a restrictiveness on its
> > scope, just to echo Bruno's comments above.
> >
> >
> >
> > Guozhang
> >
> > On Thu, Jul 27, 2023 at 4:15 AM Bruno Cadonna  wrote:
> > >
> > > Hi,
> > >
> > > A5. I have to admit that
> > > "If we envision extending this beyond just StoreSupplier types, it could
> > > be a good option."
> > > is scaring me a bit.
> > > I am wondering what would be an example for such an extension?
> > > In general, I would propose to limit the scope of a config. In this case
> > > the config should provide suppliers for state stores for the DSL.
> > >
> > > BTW, maybe it is a good idea to let DslStorePlugin extend Configurable.
> > >
> > > Best,
> > > Bruno
> > >
> > > On 7/27/23 2:15 AM, Sophie Blee-Goldman wrote:
> > > > Thanks for the feedback Bruno -- sounds like we're getting close to a
> > final
> > > > consensus here.
> > > > It sounds like the two main (only?) semi-unresolved questions that
> > still
> > > > have differing
> > > > opinions floating around are whether to deprecate the old config, and
> > what
> > > > to name the new config
> > > > + interface.
> > > >
> > > > Although I won't personally push back on any of the options listed
> > above,
> > > > here's my final two cents:
> > > >
> > > > A3. I'm still a firm believer in deprecating the old config, and agree
> > > > wholeheartedly with what Bruno said.
> > > >
> > > > A5. I also wasn't crazy about "Plugin" at first, but I will admit it's
> > > > grown on me. I think it rubbed me the wrong
> > > > way at  first because it's just not part of the standard vocabulary in
> > > > Streams so far. If we envision extending
> > > > this beyond just StoreSupplier types, it could be a good option.
> > > > DSLStoreSuppliers does make a lot of sense,
> > > > though.
> > > >
> > > > To throw out a few more ideas in case any of them stick, what about
> > > > something like DSLStoreFormat or
> > > > DSLStorageType, or even DSLStorageEngine? Or even DSLStoreFactory --
> > the
> > > > Stores class is described as
> > > > a "factory" (thoug

Re: [DISCUSS] KIP-954: expand default DSL store configuration to custom types

2023-07-27 Thread Sophie Blee-Goldman
A2: Guozhang, just to close the book on the ListValue store thing, I fully
agree it seems like overreach
to expose/force this on users, especially if it's fully internal today. But
just to make sure we're on the same
page here, you're still ok with this KIP fixing the API gap that exists
today, in which these stores cannot be
customized by the user at all?

In other words, after this KIP, the new behavior for the ListValue store in
a stream join will be:

S1: First, check if the user passed in a `DSLStoreSuppliers` (or whatever
the name will be) to the
   StreamJoined config object, and use that to obtain the
KVStoreSupplier for this ListValue store

S2: If none was provided, check if the user has set a default
DSLStoreSuppliers via the new config,
   and use that to get the KVStoreSupplier if so

S3: If neither is set, fall back to the original logic as it is today,
which is to pass in a KVStoreSupplier
   that is hard-coded to be either RocksDB or InMemory, based on what
is returned for the #persistent
   API by the StreamJoined's WindowStoreSupplier

Does that sound right? We can clarify this further in the KIP if need be

On Thu, Jul 27, 2023 at 10:48 AM Guozhang Wang 
wrote:

> Hi all,
>
> Like Almog's secretary as well! Just following up on that index:
>
> A2: I'm also happy without introducing versioned KV in this KIP as I
> would envision it to be introduced as new params into the
> KeyValuePluginParams in the future. And just to clarify on Sophie's
> previous comment, I think ListStore should not be exposed in this API
> until we see it as a common usage and hence would want to (again, we
> need to think very carefully since it would potentially ask all
> implementers to adopt) move it from the internal category to the
> public interface category. As for now, I think only having kv / window
> / session as public store types is fine.
>
> A3: Seems I was not making myself very clear at the beginning :P The
> major thing that I'd actually like to avoid having two configs
> co-exist for the same function since it will be a confusing learning
> curve for users, and hence what I was proposing is to just have the
> newly introduced interface but not introducing a new config, and I
> realized now that it is actually more aligned with the CUSTOM idea
> where the ordering would be looking at config first, and then the
> interface. I blushed as I read Almog likes it.. After thinking about
> it twice, I'm now a bit leaning towards just deprecating the old
> config with the new API+config as well.
>
> A5: Among the names we have been discussed so far:
>
> DslStorePlugin
> StoreTypeSpec
> StoreImplSpec
> DslStoreSuppliers
>
> I am in favor of DslStoreSuppliers as well as a restrictiveness on its
> scope, just to echo Bruno's comments above.
>
>
>
> Guozhang
>
> On Thu, Jul 27, 2023 at 4:15 AM Bruno Cadonna  wrote:
> >
> > Hi,
> >
> > A5. I have to admit that
> > "If we envision extending this beyond just StoreSupplier types, it could
> > be a good option."
> > is scaring me a bit.
> > I am wondering what would be an example for such an extension?
> > In general, I would propose to limit the scope of a config. In this case
> > the config should provide suppliers for state stores for the DSL.
> >
> > BTW, maybe it is a good idea to let DslStorePlugin extend Configurable.
> >
> > Best,
> > Bruno
> >
> > On 7/27/23 2:15 AM, Sophie Blee-Goldman wrote:
> > > Thanks for the feedback Bruno -- sounds like we're getting close to a
> final
> > > consensus here.
> > > It sounds like the two main (only?) semi-unresolved questions that
> still
> > > have differing
> > > opinions floating around are whether to deprecate the old config, and
> what
> > > to name the new config
> > > + interface.
> > >
> > > Although I won't personally push back on any of the options listed
> above,
> > > here's my final two cents:
> > >
> > > A3. I'm still a firm believer in deprecating the old config, and agree
> > > wholeheartedly with what Bruno said.
> > >
> > > A5. I also wasn't crazy about "Plugin" at first, but I will admit it's
> > > grown on me. I think it rubbed me the wrong
> > > way at  first because it's just not part of the standard vocabulary in
> > > Streams so far. If we envision extending
> > > this beyond just StoreSupplier types, it could be a good option.
> > > DSLStoreSuppliers does make a lot of sense,
> > > though.
> > >
> > > To throw out a few more ideas in case any of them stick, what about
> > > something like DSLStoreFormat or
> > > DSLStorageType, or even DSLStorageEngine? Or even DSLStoreFactory --
> the
> > > Stores class is described as
> > > a "factory" (though not named so) and, to me, is actually quite
> comparable
> > > -- both are providers not of the
> > > stores themselves, but of the basic building blocks of Stores (eg
> > > StoreSuppliers)
> > >
> > > Ultimately fine with anything though. We should try not to drag out
> the KIP
> > > discussion too long once it's down
> > > to just nits 

Build failed in Jenkins: Kafka » Kafka Branch Builder » trunk #2047

2023-07-27 Thread Apache Jenkins Server
See 


Changes:


--
[...truncated 296680 lines...]

Gradle Test Run :streams:integrationTest > Gradle Test Executor 180 > 
HighAvailabilityTaskAssignorIntegrationTest > 
shouldScaleOutWithWarmupTasksAndInMemoryStores(TestInfo) PASSED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 180 > 
KStreamAggregationDedupIntegrationTest > shouldReduce(TestInfo) STARTED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 180 > 
KStreamAggregationDedupIntegrationTest > shouldReduce(TestInfo) PASSED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 180 > 
KStreamAggregationDedupIntegrationTest > shouldGroupByKey(TestInfo) STARTED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 180 > 
KStreamAggregationDedupIntegrationTest > shouldGroupByKey(TestInfo) PASSED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 180 > 
KStreamAggregationDedupIntegrationTest > shouldReduceWindowed(TestInfo) STARTED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 180 > 
KStreamAggregationDedupIntegrationTest > shouldReduceWindowed(TestInfo) PASSED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 180 > 
KStreamKStreamIntegrationTest > shouldOuterJoin() STARTED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 180 > 
KStreamKStreamIntegrationTest > shouldOuterJoin() PASSED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 180 > 
KTableKTableForeignKeyInnerJoinMultiIntegrationTest > initializationError 
STARTED
org.apache.kafka.streams.integration.KTableKTableForeignKeyInnerJoinMultiIntegrationTest.initializationError
 failed, log available in 
/home/jenkins/jenkins-agent/workspace/Kafka_kafka_trunk/streams/build/reports/testOutput/org.apache.kafka.streams.integration.KTableKTableForeignKeyInnerJoinMultiIntegrationTest.initializationError.test.stdout

Gradle Test Run :streams:integrationTest > Gradle Test Executor 180 > 
KTableKTableForeignKeyInnerJoinMultiIntegrationTest > initializationError FAILED
java.lang.RuntimeException: java.util.concurrent.ExecutionException: 
org.apache.kafka.common.errors.TimeoutException: The request timed out.
at 
org.apache.kafka.streams.integration.utils.KafkaEmbedded.createTopic(KafkaEmbedded.java:180)
at 
org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster.createTopic(EmbeddedKafkaCluster.java:207)
at 
org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster.createTopic(EmbeddedKafkaCluster.java:192)
at 
org.apache.kafka.streams.integration.KTableKTableForeignKeyInnerJoinMultiIntegrationTest.startCluster(KTableKTableForeignKeyInnerJoinMultiIntegrationTest.java:96)

Caused by:
java.util.concurrent.ExecutionException: 
org.apache.kafka.common.errors.TimeoutException: The request timed out.
at 
java.base/java.util.concurrent.CompletableFuture.reportGet(CompletableFuture.java:396)
at 
java.base/java.util.concurrent.CompletableFuture.get(CompletableFuture.java:2073)
at 
org.apache.kafka.common.internals.KafkaFutureImpl.get(KafkaFutureImpl.java:165)
at 
org.apache.kafka.streams.integration.utils.KafkaEmbedded.createTopic(KafkaEmbedded.java:178)
... 3 more

Caused by:
org.apache.kafka.common.errors.TimeoutException: The request timed 
out.

Gradle Test Run :streams:integrationTest > Gradle Test Executor 180 > 
KTableSourceTopicRestartIntegrationTest > 
shouldRestoreAndProgressWhenTopicNotWrittenToDuringRestoration() STARTED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 180 > 
KTableSourceTopicRestartIntegrationTest > 
shouldRestoreAndProgressWhenTopicNotWrittenToDuringRestoration() PASSED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 180 > 
KTableSourceTopicRestartIntegrationTest > 
shouldRestoreAndProgressWhenTopicWrittenToDuringRestorationWithEosAlphaEnabled()
 STARTED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 180 > 
KTableSourceTopicRestartIntegrationTest > 
shouldRestoreAndProgressWhenTopicWrittenToDuringRestorationWithEosAlphaEnabled()
 PASSED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 180 > 
KTableSourceTopicRestartIntegrationTest > 
shouldRestoreAndProgressWhenTopicWrittenToDuringRestorationWithEosDisabled() 
STARTED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 180 > 
KTableSourceTopicRestartIntegrationTest > 
shouldRestoreAndProgressWhenTopicWrittenToDuringRestorationWithEosDisabled() 
PASSED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 180 > 
KTableSourceTopicRestartIntegrationTest > 
shouldRestoreAndProgressWhenTopicWrittenToDuringRestorationWithEosV2Enabled() 
STARTED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 180 > 
KTableSourceTopicRestartIntegrationTest >

Re: [DISCUSS] KIP-954: expand default DSL store configuration to custom types

2023-07-27 Thread Guozhang Wang
Hi all,

Like Almog's secretary as well! Just following up on that index:

A2: I'm also happy without introducing versioned KV in this KIP as I
would envision it to be introduced as new params into the
KeyValuePluginParams in the future. And just to clarify on Sophie's
previous comment, I think ListStore should not be exposed in this API
until we see it as a common usage and hence would want to (again, we
need to think very carefully since it would potentially ask all
implementers to adopt) move it from the internal category to the
public interface category. As for now, I think only having kv / window
/ session as public store types is fine.

A3: Seems I was not making myself very clear at the beginning :P The
major thing that I'd actually like to avoid having two configs
co-exist for the same function since it will be a confusing learning
curve for users, and hence what I was proposing is to just have the
newly introduced interface but not introducing a new config, and I
realized now that it is actually more aligned with the CUSTOM idea
where the ordering would be looking at config first, and then the
interface. I blushed as I read Almog likes it.. After thinking about
it twice, I'm now a bit leaning towards just deprecating the old
config with the new API+config as well.

A5: Among the names we have been discussed so far:

DslStorePlugin
StoreTypeSpec
StoreImplSpec
DslStoreSuppliers

I am in favor of DslStoreSuppliers as well as a restrictiveness on its
scope, just to echo Bruno's comments above.



Guozhang

On Thu, Jul 27, 2023 at 4:15 AM Bruno Cadonna  wrote:
>
> Hi,
>
> A5. I have to admit that
> "If we envision extending this beyond just StoreSupplier types, it could
> be a good option."
> is scaring me a bit.
> I am wondering what would be an example for such an extension?
> In general, I would propose to limit the scope of a config. In this case
> the config should provide suppliers for state stores for the DSL.
>
> BTW, maybe it is a good idea to let DslStorePlugin extend Configurable.
>
> Best,
> Bruno
>
> On 7/27/23 2:15 AM, Sophie Blee-Goldman wrote:
> > Thanks for the feedback Bruno -- sounds like we're getting close to a final
> > consensus here.
> > It sounds like the two main (only?) semi-unresolved questions that still
> > have differing
> > opinions floating around are whether to deprecate the old config, and what
> > to name the new config
> > + interface.
> >
> > Although I won't personally push back on any of the options listed above,
> > here's my final two cents:
> >
> > A3. I'm still a firm believer in deprecating the old config, and agree
> > wholeheartedly with what Bruno said.
> >
> > A5. I also wasn't crazy about "Plugin" at first, but I will admit it's
> > grown on me. I think it rubbed me the wrong
> > way at  first because it's just not part of the standard vocabulary in
> > Streams so far. If we envision extending
> > this beyond just StoreSupplier types, it could be a good option.
> > DSLStoreSuppliers does make a lot of sense,
> > though.
> >
> > To throw out a few more ideas in case any of them stick, what about
> > something like DSLStoreFormat or
> > DSLStorageType, or even DSLStorageEngine? Or even DSLStoreFactory -- the
> > Stores class is described as
> > a "factory" (though not named so) and, to me, is actually quite comparable
> > -- both are providers not of the
> > stores themselves, but of the basic building blocks of Stores (eg
> > StoreSuppliers)
> >
> > Ultimately fine with anything though. We should try not to drag out the KIP
> > discussion too long once it's down
> > to just nits :P
> >
> > Cheers,
> > Sophie
> >
> >
> >
> > On Wed, Jul 26, 2023 at 8:04 AM Almog Gavra  wrote:
> >
> >> Thanks for the comments Bruno!
> >>
> >> A3. Oops... I think I didn't do a great job updating the KIP to reflect
> >> Guozhang's suggestion. This seems like the last point of contention, where
> >> we have two options:
> >>
> >> 1. Deprecate the config entirely and replace IN_MEMORY/ROCKSDB with
> >> implementations of the DslStorePlugin
> >> 2. (What's currently in the KIP) Introduce a new config which defaults to
> >> DefaultDslStorePlugin and only the DefaultDslStorePlugin will respect the
> >> old default.store.type config
> >>
> >> I'm happy with either, I'll keep the KIP with (2) for now as that seemed
> >> like the result of the previous discussion but I have no problem changing
> >> it back to (1) which was the original proposal.
> >>
> >> A5. I like "DslStorePlugin" because it leaves room for configuring
> >> implementations beyond just supplying stores (e.g. we could introduce a
> >> `configure()` method etc...). I'll keep it as is for now (and change
> >> Materialized/Stores API sections - thanks for catching that)! I don't feel
> >> too strongly and wouldn't dig my heels in if most people preferred
> >> "DslStoreSuppliers" (I don't love DslStores as it resembles the Stores
> >> class to closely in name and they're a little different).
> >>
> >> A6. Yup, that's th

Re: Apache Kafka 3.6.0 release

2023-07-27 Thread Divij Vaidya
Thank you for the response, Ismael.

1. Specifically in context of 3.6, I wanted this compatibility
guarantee point to encourage a discussion on
https://cwiki.apache.org/confluence/display/KAFKA/KIP-952%3A+Regenerate+segment-aligned+producer+snapshots+when+upgrading+to+a+Kafka+version+supporting+Tiered+Storage.
Due to lack of producer snapshots in <2.8 versions, a customer may not
be able to upgrade to 3.6 and use TS on a topic which was created when
the cluster was on <2.8 version (see motivation for details). We can
discuss and agree that it does not break compatibility, which is fine.
But I want to ensure that we have a discussion soon on this to reach a
conclusion.

2. I will start a KIP on this for further discussion.

3. In the context of 3.6, this would mean that there should be
no-regression, if a user does "not" turn-on remote storage (early
access feature) at a cluster level. We have some known cases (such as
https://issues.apache.org/jira/browse/KAFKA-15189) which violate this
compatibility requirement. Having this guarantee mentioned in the
release plan will ensure that we are all in agreement with which cases
are truly blockers and which aren't.

4. Fair, instead of a general goal, let me put it specifically in the
context of 3.6. Let me know if this is not the right forum for this
discussion.
Once a user "turns on" tiered storage (TS) at a cluster level, I am
proposing that they should have the ability to turn it off as well at
a cluster level. Since this is a topic level feature, folks may not
spin up a separate cluster to try this feature, hence, we need to
ensure that we provide them with the ability to try tiered storage for
a topic which could be deleted and featured turned-off, so that rest
of the production cases are not impacted.

5. Agree on not making public interface change as a requirement but we
should define what "early access" means in that case. Users may not be
aware that "early access" public APIs may change (unless I am missing
some documentation somewhere completely, in which case I apologize for
bringing this naive point).

--
Divij Vaidya

On Thu, Jul 27, 2023 at 2:27 PM Ismael Juma  wrote:
>
> Hi Divij,
>
> Some of these are launch checklist items (not really goals) and some are
> compatibility guarantees. More below.
>
> On Thu, Jul 27, 2023, 12:10 PM Divij Vaidya  wrote:
>
> > Hey Satish
> >
> > Could we consider adding "launch goals" in the release plan. While
> > some of these may be implicit, it would be nice to list them down in
> > the release plan. For this release, our launch requirements would be:
> > 1. Users should be able to upgrade from any prior Kafka version to this
> > version.
> >
>
> This is part of the compatibility guarantees. The upgrade notes mention
> this already. If there is a change in a given release, it should definitely
> be highlighted.
>
> 2. On release, this version (or it's dependencies) would not have any
> > known MEDIUM/HIGH CVE.
> >
>
> This is a new policy and the details should be discussed. In particular,
> the threshold (medium or high).
>
> 3. Presence of any "early access"/"beta" feature should not impact
> > other production features when it is not enabled.
> >
>
> This is a general guideline for early access features and not specific to
> this release. It would be good to have a page that talks about these things.
>
> 4. Once enabled, users should have an option to disable any "early
> > access"/"beta" feature and resume normal production features, i.e.
> > impact of beta features should be reversible.
> >
>
> This needs discussion and I don't think it's reasonable as a general rule.
> For example, Kraft early access wasn't reversible and it was not feasible
> for it to be.
>
> 5. KIP-405 will be available in "early access"/"beta" mode. Early
> > access/beta means that the public facing interfaces won't change in
> > future but the implementation is not recommended to be used in
> > production.
>
>
> I don't think it's ok to make this a requirement. Early access is a way to
> get early feedback and all types of changes should be on the table. They
> would be discussed via KIPs as usual. I believe there were some
> incompatible changes for Kraft during the early access period although the
> team aimed to minimize work required during upgrades. I have mentioned
> Kraft a couple of times since it's a good example of a large feature that
> went through this process.
>
> Ismael


Re: [VOTE] KIP-919: Allow AdminClient to Talk Directly with the KRaft Controller Quorum and add Controller Registration

2023-07-27 Thread Colin McCabe
I would incline towards plural, but I honestly don't feel that strongly about 
it. I will just change it to --bootstrap-controller for now, to match 
--bootstrap-server. Perhaps we can discuss this further in a later KIP, if 
people are inclined...

Colin


On Thu, Jul 27, 2023, at 07:13, Ron Dagostino wrote:
>> support plural for both and the singular one is just an alias for 
>> compatibility
>
> I like that approach.  I can never remember what works, so I would
> prefer to just use what I think works and then have it work.
>
> Ron
>
> On Thu, Jul 27, 2023 at 8:33 AM Ismael Juma  wrote:
>>
>> I think singular was used because that's the common case for the cli tools.
>> To be honest, it's actually a bit more confusing to have both singular and
>> plural for the cli tools and you have to remember the exact version for
>> each one. We should either support plural for both and the singular one is
>> just an alias for compatibility or stick with singular for both.
>>
>> Ismael
>>
>> On Thu, Jul 27, 2023, 12:03 AM Colin McCabe  wrote:
>>
>> > On Wed, Jul 26, 2023, at 07:09, Ron Dagostino wrote:
>> > > Thanks, Colin.  +1 (binding) from me.
>> > >
>> > > I will note that Ziming mentioned in the DISCUSS thread that "There is
>> > > a mistake that we use `--bootstrap-server` instead of
>> > > `--bootstrap-server(s)`, so should we also change the new argument
>> > > `--bootstrap-controller` (no s).".  I agree that this is an
>> > > unfortunate historical artifact.  Furthermore, the config property is
>> > > bootstrap.servers, which does not match the command line argument
>> > > --bootstrap-server.  I don't know what we should do here -- I am not
>> > > sure that we should propagate it.  We could continue the same mismatch
>> > > and use --bootstrap-controller, but I feel like using
>> > > --bootstrap-controllers makes more sense.  Anyway, this is more an
>> > > ergonomic issue than anything else.  I am still +1 binding, but we
>> > > should arrive at an explicit decision here.
>> >
>> > Hi Ron,
>> >
>> > Thanks for the review and vote.
>> >
>> > The current KIP-919 text has "--bootstrap-controllers" for command-line
>> > and "bootstrap.controllers" for config.
>> >
>> > As far as I know, --bootstrap-server is singular rather than plural just
>> > for historical reasons so I figured the new flag can just be plural. As
>> > you mentioned, the configs have always been plural, so it's more consistent
>> > for the flags to match the configs.
>> >
>> > (Though obviously that ship has sailed on --bootstrap-server, we're not
>> > gonna change it now ;)
>> >
>> > >
>> > > Nits:
>> > > s/boostrap-servers/bootstrap-servers/
>> > > s/DescribeClusterResquest/DescribeClusterRequest/
>> > > s/ControllerRegistrationRecord/RegisterControllerRecord/g
>> > >
>> >
>> > Thanks. I fixed these typos and changed ControllerRegistrationRecord ->
>> > RegisterControllerRecord as suggested.
>> >
>> > best,
>> > Colin
>> >
>> >
>> > > Ron
>> > >
>> > > On Wed, Jul 26, 2023 at 10:06 AM David Arthur
>> > >  wrote:
>> > >>
>> > >> Thanks for driving this KIP, Colin!
>> > >>
>> > >> +1 binding
>> > >>
>> > >> -David
>> > >>
>> > >> On Wed, Jul 26, 2023 at 8:58 AM Divij Vaidya 
>> > >> wrote:
>> > >>
>> > >> > +1 (binding)
>> > >> >
>> > >> > --
>> > >> > Divij Vaidya
>> > >> >
>> > >> >
>> > >> > On Wed, Jul 26, 2023 at 2:56 PM ziming deng > > >
>> > >> > wrote:
>> > >> > >
>> > >> > > +1 (binding) from me.
>> > >> > >
>> > >> > > Thanks for the KIP!
>> > >> > >
>> > >> > > --
>> > >> > > Ziming
>> > >> > >
>> > >> > > > On Jul 26, 2023, at 20:18, Luke Chen  wrote:
>> > >> > > >
>> > >> > > > +1 (binding) from me.
>> > >> > > >
>> > >> > > > Thanks for the KIP!
>> > >> > > >
>> > >> > > > Luke
>> > >> > > >
>> > >> > > > On Tue, Jul 25, 2023 at 1:24 AM Colin McCabe 
>> > >> > wrote:
>> > >> > > >
>> > >> > > >> Hi all,
>> > >> > > >>
>> > >> > > >> I'd like to start the vote for KIP-919: Allow AdminClient to Talk
>> > >> > Directly
>> > >> > > >> with the KRaft Controller Quorum and add Controller Registration.
>> > >> > > >>
>> > >> > > >> The KIP is here: https://cwiki.apache.org/confluence/x/Owo0Dw
>> > >> > > >>
>> > >> > > >> Thanks to everyone who reviewed the proposal.
>> > >> > > >>
>> > >> > > >> best,
>> > >> > > >> Colin
>> > >> > > >>
>> > >> > >
>> > >> >
>> > >>
>> > >>
>> > >> --
>> > >> -David
>> >


Re: plz help me code review

2023-07-27 Thread Guozhang Wang
Hello Xiangyuan,

Thanks for your contributions and thanks for raising it to community's
attention. I will take a look at it sooner than later.

On Thu, Jul 27, 2023 at 2:41 AM Xiangyuan LI  wrote:
>
> Hi kafka team:
>   I raise a pr https://github.com/apache/kafka/pull/13965 to fix
> https://issues.apache.org/jira/browse/KAFKA-15106,
>   it mentions some serious bug and no one check it for a long time, plz
> help me review if could. thx!
>
>  these bugs are
> in 
> org.apache.kafka.clients.consumer.internals.AbstractStickyAssignor.ConstrainedAssignmentBuilder#ConstrainedAssignmentBuilder
> and all of them could cause rebalance stuck for ever, and they could occur
> in production environment, very, very, very easy.


[jira] [Created] (KAFKA-15263) KRaftMigrationDriver can run the migration twice

2023-07-27 Thread David Arthur (Jira)
David Arthur created KAFKA-15263:


 Summary: KRaftMigrationDriver can run the migration twice
 Key: KAFKA-15263
 URL: https://issues.apache.org/jira/browse/KAFKA-15263
 Project: Kafka
  Issue Type: Bug
Reporter: David Arthur
Assignee: David Arthur


There is a narrow race condition in KRaftMigrationDriver where a PollEvent can 
run that sees the internal state as ZK_MIGRATION and is immediately followed by 
another poll event (due to external call to {{{}wakeup(){}}}) that results in 
two MigrateMetadataEvent being enqueued. 

Since MigrateMetadataEvent lacks a check on the internal state, this causes the 
metadata migration to occur twice. 



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


Re: [VOTE] KIP-951: Leader discovery optimisations for the client

2023-07-27 Thread José Armando García Sancio
The KIP LGTM. Thanks for the design. I am looking forward to the implementation.

+1 (binding).

Thanks!
-- 
-José


Re: [DISCUSS] KIP-953: partition method to be overloaded to accept headers as well.

2023-07-27 Thread Andrew Schofield
Hi Jack,
Thanks for the KIP. I have a few concerns about the idea.

1) I think that while a client-side partitioner seems like a neat idea and it’s 
an established part of Kafka,
it’s one of the things which makes Kafka clients quite complicated. Just as 
KIP-848 is moving from
client-side assignors to server-side assignors, I wonder whether really we 
should be looking to make
partitioning a server-side capability too over time. So, I’m not convinced that 
making the Partitioner
interface richer is moving in the right direction.

2) For records with a key, the partitioner usually calculates the partition 
from the key. This means
that records with the same key end up on the same partition. Many applications 
expect this to give ordering.
Log compaction expects this. There are situations in which records have to be 
repartitioned, such as
sometimes happens with Kafka Streams. I think that a header-based partitioner 
for records which have
keys is going to be surprising and only going to have limited applicability as 
a result.

The tricky part about clever partitioning is that downstream systems have no 
idea how the partition
number was arrived at, so they do not truly understand how the ordering was 
derived. I do think that
perhaps there’s value to being able to influence the partitioning using the 
headers, but I wonder if actually
transforming the headers into an “ordering context” that then flows with the 
record as it moves through
the system would be a stronger solution. Today, the key is the ordering 
context. Maybe it should be a
concept in its own right and the Producer could configure a converter from 
headers to ordering context.
That is of course a much bigger change.

In one of the examples you mention in the KIP, you mentioned using a header to 
control priority. I guess the
idea is to preferentially process records off specific partitions so they can 
overtake lower priority records.
I suggest just sending the records explicitly to specific partitions to achieve 
this.

Sorry for the essay, but you did ask for people to share their thoughts :)

Just my opinion. Let’s see what others think.

Thanks,
Andrew

> On 25 Jul 2023, at 14:58, Jack Tomy  wrote:
>
> Hey @Sagar
>
> Thanks again for the review.
> 1. "a null headers value is equivalent to invoking the older partition
> method", this is not true. If someone makes an implementation and the
> headers come as null, still the new implementation will take effect.
> Instead I have added : "Not overriding this method in the Partitioner
> interface has the same behaviour as using the existing method."
> 2. Corrected.
>
> Hey @Sagar and everyone,
> Please have a look at the new version and share your thoughts.
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=263424937
> Like Sagar mentioned, I would also request more people who have more
> context on clients to chime in.
>
>
> On Tue, Jul 25, 2023 at 2:58 PM Sagar  wrote:
>
>> Hi Jack,
>>
>> Thanks I have a couple of final comments and then I am good.
>>
>> 1) Can you elaborate on the Javadocs of the partition headers argument to
>> specify that a null headers value is equivalent to invoking the older
>> partition method? It is apparent but generally good to call out.
>> 2) In the Compatibility section, you have mentioned backward comparable. I
>> believe it should be *backward compatible change.*
>>
>> I don't have other comments. Post this, probably someone else who has more
>> context on Clients can also chime in on this before we can move this to
>> Voting.
>>
>> Thanks!
>> Sagar.
>>
>>
>> On Sat, Jul 22, 2023 at 10:09 AM Jack Tomy  wrote:
>>
>>> Hey @Sagar,
>>>
>>> Thank you again for the response and feedback.
>>>
>>>   1. Though the ask wasn't very clear to me I have attached the Javadoc
>> as
>>>   per your suggestion. Please have a look and let me know if this meets
>>> the
>>>   expectations.
>>>   2. Done.
>>>   3. Done
>>>   4. Done
>>>
>>> Hey @Sagar and everyone,
>>> Please have a look at the new version and share your thoughts.
>>>
>> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=263424937
>>>
>>> On Thu, Jul 20, 2023 at 9:46 PM Sagar  wrote:
>>>
 Thanks Jack for the updates.

 Some more feedback:

 1) It would be better if you can add the Javadoc in the Public
>> interfaces
 section. That is a general practice used which gives the readers of the
>>> KIP
 a high level idea of the Public Interfaces.

 2) In the proposed section, the bit about marking headers as read only
 seems like an implementation detail This can generally be avoided in
>>> KIPs.

 3) Also, in the Deprecation section, can you mention again that this
>> is a
 backward compatible change and the reason for it (already done in the
 Proposed Changes section).

 4) In the Testing Plan section, there is still the KIP template bit
>>> copied
 over. That can be removed.

 Thanks!
 Sagar.


 On T

Build failed in Jenkins: Kafka » Kafka Branch Builder » 3.4 #152

2023-07-27 Thread Apache Jenkins Server
See 


Changes:


--
[...truncated 476998 lines...]

Gradle Test Run :core:integrationTest > Gradle Test Executor 170 > 
KafkaZkClientTest > testGetTopicsAndPartitions() PASSED

Gradle Test Run :core:integrationTest > Gradle Test Executor 170 > 
KafkaZkClientTest > testChroot(boolean) > 
kafka.zk.KafkaZkClientTest.testChroot(boolean)[1] STARTED

Gradle Test Run :core:integrationTest > Gradle Test Executor 170 > 
KafkaZkClientTest > testChroot(boolean) > 
kafka.zk.KafkaZkClientTest.testChroot(boolean)[1] PASSED

Gradle Test Run :core:integrationTest > Gradle Test Executor 170 > 
KafkaZkClientTest > testChroot(boolean) > 
kafka.zk.KafkaZkClientTest.testChroot(boolean)[2] STARTED

Gradle Test Run :core:integrationTest > Gradle Test Executor 170 > 
KafkaZkClientTest > testChroot(boolean) > 
kafka.zk.KafkaZkClientTest.testChroot(boolean)[2] PASSED

Gradle Test Run :core:integrationTest > Gradle Test Executor 170 > 
KafkaZkClientTest > testRegisterBrokerInfo() STARTED

Gradle Test Run :core:integrationTest > Gradle Test Executor 170 > 
KafkaZkClientTest > testRegisterBrokerInfo() PASSED

Gradle Test Run :core:integrationTest > Gradle Test Executor 170 > 
KafkaZkClientTest > testRetryRegisterBrokerInfo() STARTED

Gradle Test Run :core:integrationTest > Gradle Test Executor 170 > 
KafkaZkClientTest > testRetryRegisterBrokerInfo() PASSED

Gradle Test Run :core:integrationTest > Gradle Test Executor 170 > 
KafkaZkClientTest > testConsumerOffsetPath() STARTED

Gradle Test Run :core:integrationTest > Gradle Test Executor 170 > 
KafkaZkClientTest > testConsumerOffsetPath() PASSED

Gradle Test Run :core:integrationTest > Gradle Test Executor 170 > 
KafkaZkClientTest > testDeleteRecursiveWithControllerEpochVersionCheck() STARTED

Gradle Test Run :core:integrationTest > Gradle Test Executor 170 > 
KafkaZkClientTest > testDeleteRecursiveWithControllerEpochVersionCheck() PASSED

Gradle Test Run :core:integrationTest > Gradle Test Executor 170 > 
KafkaZkClientTest > testTopicAssignments() STARTED

Gradle Test Run :core:integrationTest > Gradle Test Executor 170 > 
KafkaZkClientTest > testTopicAssignments() PASSED

Gradle Test Run :core:integrationTest > Gradle Test Executor 170 > 
KafkaZkClientTest > testControllerManagementMethods() STARTED

Gradle Test Run :core:integrationTest > Gradle Test Executor 170 > 
KafkaZkClientTest > testControllerManagementMethods() PASSED

Gradle Test Run :core:integrationTest > Gradle Test Executor 170 > 
KafkaZkClientTest > testTopicAssignmentMethods() STARTED

Gradle Test Run :core:integrationTest > Gradle Test Executor 170 > 
KafkaZkClientTest > testTopicAssignmentMethods() PASSED

Gradle Test Run :core:integrationTest > Gradle Test Executor 170 > 
KafkaZkClientTest > testConnectionViaNettyClient() STARTED

Gradle Test Run :core:integrationTest > Gradle Test Executor 170 > 
KafkaZkClientTest > testConnectionViaNettyClient() PASSED

Gradle Test Run :core:integrationTest > Gradle Test Executor 170 > 
KafkaZkClientTest > testPropagateIsrChanges() STARTED

Gradle Test Run :core:integrationTest > Gradle Test Executor 170 > 
KafkaZkClientTest > testPropagateIsrChanges() PASSED

Gradle Test Run :core:integrationTest > Gradle Test Executor 170 > 
KafkaZkClientTest > testControllerEpochMethods() STARTED

Gradle Test Run :core:integrationTest > Gradle Test Executor 170 > 
KafkaZkClientTest > testControllerEpochMethods() PASSED

Gradle Test Run :core:integrationTest > Gradle Test Executor 170 > 
KafkaZkClientTest > testDeleteRecursive() STARTED

Gradle Test Run :core:integrationTest > Gradle Test Executor 170 > 
KafkaZkClientTest > testDeleteRecursive() PASSED

Gradle Test Run :core:integrationTest > Gradle Test Executor 170 > 
KafkaZkClientTest > testGetTopicPartitionStates() STARTED

Gradle Test Run :core:integrationTest > Gradle Test Executor 170 > 
KafkaZkClientTest > testGetTopicPartitionStates() PASSED

Gradle Test Run :core:integrationTest > Gradle Test Executor 170 > 
KafkaZkClientTest > testCreateConfigChangeNotification() STARTED

Gradle Test Run :core:integrationTest > Gradle Test Executor 170 > 
KafkaZkClientTest > testCreateConfigChangeNotification() PASSED

Gradle Test Run :core:integrationTest > Gradle Test Executor 170 > 
KafkaZkClientTest > testDelegationTokenMethods() STARTED

Gradle Test Run :core:integrationTest > Gradle Test Executor 170 > 
KafkaZkClientTest > testDelegationTokenMethods() PASSED

Gradle Test Run :core:integrationTest > Gradle Test Executor 170 > 
ZkMigrationClientTest > testUpdateExistingPartitions() STARTED

Gradle Test Run :core:integrationTest > Gradle Test Executor 170 > 
ZkMigrationClientTest > testUpdateExistingPartitions() PASSED

Gradle Test Run :core:integrationTest > Gradle Test Executor 170 > 
ZkMigrationClientTest > testEmptyWrite() STARTED

Gradle Test Run :core:integrationTest > Gradle Test Executor 170 > 
ZkMigrationClientTest > tes

[jira] [Created] (KAFKA-15262) MirrorHeartbeatConnector is not working as documented

2023-07-27 Thread Ravindranath Kakarla (Jira)
Ravindranath Kakarla created KAFKA-15262:


 Summary: MirrorHeartbeatConnector is not working as documented
 Key: KAFKA-15262
 URL: https://issues.apache.org/jira/browse/KAFKA-15262
 Project: Kafka
  Issue Type: Bug
  Components: mirrormaker
Affects Versions: 3.5.0, 3.4.0, 2.8.0
Reporter: Ravindranath Kakarla


As per the MM2 
[KIP-382|[https://cwiki.apache.org/confluence/display/KAFKA/KIP-382%3A+MirrorMaker+2.0],]
 the MirrorHeartbeatConnector should emit pings to heartbeat topic on the 
source cluster which then gets replicated to the target cluster. This can be 
used to demonstrate that MM2 is replicating the data.

"""
h2. Internal Topics

MM2 emits a *heartbeat* *topic* in each source cluster, which is replicated to 
demonstrate connectivity through the connectors. Downstream consumers can use 
this topic to verify that a) the connector is running and b) the corresponding 
source cluster is available. Heartbeats will get propagated by source and sink 
connectors s.t. chains like backup.us-west.us-east.heartbeat are possible.

"""

 

However, this is not happening right now. To contrary, the 
MirrorHeartbeatConnector is producing heartbeat pings to target cluster instead 
of source. This is not much useful as it won't help detect problems connecting 
to source cluster or with the data replication.

 

Is my understanding of the MirrorHeartbeatConnector accurate?

[Code 
Ref|https://github.com/apache/kafka/blob/ed44bcd71b3b9926c474033882eaa6c1cf35cfa4/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorHeartbeatTask.java#L65]

 



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


Re: [DISCUSS] KIP-953: partition method to be overloaded to accept headers as well.

2023-07-27 Thread Jack Tomy
Hey Everyone,

Please consider this as a gentle reminder.
Please have a look at the KIP and share your thoughts.
https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=263424937

On Tue, Jul 25, 2023 at 7:28 PM Jack Tomy  wrote:

> Hey @Sagar
>
> Thanks again for the review.
> 1. "a null headers value is equivalent to invoking the older partition
> method", this is not true. If someone makes an implementation and the
> headers come as null, still the new implementation will take effect.
> Instead I have added : "Not overriding this method in the Partitioner
> interface has the same behaviour as using the existing method."
> 2. Corrected.
>
> Hey @Sagar and everyone,
> Please have a look at the new version and share your thoughts.
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=263424937
> Like Sagar mentioned, I would also request more people who have more
> context on clients to chime in.
>
>
> On Tue, Jul 25, 2023 at 2:58 PM Sagar  wrote:
>
>> Hi Jack,
>>
>> Thanks I have a couple of final comments and then I am good.
>>
>> 1) Can you elaborate on the Javadocs of the partition headers argument to
>> specify that a null headers value is equivalent to invoking the older
>> partition method? It is apparent but generally good to call out.
>> 2) In the Compatibility section, you have mentioned backward comparable. I
>> believe it should be *backward compatible change.*
>>
>> I don't have other comments. Post this, probably someone else who has more
>> context on Clients can also chime in on this before we can move this to
>> Voting.
>>
>> Thanks!
>> Sagar.
>>
>>
>> On Sat, Jul 22, 2023 at 10:09 AM Jack Tomy  wrote:
>>
>> > Hey @Sagar,
>> >
>> > Thank you again for the response and feedback.
>> >
>> >1. Though the ask wasn't very clear to me I have attached the
>> Javadoc as
>> >per your suggestion. Please have a look and let me know if this meets
>> > the
>> >expectations.
>> >2. Done.
>> >3. Done
>> >4. Done
>> >
>> > Hey @Sagar and everyone,
>> > Please have a look at the new version and share your thoughts.
>> >
>> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=263424937
>> >
>> > On Thu, Jul 20, 2023 at 9:46 PM Sagar 
>> wrote:
>> >
>> > > Thanks Jack for the updates.
>> > >
>> > > Some more feedback:
>> > >
>> > > 1) It would be better if you can add the Javadoc in the Public
>> interfaces
>> > > section. That is a general practice used which gives the readers of
>> the
>> > KIP
>> > > a high level idea of the Public Interfaces.
>> > >
>> > > 2) In the proposed section, the bit about marking headers as read only
>> > > seems like an implementation detail This can generally be avoided in
>> > KIPs.
>> > >
>> > > 3) Also, in the Deprecation section, can you mention again that this
>> is a
>> > > backward compatible change and the reason for it (already done in the
>> > > Proposed Changes section).
>> > >
>> > > 4) In the Testing Plan section, there is still the KIP template bit
>> > copied
>> > > over. That can be removed.
>> > >
>> > > Thanks!
>> > > Sagar.
>> > >
>> > >
>> > > On Thu, Jul 20, 2023 at 2:48 PM Jack Tomy 
>> wrote:
>> > >
>> > > > Hey Everyone,
>> > > >
>> > > > Please consider this as a reminder and share your feedback. Thank
>> you.
>> > > >
>> > >
>> >
>> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=263424937
>> > > >
>> > > > On Tue, Jul 18, 2023 at 5:43 PM Jack Tomy 
>> > wrote:
>> > > >
>> > > > > Hey @Sagar,
>> > > > >
>> > > > > Thank you for the response and feedback.
>> > > > >
>> > > > >1. Done
>> > > > >2. Yeah, that was a mistake from my end. Corrected.
>> > > > >3. Can you please elaborate this, I have added the java doc
>> along
>> > > with
>> > > > >the code changes. Should I paste the same in KIP too?
>> > > > >4. Moved.
>> > > > >5. I have added one more use case, it is actually helpful in
>> any
>> > > > >situation where you want to pass some information to partition
>> > > method
>> > > > but
>> > > > >don't have to have it in the key or value.
>> > > > >6. Added.
>> > > > >
>> > > > >
>> > > > > Hey @Sagar and everyone,
>> > > > > Please have a look at the new version and share your thoughts.
>> > > > >
>> > > >
>> > >
>> >
>> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=263424937
>> > > > >
>> > > > >
>> > > > > On Tue, Jul 18, 2023 at 9:53 AM Sagar 
>> > > wrote:
>> > > > >
>> > > > >> Hi Jack,
>> > > > >>
>> > > > >> Thanks for the KIP! Seems like an interesting idea. I have some
>> > > > feedback:
>> > > > >>
>> > > > >> 1) It would be great if you could clean up the text that seems to
>> > > mimic
>> > > > >> the
>> > > > >> KIP template. It is generally not required in the KIP.
>> > > > >>
>> > > > >> 2) In the Public Interfaces where you mentioned *Partitioner
>> method
>> > in
>> > > > >> **org/apache/kafka/clients/producer
>> > > > >> will have the following update*, I believe you meant the
>> Partitioner
>> > > > >>

Re: [VOTE] KIP-919: Allow AdminClient to Talk Directly with the KRaft Controller Quorum and add Controller Registration

2023-07-27 Thread Ron Dagostino
> support plural for both and the singular one is just an alias for 
> compatibility

I like that approach.  I can never remember what works, so I would
prefer to just use what I think works and then have it work.

Ron

On Thu, Jul 27, 2023 at 8:33 AM Ismael Juma  wrote:
>
> I think singular was used because that's the common case for the cli tools.
> To be honest, it's actually a bit more confusing to have both singular and
> plural for the cli tools and you have to remember the exact version for
> each one. We should either support plural for both and the singular one is
> just an alias for compatibility or stick with singular for both.
>
> Ismael
>
> On Thu, Jul 27, 2023, 12:03 AM Colin McCabe  wrote:
>
> > On Wed, Jul 26, 2023, at 07:09, Ron Dagostino wrote:
> > > Thanks, Colin.  +1 (binding) from me.
> > >
> > > I will note that Ziming mentioned in the DISCUSS thread that "There is
> > > a mistake that we use `--bootstrap-server` instead of
> > > `--bootstrap-server(s)`, so should we also change the new argument
> > > `--bootstrap-controller` (no s).".  I agree that this is an
> > > unfortunate historical artifact.  Furthermore, the config property is
> > > bootstrap.servers, which does not match the command line argument
> > > --bootstrap-server.  I don't know what we should do here -- I am not
> > > sure that we should propagate it.  We could continue the same mismatch
> > > and use --bootstrap-controller, but I feel like using
> > > --bootstrap-controllers makes more sense.  Anyway, this is more an
> > > ergonomic issue than anything else.  I am still +1 binding, but we
> > > should arrive at an explicit decision here.
> >
> > Hi Ron,
> >
> > Thanks for the review and vote.
> >
> > The current KIP-919 text has "--bootstrap-controllers" for command-line
> > and "bootstrap.controllers" for config.
> >
> > As far as I know, --bootstrap-server is singular rather than plural just
> > for historical reasons so I figured the new flag can just be plural. As
> > you mentioned, the configs have always been plural, so it's more consistent
> > for the flags to match the configs.
> >
> > (Though obviously that ship has sailed on --bootstrap-server, we're not
> > gonna change it now ;)
> >
> > >
> > > Nits:
> > > s/boostrap-servers/bootstrap-servers/
> > > s/DescribeClusterResquest/DescribeClusterRequest/
> > > s/ControllerRegistrationRecord/RegisterControllerRecord/g
> > >
> >
> > Thanks. I fixed these typos and changed ControllerRegistrationRecord ->
> > RegisterControllerRecord as suggested.
> >
> > best,
> > Colin
> >
> >
> > > Ron
> > >
> > > On Wed, Jul 26, 2023 at 10:06 AM David Arthur
> > >  wrote:
> > >>
> > >> Thanks for driving this KIP, Colin!
> > >>
> > >> +1 binding
> > >>
> > >> -David
> > >>
> > >> On Wed, Jul 26, 2023 at 8:58 AM Divij Vaidya 
> > >> wrote:
> > >>
> > >> > +1 (binding)
> > >> >
> > >> > --
> > >> > Divij Vaidya
> > >> >
> > >> >
> > >> > On Wed, Jul 26, 2023 at 2:56 PM ziming deng  > >
> > >> > wrote:
> > >> > >
> > >> > > +1 (binding) from me.
> > >> > >
> > >> > > Thanks for the KIP!
> > >> > >
> > >> > > --
> > >> > > Ziming
> > >> > >
> > >> > > > On Jul 26, 2023, at 20:18, Luke Chen  wrote:
> > >> > > >
> > >> > > > +1 (binding) from me.
> > >> > > >
> > >> > > > Thanks for the KIP!
> > >> > > >
> > >> > > > Luke
> > >> > > >
> > >> > > > On Tue, Jul 25, 2023 at 1:24 AM Colin McCabe 
> > >> > wrote:
> > >> > > >
> > >> > > >> Hi all,
> > >> > > >>
> > >> > > >> I'd like to start the vote for KIP-919: Allow AdminClient to Talk
> > >> > Directly
> > >> > > >> with the KRaft Controller Quorum and add Controller Registration.
> > >> > > >>
> > >> > > >> The KIP is here: https://cwiki.apache.org/confluence/x/Owo0Dw
> > >> > > >>
> > >> > > >> Thanks to everyone who reviewed the proposal.
> > >> > > >>
> > >> > > >> best,
> > >> > > >> Colin
> > >> > > >>
> > >> > >
> > >> >
> > >>
> > >>
> > >> --
> > >> -David
> >


Re: [DISCUSS] KIP-960: Support interactive queries (IQv2) for versioned state stores

2023-07-27 Thread Alieh Saeedi
Thanks, Bruno, for the feedback.


   - I agree with both points 2 and 3. About 3: Having "VersionsQualifier"
   reduces the number of methods and makes everything less confusing. At the
   end, that will be easier to use for the developers.
   - About point 4: I renamed all the properties and parameters from
   "asOfTimestamp" to "fromTimestamp". That was my misunderstanding. So Now we
   have these two timestamp bounds: "fromTimestamp" and "untilTimestamp".
   - About point 5: Do we need system tests here? I assumed just
   integration tests were enough.
   - Regarding long vs timestamp instance: I think yes, that 's why I used
   Long as timestamp.

Bests,
Alieh






On Thu, Jul 27, 2023 at 2:28 PM Bruno Cadonna  wrote:

> Hi Alieh,
>
> Thanks for the KIP!
>
>
> Here my feedback.
>
> 1.
> You can remove the private fields and constructors from the KIP. Those
> are implementation details.
>
>
> 2.
> Some proposals for renamings
>
> in VersionedKeyQuery
>
> withKeyWithTimestampBound()
>-> withKeyAndAsOf()
>
> withKeyWithTimestampRange()
>-> withKeyAndTimeRange()
>
> in VersionedRangeQuery
>
> KeyRangeWithTimestampBound()
>-> withKeyRangeAndAsOf()
>
> withLowerBoundWithTimestampBound()
>-> withLowerBoundAndAsOf()
>
> withUpperBoundWithTimestampBound()
>-> withUpperBoundAndAsOf()
>
> withNoBoundWithTimestampBound()
>-> withNoBoundsAndAsOf
>
> keyRangeWithTimestampRange()
>-> withKeyRangeAndTimeRange()
>
> withLowerBoundWithTimestampRange()
>-> withLowerBoundAndTimeRange()
>
> withUpperBoundWithTimestampRange()
>-> withUpperBounfAndTimeRange()
>
> withNoBoundWithTimestampRange()
>-> withNoBoundsAndTimeRange()
>
>
> 3.
> Would it make sense to merge
> withKeyLatestValue(final K key)
> and
> withKeyAllVersions(final K key)
> into
> withKey(final K key, final VersionsQualifier versionsQualifier)
> where VersionsQualifier is an enum with values (ALL, LATEST). We could
> also add EARLIEST if we feel it might be useful.
> Same applies to all methods that end in LatestValue or AllVersions
>
>
> 4.
> I think getAsOfTimestamp() should not return the lower bound. If I query
> a version as of a timestamp then the query should return the latest
> version less than the timestamp.
> I propose to rename the getters to getTimeFrom() and getTimeTo() as in
> WindowRangeQuery.
>
>
> 5.
> Please add the Test Plan section.
>
>
> Regarding long vs Instant: Did we miss to use Instant instead of long
> for all interfaces of the versioned state stores?
>
>
> Best,
> Bruno
>
>
>
>
>
>
>
>
> On 7/26/23 11:40 PM, Matthias J. Sax wrote:
> > Thanks for the KIP Alieh. Glad to see that we can add IQ to the new
> > versioned stores!
> >
> >
> >
> > Couple of questions:
> >
> >> single-key lookup with timestamp (upper) bound
> >
> > Not sure if "bound" is the right term? In the end, it's a point lookup
> > for a key plus timestamps, so it's an as-of timestamp (not a bound)? Of
> > course, the returned record would most likely have a different (smaller)
> > timestamp, but that's expected but does not make the passed in timestamp
> > a "bound" IMHO?
> >
> >> single-key query with timestamp range
> >> single-key all versions query
> >
> > Should we also add `withLowerTimeBound` and `withUpperTimeBound`
> > (similar to what `RangeQuery` has)?
> >
> > Btw: I think we should not pass `long` for timestamps, but `Instance`
> > types.
> >
> > For time-range queries, do we iterate over the values in timestamp
> > ascending order? If yes, the interface should specify it? Also, would it
> > make sense to add reverse order (also ok to exclude and only do if there
> > is demand in a follow up KIP; if not, please add to "Rejected
> > alternatives" section).
> >
> > Also, for time-range query, what are the exact bound for stuff we
> > include? In the end, a value was a "valid range" (conceptually), so do
> > we include a record if it's valid range overlaps the search time-range,
> > or must it be fully included? Or would we only say, that the `validFrom`
> > timestamp that is stored must be in the search range (what implies that
> > the lower end would be a non-overlapping but "fully included" bound,
> > while the upper end would be a overlapping bound).
> >
> > For key-range / time-range queries: do we return the result in ``
> > order or `` order? Also, what about reverse iterators?
> >
> > About ` ValueIterator` -- think the JavaDocs have c&p error in it for
> > `peekNextRecord` (also, should it be called `peekNextValue`? (Also some
> > other JavaDocs seem to be incomplete and not describe all parameters?)
> >
> >
> > Thanks.
> >
> >
> >
> > -Matthias
> >
> >
> >
> > On 7/26/23 7:24 AM, Alieh Saeedi wrote:
> >> Hi all,
> >>
> >> I would like to propose a KIP to support IQv2 for versioned state
> stores.
> >>
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-960%3A+Support+interactive+queries+%28IQv2%29+for+versioned+state+stores
> >>
> >> Looking forward to your feedback!
> >>
> >> Cheers,
> >> Alieh
> 

Re: [DISCUSS] KIP-960: Support interactive queries (IQv2) for versioned state stores

2023-07-27 Thread Alieh Saeedi
Thanks, Matthias, for the feedback!

> Not sure if "bound" is the right term?
My understanding from your comment is that just the name "*bound*" is not
the right one, while having this query type makes sense.  Right? If *bound*
is a confusing word, we can simply change it to a better one. I do not have
a better idea at the moment. We can change it to "*whatever query type*
with untilTimestamp"? and change the corresponding method name from "
withWhateverWithTimestampBound" to "withWhateverWithUntilTimestamp"?

> Should we also add `withLowerTimeBound` and `withUpperTimeBound` (similar
to what `RangeQuery` has)?
In none of the range queries, we have a lower bound for time. If you look
at them, they have either both time bounds, no bounds (all-versions), or
just upper bound (UntilTimestamp as an input parameter). Do you think we
should add those types of queries as well? We can add them for both key and
range queries. I agree with you.

> Btw: I think we should not pass `long` for timestamps, but `Instance`
types.
Good point. Making the user convert the timestamp to a long does not make
sense.

> For time-range queries, do we iterate over the values in timestamp
ascending order?
That was my assumption, at least.
> If yes, the interface should specify it?
Mmmm. You mean having that in the method input parameters?
> Also, would it make sense to add reverse order
I think yes.

> Also, for time-range query, what are the exact bound for stuff we include?
I 'm not sure if I get what you mean. Your point is that we have the
following options and should choose one?!

   1. *(*asOfTimestamp, untilTimestamp*)*
   2. *(*asOfTimestamp, untilTimestamp*]*
   3. *[*asOfTimestamp, untilTimestamp*)*
   4. *[*asOfTimestamp, untilTimestamp*]*



> For key-range / time-range queries: do we return the result in ``
order or `` order? Also, what about reverse iterators?
I suggest having both. The user can specify that. About reverse iterators,
again, my idea is that it makes sense to have them.

> About ` ValueIterator` .
Thanks. I got that and corrected the KIP.



On Wed, Jul 26, 2023 at 11:40 PM Matthias J. Sax  wrote:

> Thanks for the KIP Alieh. Glad to see that we can add IQ to the new
> versioned stores!
>
>
>
> Couple of questions:
>
> > single-key lookup with timestamp (upper) bound
>
> Not sure if "bound" is the right term? In the end, it's a point lookup
> for a key plus timestamps, so it's an as-of timestamp (not a bound)? Of
> course, the returned record would most likely have a different (smaller)
> timestamp, but that's expected but does not make the passed in timestamp
> a "bound" IMHO?
>
> > single-key query with timestamp range
> > single-key all versions query
>
> Should we also add `withLowerTimeBound` and `withUpperTimeBound`
> (similar to what `RangeQuery` has)?
>
> Btw: I think we should not pass `long` for timestamps, but `Instance`
> types.
>
> For time-range queries, do we iterate over the values in timestamp
> ascending order? If yes, the interface should specify it? Also, would it
> make sense to add reverse order (also ok to exclude and only do if there
> is demand in a follow up KIP; if not, please add to "Rejected
> alternatives" section).
>
> Also, for time-range query, what are the exact bound for stuff we
> include? In the end, a value was a "valid range" (conceptually), so do
> we include a record if it's valid range overlaps the search time-range,
> or must it be fully included? Or would we only say, that the `validFrom`
> timestamp that is stored must be in the search range (what implies that
> the lower end would be a non-overlapping but "fully included" bound,
> while the upper end would be a overlapping bound).
>
> For key-range / time-range queries: do we return the result in ``
> order or `` order? Also, what about reverse iterators?
>
> About ` ValueIterator` -- think the JavaDocs have c&p error in it for
> `peekNextRecord` (also, should it be called `peekNextValue`? (Also some
> other JavaDocs seem to be incomplete and not describe all parameters?)
>
>
> Thanks.
>
>
>
> -Matthias
>
>
>
> On 7/26/23 7:24 AM, Alieh Saeedi wrote:
> > Hi all,
> >
> > I would like to propose a KIP to support IQv2 for versioned state stores.
> >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-960%3A+Support+interactive+queries+%28IQv2%29+for+versioned+state+stores
> >
> > Looking forward to your feedback!
> >
> > Cheers,
> > Alieh
> >
>


Re: [VOTE] KIP-919: Allow AdminClient to Talk Directly with the KRaft Controller Quorum and add Controller Registration

2023-07-27 Thread Ismael Juma
I think singular was used because that's the common case for the cli tools.
To be honest, it's actually a bit more confusing to have both singular and
plural for the cli tools and you have to remember the exact version for
each one. We should either support plural for both and the singular one is
just an alias for compatibility or stick with singular for both.

Ismael

On Thu, Jul 27, 2023, 12:03 AM Colin McCabe  wrote:

> On Wed, Jul 26, 2023, at 07:09, Ron Dagostino wrote:
> > Thanks, Colin.  +1 (binding) from me.
> >
> > I will note that Ziming mentioned in the DISCUSS thread that "There is
> > a mistake that we use `--bootstrap-server` instead of
> > `--bootstrap-server(s)`, so should we also change the new argument
> > `--bootstrap-controller` (no s).".  I agree that this is an
> > unfortunate historical artifact.  Furthermore, the config property is
> > bootstrap.servers, which does not match the command line argument
> > --bootstrap-server.  I don't know what we should do here -- I am not
> > sure that we should propagate it.  We could continue the same mismatch
> > and use --bootstrap-controller, but I feel like using
> > --bootstrap-controllers makes more sense.  Anyway, this is more an
> > ergonomic issue than anything else.  I am still +1 binding, but we
> > should arrive at an explicit decision here.
>
> Hi Ron,
>
> Thanks for the review and vote.
>
> The current KIP-919 text has "--bootstrap-controllers" for command-line
> and "bootstrap.controllers" for config.
>
> As far as I know, --bootstrap-server is singular rather than plural just
> for historical reasons so I figured the new flag can just be plural. As
> you mentioned, the configs have always been plural, so it's more consistent
> for the flags to match the configs.
>
> (Though obviously that ship has sailed on --bootstrap-server, we're not
> gonna change it now ;)
>
> >
> > Nits:
> > s/boostrap-servers/bootstrap-servers/
> > s/DescribeClusterResquest/DescribeClusterRequest/
> > s/ControllerRegistrationRecord/RegisterControllerRecord/g
> >
>
> Thanks. I fixed these typos and changed ControllerRegistrationRecord ->
> RegisterControllerRecord as suggested.
>
> best,
> Colin
>
>
> > Ron
> >
> > On Wed, Jul 26, 2023 at 10:06 AM David Arthur
> >  wrote:
> >>
> >> Thanks for driving this KIP, Colin!
> >>
> >> +1 binding
> >>
> >> -David
> >>
> >> On Wed, Jul 26, 2023 at 8:58 AM Divij Vaidya 
> >> wrote:
> >>
> >> > +1 (binding)
> >> >
> >> > --
> >> > Divij Vaidya
> >> >
> >> >
> >> > On Wed, Jul 26, 2023 at 2:56 PM ziming deng  >
> >> > wrote:
> >> > >
> >> > > +1 (binding) from me.
> >> > >
> >> > > Thanks for the KIP!
> >> > >
> >> > > --
> >> > > Ziming
> >> > >
> >> > > > On Jul 26, 2023, at 20:18, Luke Chen  wrote:
> >> > > >
> >> > > > +1 (binding) from me.
> >> > > >
> >> > > > Thanks for the KIP!
> >> > > >
> >> > > > Luke
> >> > > >
> >> > > > On Tue, Jul 25, 2023 at 1:24 AM Colin McCabe 
> >> > wrote:
> >> > > >
> >> > > >> Hi all,
> >> > > >>
> >> > > >> I'd like to start the vote for KIP-919: Allow AdminClient to Talk
> >> > Directly
> >> > > >> with the KRaft Controller Quorum and add Controller Registration.
> >> > > >>
> >> > > >> The KIP is here: https://cwiki.apache.org/confluence/x/Owo0Dw
> >> > > >>
> >> > > >> Thanks to everyone who reviewed the proposal.
> >> > > >>
> >> > > >> best,
> >> > > >> Colin
> >> > > >>
> >> > >
> >> >
> >>
> >>
> >> --
> >> -David
>


Re: [DISCUSS] KIP-960: Support interactive queries (IQv2) for versioned state stores

2023-07-27 Thread Bruno Cadonna

Hi Alieh,

Thanks for the KIP!


Here my feedback.

1.
You can remove the private fields and constructors from the KIP. Those 
are implementation details.



2.
Some proposals for renamings

in VersionedKeyQuery

withKeyWithTimestampBound()
  -> withKeyAndAsOf()

withKeyWithTimestampRange()
  -> withKeyAndTimeRange()

in VersionedRangeQuery

KeyRangeWithTimestampBound()
  -> withKeyRangeAndAsOf()

withLowerBoundWithTimestampBound()
  -> withLowerBoundAndAsOf()

withUpperBoundWithTimestampBound()
  -> withUpperBoundAndAsOf()

withNoBoundWithTimestampBound()
  -> withNoBoundsAndAsOf

keyRangeWithTimestampRange()
  -> withKeyRangeAndTimeRange()

withLowerBoundWithTimestampRange()
  -> withLowerBoundAndTimeRange()

withUpperBoundWithTimestampRange()
  -> withUpperBounfAndTimeRange()

withNoBoundWithTimestampRange()
  -> withNoBoundsAndTimeRange()


3.
Would it make sense to merge
withKeyLatestValue(final K key)
and
withKeyAllVersions(final K key)
into
withKey(final K key, final VersionsQualifier versionsQualifier)
where VersionsQualifier is an enum with values (ALL, LATEST). We could 
also add EARLIEST if we feel it might be useful.

Same applies to all methods that end in LatestValue or AllVersions


4.
I think getAsOfTimestamp() should not return the lower bound. If I query 
a version as of a timestamp then the query should return the latest 
version less than the timestamp.
I propose to rename the getters to getTimeFrom() and getTimeTo() as in 
WindowRangeQuery.



5.
Please add the Test Plan section.


Regarding long vs Instant: Did we miss to use Instant instead of long 
for all interfaces of the versioned state stores?



Best,
Bruno








On 7/26/23 11:40 PM, Matthias J. Sax wrote:
Thanks for the KIP Alieh. Glad to see that we can add IQ to the new 
versioned stores!




Couple of questions:


single-key lookup with timestamp (upper) bound


Not sure if "bound" is the right term? In the end, it's a point lookup 
for a key plus timestamps, so it's an as-of timestamp (not a bound)? Of 
course, the returned record would most likely have a different (smaller) 
timestamp, but that's expected but does not make the passed in timestamp 
a "bound" IMHO?



single-key query with timestamp range
single-key all versions query


Should we also add `withLowerTimeBound` and `withUpperTimeBound` 
(similar to what `RangeQuery` has)?


Btw: I think we should not pass `long` for timestamps, but `Instance` 
types.


For time-range queries, do we iterate over the values in timestamp 
ascending order? If yes, the interface should specify it? Also, would it 
make sense to add reverse order (also ok to exclude and only do if there 
is demand in a follow up KIP; if not, please add to "Rejected 
alternatives" section).


Also, for time-range query, what are the exact bound for stuff we 
include? In the end, a value was a "valid range" (conceptually), so do 
we include a record if it's valid range overlaps the search time-range, 
or must it be fully included? Or would we only say, that the `validFrom` 
timestamp that is stored must be in the search range (what implies that 
the lower end would be a non-overlapping but "fully included" bound, 
while the upper end would be a overlapping bound).


For key-range / time-range queries: do we return the result in `` 
order or `` order? Also, what about reverse iterators?


About ` ValueIterator` -- think the JavaDocs have c&p error in it for 
`peekNextRecord` (also, should it be called `peekNextValue`? (Also some 
other JavaDocs seem to be incomplete and not describe all parameters?)



Thanks.



-Matthias



On 7/26/23 7:24 AM, Alieh Saeedi wrote:

Hi all,

I would like to propose a KIP to support IQv2 for versioned state stores.

https://cwiki.apache.org/confluence/display/KAFKA/KIP-960%3A+Support+interactive+queries+%28IQv2%29+for+versioned+state+stores

Looking forward to your feedback!

Cheers,
Alieh



Re: Apache Kafka 3.6.0 release

2023-07-27 Thread Ismael Juma
Hi Divij,

Some of these are launch checklist items (not really goals) and some are
compatibility guarantees. More below.

On Thu, Jul 27, 2023, 12:10 PM Divij Vaidya  wrote:

> Hey Satish
>
> Could we consider adding "launch goals" in the release plan. While
> some of these may be implicit, it would be nice to list them down in
> the release plan. For this release, our launch requirements would be:
> 1. Users should be able to upgrade from any prior Kafka version to this
> version.
>

This is part of the compatibility guarantees. The upgrade notes mention
this already. If there is a change in a given release, it should definitely
be highlighted.

2. On release, this version (or it's dependencies) would not have any
> known MEDIUM/HIGH CVE.
>

This is a new policy and the details should be discussed. In particular,
the threshold (medium or high).

3. Presence of any "early access"/"beta" feature should not impact
> other production features when it is not enabled.
>

This is a general guideline for early access features and not specific to
this release. It would be good to have a page that talks about these things.

4. Once enabled, users should have an option to disable any "early
> access"/"beta" feature and resume normal production features, i.e.
> impact of beta features should be reversible.
>

This needs discussion and I don't think it's reasonable as a general rule.
For example, Kraft early access wasn't reversible and it was not feasible
for it to be.

5. KIP-405 will be available in "early access"/"beta" mode. Early
> access/beta means that the public facing interfaces won't change in
> future but the implementation is not recommended to be used in
> production.


I don't think it's ok to make this a requirement. Early access is a way to
get early feedback and all types of changes should be on the table. They
would be discussed via KIPs as usual. I believe there were some
incompatible changes for Kraft during the early access period although the
team aimed to minimize work required during upgrades. I have mentioned
Kraft a couple of times since it's a good example of a large feature that
went through this process.

Ismael


Re: [DISCUSS] KIP-954: expand default DSL store configuration to custom types

2023-07-27 Thread Bruno Cadonna

Hi,

A5. I have to admit that
"If we envision extending this beyond just StoreSupplier types, it could 
be a good option."

is scaring me a bit.
I am wondering what would be an example for such an extension?
In general, I would propose to limit the scope of a config. In this case 
the config should provide suppliers for state stores for the DSL.


BTW, maybe it is a good idea to let DslStorePlugin extend Configurable.

Best,
Bruno

On 7/27/23 2:15 AM, Sophie Blee-Goldman wrote:

Thanks for the feedback Bruno -- sounds like we're getting close to a final
consensus here.
It sounds like the two main (only?) semi-unresolved questions that still
have differing
opinions floating around are whether to deprecate the old config, and what
to name the new config
+ interface.

Although I won't personally push back on any of the options listed above,
here's my final two cents:

A3. I'm still a firm believer in deprecating the old config, and agree
wholeheartedly with what Bruno said.

A5. I also wasn't crazy about "Plugin" at first, but I will admit it's
grown on me. I think it rubbed me the wrong
way at  first because it's just not part of the standard vocabulary in
Streams so far. If we envision extending
this beyond just StoreSupplier types, it could be a good option.
DSLStoreSuppliers does make a lot of sense,
though.

To throw out a few more ideas in case any of them stick, what about
something like DSLStoreFormat or
DSLStorageType, or even DSLStorageEngine? Or even DSLStoreFactory -- the
Stores class is described as
a "factory" (though not named so) and, to me, is actually quite comparable
-- both are providers not of the
stores themselves, but of the basic building blocks of Stores (eg
StoreSuppliers)

Ultimately fine with anything though. We should try not to drag out the KIP
discussion too long once it's down
to just nits :P

Cheers,
Sophie



On Wed, Jul 26, 2023 at 8:04 AM Almog Gavra  wrote:


Thanks for the comments Bruno!

A3. Oops... I think I didn't do a great job updating the KIP to reflect
Guozhang's suggestion. This seems like the last point of contention, where
we have two options:

1. Deprecate the config entirely and replace IN_MEMORY/ROCKSDB with
implementations of the DslStorePlugin
2. (What's currently in the KIP) Introduce a new config which defaults to
DefaultDslStorePlugin and only the DefaultDslStorePlugin will respect the
old default.store.type config

I'm happy with either, I'll keep the KIP with (2) for now as that seemed
like the result of the previous discussion but I have no problem changing
it back to (1) which was the original proposal.

A5. I like "DslStorePlugin" because it leaves room for configuring
implementations beyond just supplying stores (e.g. we could introduce a
`configure()` method etc...). I'll keep it as is for now (and change
Materialized/Stores API sections - thanks for catching that)! I don't feel
too strongly and wouldn't dig my heels in if most people preferred
"DslStoreSuppliers" (I don't love DslStores as it resembles the Stores
class to closely in name and they're a little different).

A6. Yup, that's the suggestion.

- Almog

On Wed, Jul 26, 2023 at 6:38 AM Bruno Cadonna  wrote:


Hi,

Sorry for being late to the party!

A1: I agree with Sophie, Guozhang, and Almog not to block the KIP on
gaps in the implementation.

A2: I am happy with not considering anything special w.r.t. versioned
state stores in this KIP.

A3: Here I agree with Sophie to deprecate the old config. I would also
not use config value CUSTOM. Having two configs that sometimes depend on
each other to configure one single concept seems confusing to me. I see
future me looking at default.dsl.store = IN_MEMORY and wondering why
something is written to disk because I did not check config
dsl.store.plugin.class?
BTW, the KIP in its current version is not clear about whether
default.dsl.store will be deprecated or not. In "Compatibility,
Deprecation, and Migration Plan" it says default.dsl.store will be
deprecated but in "Configuration" default.dsl.store seems to be an
essential part of the configuration.

A4: I agree

A5: I do not completely like the name "DslStorePlugin". What about
naming it simply "DslStores" or "DslStoreSuppliers"? If we decide to
rename we should also rename dsl.store.plugin.class to
dsl.store.suppliers.class or similar.
BTW, I think you missed to rename some occurrences in section
"Materialized API" especially in the code section "Stores.java".

A6: Actually I am not sure if I completely follow here. Is this about
the static methods in class Stores? If yes, I agree with Almog to keep
this out of the KIP.

Best,
Bruno

On 7/26/23 5:20 AM, Almog Gavra wrote:

I have updated the KIP with the points as discussed above. @Guozhang,

the

suggested configuration makes it a little more awkward around the
Materialized.as and Materialized.withStoreType APIs than it was when we
were totally deprecating the old configuration. Let me know what you

think.


I will open the voting tomorr

Re: Apache Kafka 3.6.0 release

2023-07-27 Thread Divij Vaidya
Hey Satish

Could we consider adding "launch goals" in the release plan. While
some of these may be implicit, it would be nice to list them down in
the release plan. For this release, our launch requirements would be:
1. Users should be able to upgrade from any prior Kafka version to this version.
2. On release, this version (or it's dependencies) would not have any
known MEDIUM/HIGH CVE.
3. Presence of any "early access"/"beta" feature should not impact
other production features when it is not enabled.
4. Once enabled, users should have an option to disable any "early
access"/"beta" feature and resume normal production features, i.e.
impact of beta features should be reversible.
5. KIP-405 will be available in "early access"/"beta" mode. Early
access/beta means that the public facing interfaces won't change in
future but the implementation is not recommended to be used in
production.

Thoughts?

--
Divij Vaidya

On Wed, Jul 26, 2023 at 6:31 PM Hector Geraldino (BLOOMBERG/ 919 3RD
A)  wrote:
>
> Yes, still need one more binding vote to pass. I'll send a reminder if the 
> vote is still pending after the waiting period.
>
> Cheers,
>
> From: dev@kafka.apache.org At: 07/26/23 12:17:10 UTC-4:00To:  
> dev@kafka.apache.org
> Subject: Re: Apache Kafka 3.6.0 release
>
> Hi Hector/Yash,
> Are you planning to reach out to other committers to vote on the KIP
> and close the vote in the next couple of days?
>
> Thanks,
> Satish.
>
> On Wed, 26 Jul 2023 at 20:08, Yash Mayya  wrote:
> >
> > Hi Hector,
> >
> > KIP-959 actually still requires 2 more binding votes to be accepted (
> > https://cwiki.apache.org/confluence/display/KAFKA/Bylaws#Bylaws-Approvals).
> > The non-binding votes from people who aren't committers (including myself)
> > don't count towards the required lazy majority.
> >
> > Thanks,
> > Yash
> >
> > On Wed, Jul 26, 2023 at 7:35 PM Satish Duggana 
> > wrote:
> >
> > > Hi Hector,
> > > Thanks for the update on KIP-959.
> > >
> > > ~Satish.
> > >
> > > On Wed, 26 Jul 2023 at 18:38, Hector Geraldino (BLOOMBERG/ 919 3RD A)
> > >  wrote:
> > > >
> > > > Hi Satish,
> > > >
> > > > I added KIP-959 [1] to the list. The KIP has received enough votes to
> > > pass, but I'm waiting the 72 hours before announcing the results. There's
> > > also a (small) PR with the implementation for this KIP that hopefully will
> > > get reviewed/merged soon.
> > > >
> > > > Best,
> > > >
> > > > [1]
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-959%3A+Add+BooleanConverte
> r+to+Kafka+Connect
> > > >
> > > > From: dev@kafka.apache.org At: 06/12/23 06:22:00 UTC-4:00To:
> > > dev@kafka.apache.org
> > > > Subject: Re: Apache Kafka 3.6.0 release
> > > >
> > > > Hi,
> > > > I have created a release plan for Apache Kafka version 3.6.0 on the
> > > > wiki. You can access the release plan and all related information by
> > > > following this link:
> > > > https://cwiki.apache.org/confluence/display/KAFKA/Release+Plan+3.6.0
> > > >
> > > > The release plan outlines the key milestones and important dates for
> > > > version 3.6.0. Currently, the following dates have been set for the
> > > > release:
> > > >
> > > > KIP Freeze: 26th July 23
> > > > Feature Freeze : 16th Aug 23
> > > > Code Freeze : 30th Aug 23
> > > >
> > > > Please review the release plan and provide any additional information
> > > > or updates regarding KIPs targeting version 3.6.0. If you have
> > > > authored any KIPs that are missing a status or if there are incorrect
> > > > status details, please make the necessary updates and inform me so
> > > > that I can keep the plan accurate and up to date.
> > > >
> > > > Thanks,
> > > > Satish.
> > > >
> > > > On Mon, 17 Apr 2023 at 21:17, Luke Chen  wrote:
> > > > >
> > > > > Thanks for volunteering!
> > > > >
> > > > > +1
> > > > >
> > > > > Luke
> > > > >
> > > > > On Mon, Apr 17, 2023 at 2:03 AM Ismael Juma  wrote:
> > > > >
> > > > > > Thanks for volunteering Satish. +1.
> > > > > >
> > > > > > Ismael
> > > > > >
> > > > > > On Sun, Apr 16, 2023 at 10:08 AM Satish Duggana <
> > > satish.dugg...@gmail.com>
> > > > > > wrote:
> > > > > >
> > > > > > > Hi,
> > > > > > > I would like to volunteer as release manager for the next release,
> > > > > > > which will be Apache Kafka 3.6.0.
> > > > > > >
> > > > > > > If there are no objections, I will start a release plan a week
> > > after
> > > > > > > 3.5.0 release(around early May).
> > > > > > >
> > > > > > > Thanks,
> > > > > > > Satish.
> > > > > > >
> > > > > >
> > > >
> > > >
> > >
>
>


Build failed in Jenkins: Kafka » Kafka Branch Builder » 3.5 #51

2023-07-27 Thread Apache Jenkins Server
See 


Changes:


--
[...truncated 471806 lines...]
> Task :raft:compileTestJava UP-TO-DATE
> Task :raft:testClasses UP-TO-DATE
> Task :connect:json:testSrcJar
> Task :streams:generateMetadataFileForMavenJavaPublication
> Task :group-coordinator:compileTestJava UP-TO-DATE
> Task :group-coordinator:testClasses UP-TO-DATE
> Task :metadata:compileTestJava UP-TO-DATE
> Task :metadata:testClasses UP-TO-DATE
> Task :clients:generateMetadataFileForMavenJavaPublication

> Task :connect:api:javadoc
/home/jenkins/workspace/Kafka_kafka_3.5/connect/api/src/main/java/org/apache/kafka/connect/source/SourceRecord.java:44:
 warning - Tag @link: reference not found: org.apache.kafka.connect.data
1 warning

> Task :connect:api:copyDependantLibs UP-TO-DATE
> Task :connect:api:jar UP-TO-DATE
> Task :connect:api:generateMetadataFileForMavenJavaPublication
> Task :connect:json:copyDependantLibs UP-TO-DATE
> Task :connect:json:jar UP-TO-DATE
> Task :connect:json:generateMetadataFileForMavenJavaPublication
> Task :connect:api:javadocJar
> Task :connect:json:publishMavenJavaPublicationToMavenLocal
> Task :connect:json:publishToMavenLocal
> Task :connect:api:compileTestJava UP-TO-DATE
> Task :connect:api:testClasses UP-TO-DATE
> Task :connect:api:testJar
> Task :connect:api:testSrcJar
> Task :connect:api:publishMavenJavaPublicationToMavenLocal
> Task :connect:api:publishToMavenLocal
> Task :streams:javadoc
> Task :streams:javadocJar

> Task :clients:javadoc
/home/jenkins/workspace/Kafka_kafka_3.5/clients/src/main/java/org/apache/kafka/clients/admin/ScramMechanism.java:32:
 warning - Tag @see: missing final '>': "https://cwiki.apache.org/confluence/display/KAFKA/KIP-554%3A+Add+Broker-side+SCRAM+Config+API";>KIP-554:
 Add Broker-side SCRAM Config API

 This code is duplicated in 
org.apache.kafka.common.security.scram.internals.ScramMechanism.
 The type field in both files must match and must not change. The type field
 is used both for passing ScramCredentialUpsertion and for the internal
 UserScramCredentialRecord. Do not change the type field."
/home/jenkins/workspace/Kafka_kafka_3.5/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/package-info.java:21:
 warning - Tag @link: reference not found: 
org.apache.kafka.common.security.oauthbearer
2 warnings

> Task :clients:javadocJar
> Task :clients:srcJar
> Task :clients:testJar
> Task :clients:testSrcJar
> Task :clients:publishMavenJavaPublicationToMavenLocal
> Task :clients:publishToMavenLocal
> Task :core:compileScala
> Task :core:classes
> Task :core:compileTestJava NO-SOURCE
> Task :core:compileTestScala
> Task :core:testClasses
> Task :streams:compileTestJava UP-TO-DATE
> Task :streams:testClasses UP-TO-DATE
> Task :streams:testJar
> Task :streams:testSrcJar
> Task :streams:publishMavenJavaPublicationToMavenLocal
> Task :streams:publishToMavenLocal

Deprecated Gradle features were used in this build, making it incompatible with 
Gradle 9.0.

You can use '--warning-mode all' to show the individual deprecation warnings 
and determine if they come from your own scripts or plugins.

See 
https://docs.gradle.org/8.0.2/userguide/command_line_interface.html#sec:command_line_warnings

BUILD SUCCESSFUL in 3m 1s
89 actionable tasks: 33 executed, 56 up-to-date
[Pipeline] sh
+ grep ^version= gradle.properties
+ cut -d= -f 2
[Pipeline] dir
Running in /home/jenkins/workspace/Kafka_kafka_3.5/streams/quickstart
[Pipeline] {
[Pipeline] sh
+ mvn clean install -Dgpg.skip
[INFO] Scanning for projects...
[INFO] 
[INFO] Reactor Build Order:
[INFO] 
[INFO] Kafka Streams :: Quickstart[pom]
[INFO] streams-quickstart-java[maven-archetype]
[INFO] 
[INFO] < org.apache.kafka:streams-quickstart >-
[INFO] Building Kafka Streams :: Quickstart 3.5.2-SNAPSHOT[1/2]
[INFO]   from pom.xml
[INFO] [ pom ]-
[INFO] 
[INFO] --- clean:3.0.0:clean (default-clean) @ streams-quickstart ---
[INFO] 
[INFO] --- remote-resources:1.5:process (process-resource-bundles) @ 
streams-quickstart ---
[INFO] 
[INFO] --- site:3.5.1:attach-descriptor (attach-descriptor) @ 
streams-quickstart ---
[INFO] 
[INFO] --- gpg:1.6:sign (sign-artifacts) @ streams-quickstart ---
[INFO] 
[INFO] --- install:2.5.2:install (default-install) @ streams-quickstart ---
[INFO] Installing 
/home/jenkins/workspace/Kafka_kafka_3.5/streams/quickstart/pom.xml to 
/home/jenkins/.m2/repository/org/apache/kafka/streams-quickstart/3.5.2-SNAPSHOT/streams-quickstart-3.5.2-SNAPSHOT.pom
[INFO] 
[INFO] --< org.apache.kafka:streams-quickstart-java >--
[INFO] Building streams-quickstart-java 3.5.2-SNAPSHOT[2/2]
[INFO]   from java/pom.xml

[jira] [Resolved] (KAFKA-15251) Upgrade system test to use 3.5.1

2023-07-27 Thread Divij Vaidya (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-15251?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Divij Vaidya resolved KAFKA-15251.
--
Resolution: Fixed

> Upgrade system test to use 3.5.1
> 
>
> Key: KAFKA-15251
> URL: https://issues.apache.org/jira/browse/KAFKA-15251
> Project: Kafka
>  Issue Type: Test
>  Components: streams, system tests
>Reporter: Matthias J. Sax
>Assignee: Divij Vaidya
>Priority: Major
> Fix For: 3.6.0, 3.5.1
>
>
> 3.5.1 was released and we should update the upgrade system tests accordingly 
> to use the new version



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


Jenkins build is still unstable: Kafka » Kafka Branch Builder » trunk #2046

2023-07-27 Thread Apache Jenkins Server
See 




plz help me code review

2023-07-27 Thread Xiangyuan LI
Hi kafka team:
  I raise a pr https://github.com/apache/kafka/pull/13965 to fix
https://issues.apache.org/jira/browse/KAFKA-15106,
  it mentions some serious bug and no one check it for a long time, plz
help me review if could. thx!

 these bugs are
in 
org.apache.kafka.clients.consumer.internals.AbstractStickyAssignor.ConstrainedAssignmentBuilder#ConstrainedAssignmentBuilder
and all of them could cause rebalance stuck for ever, and they could occur
in production environment, very, very, very easy.


[jira] [Created] (KAFKA-15261) ReplicaFetcher thread should not block if RLMM is not initialized

2023-07-27 Thread Abhijeet Kumar (Jira)
Abhijeet Kumar created KAFKA-15261:
--

 Summary: ReplicaFetcher thread should not block if RLMM is not 
initialized
 Key: KAFKA-15261
 URL: https://issues.apache.org/jira/browse/KAFKA-15261
 Project: Kafka
  Issue Type: Sub-task
Reporter: Abhijeet Kumar
Assignee: Abhijeet Kumar


While building remote log aux state, the replica fetcher fetches the remote log 
segment metadata. If the TBRLMM is not initialized yet, the call blocks. Since 
replica fetchers share a common lock, it prevents other replica fetchers from 
running as well. Also the same lock is shared in the handle LeaderAndISR 
request path, hence those calls get blocked as well.

Instead, replica fetcher should check if RLMM is initialized before attempting 
to fetch the remote log segment metadata. If RLMM is not initialized, it should 
throw a retryable error so that it can be retried later, and also does not 
block other operations.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-15260) RLM Task should wait until RLMM is initialized before copying segments to remote

2023-07-27 Thread Abhijeet Kumar (Jira)
Abhijeet Kumar created KAFKA-15260:
--

 Summary: RLM Task should wait until RLMM is initialized before 
copying segments to remote
 Key: KAFKA-15260
 URL: https://issues.apache.org/jira/browse/KAFKA-15260
 Project: Kafka
  Issue Type: Sub-task
Reporter: Abhijeet Kumar


The RLM Task uploads segment to the remote storage for its leader partitions 
and after each upload it sends a message 'COPY_SEGMENT_STARTED' to the Topic 
based RLMM topic and then waits for the TBRLMM to consume the message before 
continuing.

If the RLMM is not initialized, TBRLMM may not be able to consume the message 
within the stipulated time and timeout and RLMM will repeat later. It make take 
a few mins for the TBRLMM to initialize during which RLM Task will keep timing 
out.

Instead the RLM task should wait until RLMM is initialized before attempting to 
copy segments to remote storage.

 



--
This message was sent by Atlassian Jira
(v8.20.10#820010)