Re: [DISCUSS] KIP-1087: Removing intermediateTopicsOption from StreamsResetter

2024-09-02 Thread Lucas Brutschy
Hi Arnav,

thanks for the KIP!

I agree with it, I don't have any comments.

I think it could be an option to open the vote immediately, since it's
not going to be a controversial change.

Cheers
Lucas

On Sun, Sep 1, 2024 at 10:19 PM Arnav Dadarya  wrote:
>
> Hi all,
>
> I would like to start a discussion thread on KIP-1087 to remove the
> intermediateTopicsOption from the StreamsResetter tool.
>
> The KIP can be found here:
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-1087%3A+Removing+intermediateTopicsOption+from+StreamsResetter
>
> I look forward to the discussion and your feedback.
>
> Thanks,
> Arnav Dadarya


Re: [VOTE] KIP-1042: Support for wildcard when creating new acls

2024-09-02 Thread Claude Warren, Jr
Colin,

I would like to leave the framework that is in KAFKA-17316 as it makes
testing the new implementations easier.  But let's discuss that elsewhere
(I will start another discussion for 17316 and 17423 together).

PatternType.GLOB makes sense, I will adjust the KIP to use that
terminology.  I understand from some of my compatriots that there is a
desire to allow GLOBs in resource names, Kafka Principals and Host names.
(Basically all the non-enum based ACL data).

With the Trie implementation for the resource names we can force the trie
to put the '*' and '?' on individual nodes when it encounters them.  This
makes them a natural pivot point when traversing the trie, It is how I
implemented it initially.  The idea is that the more "specific" an ACL is
the better a match it is assumed to be.  So the searching algorithm is
start descending the trie until a matching DENY ACL, a matching LITERAL
ACL, or leaf node is found.  A DENY ACL yields a DENY, a matching LITERAL
ACL yields its specified result.  A non matching leaf node requires further
searching.  We start back up the path starting with the leaf node, looking
for matching PREFIXED nodes.  If a GLOB symbol ('*' or '?') is located as
the child of the current node check for a GLOB match.  Continue until a
PREFIXED ACL match, GLOB ACL Match or the root node is encountered.
PEFIXED and GLOB return their specified value, root return a NOT FOUND
state.

With respect to the Principal and Host names,  I have been recently working
on Apache Rat moving maven based file exclusion to the core and have found
that the Plexus pattern match and selector code is very fast and would
probably not be a significant change in performance from straight check to
GLOB checking.  In these cases we would create what the plexus code calls a
"MatchPattern" and for collections of patterns "MatchPatterns".  We can
then simply check if the requested ACL matches the pattern.  The
MatchPattern will return the original text so we can perform the standard
string match we do today if desired.

I think that the KIP will need updating and that this will require changes
in the Client, Server and Metadata components.  I also believe that we
should prohibit the use of "java.util.stream" classes in the
metadata/authorizer package.  Currently it is only used in a couple of test
cases, but the overhead of streams would kill the authorizer performance so
best to restrict it before someone brings code.

Thoughts?

Claude



On Fri, Aug 30, 2024 at 4:38 PM Colin McCabe  wrote:

> Hi Claude,
>
> I think this is great work. Speeding up the Authorizer will be a big win
> for us.
>
> I don't think we need to add additional interfaces for this, though. Just
> get rid of the old slower implementation that I wrote, and replace it with
> your newer, faster one.
>
> Also, I think we should continue the discussion about globs... after all,
> it is the topic of this KIP! (You are calling it "wildcards" but I don't
> think that is good terminiology since it will cause confusion with the
> existing "*" wildcard). Anyway, earlier I requested that you add
> PatternType.GLOB and use it for these things. Does that make sense? I don't
> see another path to doing it compatibly. I certainly wouldn't want to
> create a "Python 2 vs. Python 3" type situation where people get stuck on
> an older authorizer fork because the new one requires globs and they can't
> handle that. No authorizer forks (or at least, not for things like this.)
>
> best,
> Colin
>
>
> On Fri, Aug 30, 2024, at 01:32, Claude Warren, Jr wrote:
> > I made it easier to replace the existing StandardAuthorizerData with a
> > different implementation in order show the Trie implementation met all
> the
> > requirements of the StandardAuthorizerData and could be replaced without
> > changing the StandardAuthorizer implementation.
> >
> > Replacing the current StandardAuthorizerData with a Trie
> > implementation makes sense because it is an order of magnitude faster.
> > This means that when we go to implement the Wildcard types we can perform
> > the search in times that are equivalent to the literal search times in
> the
> > current StandardAuthorizerData implementation.
> >
> > The changes for the first pull request is to create an interface for
> > AuthorizerData and create an "authorizeByResourceType" method within that
> > interface.  This adds an initial implementation in StandardAuthorizerData
> > that mirrors the default implementation found in the Authorizer
> interface.
> >
> > The Trie implementation is not dependent upon other classes local to
> > StandardAuthorizerData other than the MatchingRule and some of its
> > implementations which are now found in the AuthorizerData interface
> instead
> > of the StandardAuthorizerData implementation.  Obviously it is dependent
> > upon AuthorizationResult, AclBinding, KafkaPrincipal and other classes
> that
> > are defined elsewhere.
> >
> > In addition, abstract test cases are implemented to validate

[DISCUSS] KAFKA-17316 and KAFKA-17423

2024-09-02 Thread Claude Warren, Jr
I have been working on implementing a Trie structure to store ACLs and
improve the performance in the metadata/authorization code.  The upshot of
this was that I found it very difficult to determine if the implementation
was correctly reimplementing the current implementation.

My goal was to simply change the StandardAuthorizerData implementation and
leave the rest of the code as is.  However, there were no abstracted tests
that would allow me to test this.

KAFKA-17316 addresses this issue by creating some internal interfaces for
the "metadata/authorizer" package.  The one change to the
StandardAuthorizer was to implement the"authorizeByResourceType" defined in
the "org.apache.kafka.server.authorizer.Authorizer" interface by passing
the request down to the AuthorizerData implementation.

This change allowed me to create three test implementations.  One that
implemented "authorizeByResourceType" as it is in the released code base,
one that verified that the StandardAuthorizerData implementation did not
change the expected results, and one that showed the Trie implementation in
KAFKA-17423 was also correct.

I think that retaining the work in KAFKA-17316 makes sense as when the next
faster implementation comes along we can drop in the replacement and verify
that it works correctly.

KAFKA-17423 builds on KAFKA-17316 by implementing a Trie based
AuthorizerData implementation.  By splitting the data into a Trie format
the search for matching ACLs is improved by an order of magnitude.  The
trie implementation allows us to quickly locate the candidate ACLs by
splitting them into groups based upon the similarity of the resource name.
In addition since we are moving through the trie based on resource name we
have several advantages:

   1. If we encounter a matching DENY while descending the Trie we can stop
   as it overrides anything that may be found at lower levels.
   2. We only look for LITERAL matches on the descent.  If we reach a
   matching resource name or a leaf node we know there are no LITERAL matches.
   3. If we don't have a DENY or a LITERAL match we walk back up the path
   checking the nodes from the descent looking for a PREFIX match.  The
   advantage here is that we don't have to search again but can simply retrace
   the path using the Trie structure.

I believe that #1 and #2 above account for a significant portion of the
speed increase as we do not have to reposition within a sorted list of all
ACLs using a binary search.

Finally, I think that we should prohibit the use of the java.util.stream
classes within the authorizer due to hot path speed considerations.  The
only existing code that uses streams within that package were test cases.
We can prohibit the use by a simple checkstyle prohibition.  Doing so will
short circuit any misguided potential changes.

Thoughs?
Claude


[jira] [Resolved] (KAFKA-17442) Implement persister error handling and make write calls async

2024-09-02 Thread Apoorv Mittal (Jira)


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

Apoorv Mittal resolved KAFKA-17442.
---
Resolution: Fixed

> Implement persister error handling and make write calls async
> -
>
> Key: KAFKA-17442
> URL: https://issues.apache.org/jira/browse/KAFKA-17442
> Project: Kafka
>  Issue Type: Sub-task
>Reporter: Apoorv Mittal
>Assignee: Apoorv Mittal
>Priority: Major
>




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


Re: [DISCUSS] KIP-1071: Streams Rebalance Protocol

2024-09-02 Thread Bruno Cadonna

Hi all,

For your info, I updated the StreamsGroupInitialize request with the 
following changes:


1. I added the topology ID to the request so that the group coordinator 
knows for which topology it got the initialization.


2. I renamed field "Subtopology" to "SubtopologyId" since the field is 
the ID of the subtopology but that was not clear from the name.


Best,
Bruno


On 8/28/24 2:06 PM, Lucas Brutschy wrote:

Hi Sophie,

Thanks for your detailed comments - much appreciated! I think you read
a version of the KIP that did not yet include the admin command-line
tool and the Admin API extensions, so some of the comments are already
addressed in the KIP.

S1. StreamsGroupHeartbeat and StreamsGroupInitialize are called in the
consumer background thread. Note that in the new consumer threading
model, all RPCs are run by the background thread. Check out this:
https://cwiki.apache.org/confluence/display/KAFKA/Consumer+threading+refactor+design
for more information. Both our RPCs are just part of the group
membership management and do not need to be invoked explicitly by the
application thread. You could imagine calling the initialize RPC
explicitly (to implement explicit initialization), but this would
still mean sending an event to the background thread, and the
background thread in turn invokes the RPC. However, explicit
initialization would require some additional public interfaces that we
are not including in this KIP. StreamsGroupDescribe is called by the
AdminClient, and used by the command-line tool
kafka-streams-groups.sh.

S2. I think the max.warmup.replicas=100 suggested by Nick was intended
as the upper limit for setting the group configuration on the broker.
Just to make sure that this was not a misunderstanding. By default,
values above 100 should be rejected when setting a specific value for
group. Are you suggesting 20 or 30 for the default value for groups,
or the default upper limit for the group configuration?

S3. Yes, it's supposed to be used like SHUTDOWN_APPLICATION. The
MemberEpoch=-1 is a left-over from an earlier discussion. It means
that the member is leaving the group, so the intention was that the
member must leave the group when it asks the other members to shut
down. We later reconsidered this and decided that all applications
should just react to the shutdown application signal that is returned
by the broker, so the client first sets the ShutdownApplication and
later leaves the group. Thanks for spotting this, I removed it.

S4. Not sure if this refers to the latest version of the KIP. We added
an extension of the admin API and a kafka-streams-groups.sh
command-line tool to the KIP already.

S5. All RPCs for dealing with offsets will keep working with streams
groups. The extension of the admin API is rather cosmetic, since the
method names use "consumer group". The RPCs, however, are generic and
do not need to be changed.

S6. Yes, you can use the DeleteGroup RPC with any group ID, whether
streams group or not.

S7. See the admin API section.

S8. I guess for both A and B, I am not sure what you are suggesting.
Do you want to make the broker-side topology immutable and not include
any information about the topology, like the topology ID in the RPC?
It would seem that this would be a massive food-gun for people, if
they start changing their topology and don't notice that the broker is
looking at a completely different version of the topology. Or did you
mean that there is some kind of topology ID, so that at least we can
detect inconsistencies between broker and client-side topologies, and
we fence out any member with an incorrect topology ID? Then we seem to
end up with mostly the same RPCs and the same questions (how is the
topology ID generated?). I agree that the latter could be an option.
See summary at the end of this message.

S9. If we flat out refuse topology updates, agree - we cannot let the
application crash on minor changes to the topology. However, if we
allow topology updates as described in the KIP, there are only upsides
to making the topology ID more sensitive. All it will cause is that a
client will have to resend a `StreamsGroupInitialize`, and during the
rolling bounce, older clients will not get tasks assigned.

S10. The intention in the KIP is really just this - old clients can
only retain tasks, but not get new ones. If the topology has
sub-topologies, these will initially be assigned to the new clients,
which also have the new topology. You are right that rolling an
application where the old structure and the new structure are
incompatible (e.g. different subtopologies access the same partition)
will cause problems. But this will also cause problems in the current
protocol, so I'm not sure, if it's strictly a regression, it's just
unsupported (which we can only make the best effort to detect).

In summary, for the topology ID discussion, I mainly see two options:
1) stick with the current KIP proposal
2) define the topology ID as the hash of the Strea

Re: [DISCUSS] KIP-1071: Streams Rebalance Protocol

2024-09-02 Thread Bruno Cadonna

Hi all,

For your info, I updated the StreamsGroupInitialize request with the 
following changes:


1. I added the topology ID to the request so that the group coordinator 
knows for which topology it got the initialization.


2. I renamed field "Subtopology" to "SubtopologyId" since the field is 
the ID of the subtopology but that was not clear from the name.


Best,
Bruno


On 8/28/24 2:06 PM, Lucas Brutschy wrote:

Hi Sophie,

Thanks for your detailed comments - much appreciated! I think you read
a version of the KIP that did not yet include the admin command-line
tool and the Admin API extensions, so some of the comments are already
addressed in the KIP.

S1. StreamsGroupHeartbeat and StreamsGroupInitialize are called in the
consumer background thread. Note that in the new consumer threading
model, all RPCs are run by the background thread. Check out this:
https://cwiki.apache.org/confluence/display/KAFKA/Consumer+threading+refactor+design
for more information. Both our RPCs are just part of the group
membership management and do not need to be invoked explicitly by the
application thread. You could imagine calling the initialize RPC
explicitly (to implement explicit initialization), but this would
still mean sending an event to the background thread, and the
background thread in turn invokes the RPC. However, explicit
initialization would require some additional public interfaces that we
are not including in this KIP. StreamsGroupDescribe is called by the
AdminClient, and used by the command-line tool
kafka-streams-groups.sh.

S2. I think the max.warmup.replicas=100 suggested by Nick was intended
as the upper limit for setting the group configuration on the broker.
Just to make sure that this was not a misunderstanding. By default,
values above 100 should be rejected when setting a specific value for
group. Are you suggesting 20 or 30 for the default value for groups,
or the default upper limit for the group configuration?

S3. Yes, it's supposed to be used like SHUTDOWN_APPLICATION. The
MemberEpoch=-1 is a left-over from an earlier discussion. It means
that the member is leaving the group, so the intention was that the
member must leave the group when it asks the other members to shut
down. We later reconsidered this and decided that all applications
should just react to the shutdown application signal that is returned
by the broker, so the client first sets the ShutdownApplication and
later leaves the group. Thanks for spotting this, I removed it.

S4. Not sure if this refers to the latest version of the KIP. We added
an extension of the admin API and a kafka-streams-groups.sh
command-line tool to the KIP already.

S5. All RPCs for dealing with offsets will keep working with streams
groups. The extension of the admin API is rather cosmetic, since the
method names use "consumer group". The RPCs, however, are generic and
do not need to be changed.

S6. Yes, you can use the DeleteGroup RPC with any group ID, whether
streams group or not.

S7. See the admin API section.

S8. I guess for both A and B, I am not sure what you are suggesting.
Do you want to make the broker-side topology immutable and not include
any information about the topology, like the topology ID in the RPC?
It would seem that this would be a massive food-gun for people, if
they start changing their topology and don't notice that the broker is
looking at a completely different version of the topology. Or did you
mean that there is some kind of topology ID, so that at least we can
detect inconsistencies between broker and client-side topologies, and
we fence out any member with an incorrect topology ID? Then we seem to
end up with mostly the same RPCs and the same questions (how is the
topology ID generated?). I agree that the latter could be an option.
See summary at the end of this message.

S9. If we flat out refuse topology updates, agree - we cannot let the
application crash on minor changes to the topology. However, if we
allow topology updates as described in the KIP, there are only upsides
to making the topology ID more sensitive. All it will cause is that a
client will have to resend a `StreamsGroupInitialize`, and during the
rolling bounce, older clients will not get tasks assigned.

S10. The intention in the KIP is really just this - old clients can
only retain tasks, but not get new ones. If the topology has
sub-topologies, these will initially be assigned to the new clients,
which also have the new topology. You are right that rolling an
application where the old structure and the new structure are
incompatible (e.g. different subtopologies access the same partition)
will cause problems. But this will also cause problems in the current
protocol, so I'm not sure, if it's strictly a regression, it's just
unsupported (which we can only make the best effort to detect).

In summary, for the topology ID discussion, I mainly see two options:
1) stick with the current KIP proposal
2) define the topology ID as the hash of the Strea

[DISCUSS] KIP-1042: Support for GLOBs when creating new acls

2024-09-02 Thread Claude Warren, Jr
After some discussion about the earlier KIP-1042 we have rewritten it to
focus on the implementation of a GLOB pattern type.  Please review and
comment.

We have removed all discussion of the Trie implementation and focus on what
is required for the GLOB implementation.  The KIP does assume that the Trie
implementation will be available.

Thank you,
Claude


[jira] [Created] (KAFKA-17465) Make getMembersFromGroup be non-blocking

2024-09-02 Thread Chia-Ping Tsai (Jira)
Chia-Ping Tsai created KAFKA-17465:
--

 Summary: Make getMembersFromGroup be non-blocking
 Key: KAFKA-17465
 URL: https://issues.apache.org/jira/browse/KAFKA-17465
 Project: Kafka
  Issue Type: Improvement
Reporter: Chia-Ping Tsai
Assignee: Chia-Ping Tsai


https://github.com/apache/kafka/blob/359ddce3b2ce9288389751f5a64a965db9bf12be/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java#L4120

The request used to collect all members is blocking, and we fix that blocking 
call.



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


[jira] [Resolved] (KAFKA-8499) Ducker missing java commands in path for ducker user on openjdk docker images

2024-09-02 Thread Chia-Ping Tsai (Jira)


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

Chia-Ping Tsai resolved KAFKA-8499.
---
Fix Version/s: 2.4.0
 Assignee: Lucas Bradstreet
   Resolution: Fixed

> Ducker missing java commands in path for ducker user on openjdk docker images
> -
>
> Key: KAFKA-8499
> URL: https://issues.apache.org/jira/browse/KAFKA-8499
> Project: Kafka
>  Issue Type: Bug
>  Components: system tests
>Affects Versions: 2.2.0, 2.3.0
>Reporter: Lucas Bradstreet
>Assignee: Lucas Bradstreet
>Priority: Major
> Fix For: 2.4.0
>
>
> openjdk:8/openjdk:11 used to include java and other java programs in 
> /usr/bin. It has since been moved to /usr/local/openjdk-VERSION/bin, which 
> will cause problems when the system tests invoke any java related utility as 
> well as java itself if the user is using a later image with the same tag. The 
> openjdk images have been updated with the same tag, so this can happen 
> suddenly without any other code changes if the new version is pulled.
> The primary symptom on running the tests in ducker is that ZooKeeper will 
> fail to start due to java not being found.
> We need to ensure the ducker user that is created in the Dockerfile includes 
> the new location that java is installed to is included in its path.



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


[jira] [Resolved] (KAFKA-9857) Failed to build image ducker-ak-openjdk-8 on arm

2024-09-02 Thread Chia-Ping Tsai (Jira)


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

Chia-Ping Tsai resolved KAFKA-9857.
---
Resolution: Duplicate

duplicate to KAFKA-10759

> Failed to build image ducker-ak-openjdk-8 on arm
> 
>
> Key: KAFKA-9857
> URL: https://issues.apache.org/jira/browse/KAFKA-9857
> Project: Kafka
>  Issue Type: Bug
>  Components: build, core, system tests
>Reporter: jiamei xie
>Assignee: jiamei xie
>Priority: Major
>
> It failed to build image ducker-ak-openjdk-8 on arm and below is its log. 
> This issue is to fix it.
> kafka/tests/docker$ ./run_tests.sh
> Sending build context to Docker daemon  53.76kB
> Step 1/43 : ARG jdk_version=openjdk:8
> Step 2/43 : FROM $jdk_version
> 8: Pulling from library/openjdk
> no matching manifest for linux/arm64/v8 in the manifest list entries
> docker failed
> + die 'ducker-ak up failed'
> + echo ducker-ak up failed
> ducker-ak up failed
> + exit 1



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


Change the base image of e2e from openjdk to eclipse-temurin

2024-09-02 Thread Chia-Ping Tsai
hi Kafka developers,

the last update of openjdk:8 image is 2 years ago. IMHO, it is too stale to
be the base image of e2e. It may lacks of some important backport. For
example, we observe memory usage issue in the containers (KAFKA-17343) and
the root cause could be the stale JDK.

For another, the last update of openjdk:11 is also 2 years ago. That means
we will encounter the stale image again even though we drop jdk 8.

`eclipse-temurin`is a good replacement because it is already used to run
tests on GitHub CI.

I haven't test eclipse-temurin:8 (but I will do it later), but it will be
great to have some feedback from kafka developers.

Best,
Chia-Ping


Re: New release branch 3.9

2024-09-02 Thread Luke Chen
Hi Colin,

I've made KAFKA-17412  (
PR ) as v3.9.0 blocker for
documenting the unclean leader election behavior change in KRaft mode.
This is just a document change, so I think it should be fine.

Let me know if you have any concerns.

Thanks.
Luke

On Sat, Aug 31, 2024 at 2:53 AM Chia-Ping Tsai  wrote:

> hi Colin
>
> I open the https://issues.apache.org/jira/browse/KAFKA-17454 to be a
> blocker for 3.9 since the e2e transactions_mixed_versions_test fail on 3.9
> definitely
>
> We will fix it tomorrow.
>
> Best,
> Chia-Ping
>
> On 2024/08/30 15:38:48 Colin McCabe wrote:
> > No worries. Thanks for checking.
> >
> > best,
> > Colin
> >
> > On Thu, Aug 29, 2024, at 19:00, Kirk True wrote:
> > > Thanks Chia-Ping!
> > >
> > > Sorry for the noise, everyone else :)
> > >
> > >> On Aug 29, 2024, at 6:36 PM, Chia-Ping Tsai 
> wrote:
> > >>
> > >> hi Kirk
> > >>
> > >> I have updated the fix versions of KAFKA-17335 to include 3.9
> > >>
> > >> https://issues.apache.org/jira/browse/KAFKA-17335
> > >>
> > >> Best,
> > >> Chia-Ping
> > >>
> > >> Chia-Ping Tsai  於 2024年8月30日 週五 上午9:32寫道:
> > >>
> > >>> hi Kirk
> > >>>
> > >>> KAFKA-17335 is already in 3.9 (
> > >>>
> https://github.com/apache/kafka/commit/a87b501a47a55a1f9038b6c52e31b71590c3a80a
> > >>> )
> > >>>
> > >>> Best,
> > >>> Chia-Ping
> > >>>
> > >>> Kirk True  於 2024年8月30日 週五 上午4:36寫道:
> > >>>
> >  Hi Colin,
> > 
> >  KAFKA-17335 (Lack of default for URL encoding configuration for
> OAuth
> >  causes NPE) affects the internal use of the OAuth-based
> >  AuthenticateCallbackHandler. Can we back port the fix (
> >  https://github.com/apache/kafka/pull/16990) to the 3.9 branch?
> > 
> >  Thanks,
> >  Kirk
> > 
> > > On Aug 29, 2024, at 11:29 AM, Colin McCabe 
> wrote:
> > >
> > > Hi all,
> > >
> > > Code freeze has officially arrived for Apache Kafka 3.9! If you
> have
> >  additional fixes to get into the branch at this point, please ping
> me first
> >  (unless it's already marked as a blocker in JIRA).
> > >
> > > I did a first pass on the list of open JIRAs for 3.9, and moved
> out any
> >  that were not blockers for 3.9. In a few cases, I asked a question
> about
> >  whether the JIRA was a blocker or not.
> > >
> > > If all goes according to plan, we should now have two more weeks of
> >  stabilization before the release.
> > >
> > > Thanks, everyone.
> > >
> > > Colin
> > >
> > >
> > > On Fri, Aug 23, 2024, at 13:30, Colin McCabe wrote:
> > >> I wanted to give a quick update on 3.9. Code freeze is coming up
> next
> > >> Thursday on the 29th. As usual, after that point, only critical
> bug
> > >> fixes will be accepted.
> > >>
> > >> Thanks again to everyone who has helped with this release.
> > >>
> > >> best,
> > >> Colin
> > >>
> > >>
> > >> On Fri, Aug 23, 2024, at 13:18, Colin McCabe wrote:
> > >>> Thanks for the heads up, Josep. I saw both emails but I didn't
> make
> >  the
> > >>> connection until now. It does seem likely that we'll have to
> modify
> > >>> this (either for 3.9... or soon.)
> > >>>
> > >>> best,
> > >>> Colin
> > >>>
> > >>> On Tue, Aug 13, 2024, at 05:09, Josep Prat wrote:
> >  Hi Colin,
> > 
> >  Just to make sure you (as a Release Manager) are aware of the
> >  change. ASF
> >  Infra just communicated the removal of the home.apache.org
> machines
> >  (in 4
> >  weeks they will be gone). This means the release.py script
> wouldn't
> >  work if
> >  executed after that date and would need to be modified to store
> RC
> >  artifacts under dist.apache.org/dev instead.
> > 
> >  Best,
> > 
> >  On Fri, Aug 9, 2024 at 11:38 PM Colin McCabe <
> cmcc...@apache.org>
> >  wrote:
> > 
> > > Updates:
> > >
> > > KIP-1007: I have removed this PR from the Apache Kafka 3.9
> release
> >  plan.
> > > Since the KIP was discarded, I did not add it in the
> "postponed to
> > > subsequent release" section.
> > >
> > > KIP-1025: Marked 3.9 status as Done
> > >
> > > KIP-1023: Postponed to subsequent release
> > >
> > > KIP-1005: Marked 3.9 status as Done
> > >
> > > KIP-996: Postponed to subsequent release
> > >
> > > KIP-994: Postponed to subsequent release
> > >
> > > KIP-997: Postponed to subsequent release
> > >
> > > KIP-966: Postponed to subsequent release
> > >
> > > KIP-956: Postponed to subsequent release
> > >
> > > KIP-950: Marked 3.9 status as Done
> > >
> > > best,
> > > Colin
> > >
> > > On Fri, Aug 9, 202

[jira] [Resolved] (KAFKA-17450) Optimise the handler methods in ShareConsumeRequestManager.

2024-09-02 Thread Chia-Ping Tsai (Jira)


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

Chia-Ping Tsai resolved KAFKA-17450.

Fix Version/s: 4.0.0
   Resolution: Fixed

> Optimise the handler methods in ShareConsumeRequestManager.
> ---
>
> Key: KAFKA-17450
> URL: https://issues.apache.org/jira/browse/KAFKA-17450
> Project: Kafka
>  Issue Type: Sub-task
>Reporter: Shivsundar R
>Assignee: Shivsundar R
>Priority: Major
> Fix For: 4.0.0
>
>
> Currently there are 4 handler functions for ShareAcknowledge responses. 
> Instead using AcknowledgeRequestType, we could merge the code and have only 2 
> handler functions, one for ShareAcknowledge success and one for 
> ShareAcknowledge failure.



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


[jira] [Created] (KAFKA-17466) Revisit failAndRemoveExpiredCommitRequests method

2024-09-02 Thread TengYao Chi (Jira)
TengYao Chi created KAFKA-17466:
---

 Summary: Revisit failAndRemoveExpiredCommitRequests method
 Key: KAFKA-17466
 URL: https://issues.apache.org/jira/browse/KAFKA-17466
 Project: Kafka
  Issue Type: Improvement
Reporter: TengYao Chi
Assignee: TengYao Chi


see discussion: 
[https://github.com/apache/kafka/pull/16833#issuecomment-2301334628]

 

In short, we should consider removing the `failAndRemoveExpiredCommitRequests` 
method since its functionality is already handled elsewhere.



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