Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

KAFKA-15305: The background thread should try to process the remaining task until the shutdown timer is expired. #16156

Merged
merged 7 commits into from
Jun 4, 2024

Conversation

frankvicky
Copy link
Contributor

Currently, ConsumerNetworkThread does not consider if there are any in-flight requests when closing, which is not desirable. It should keep processing the remaining requests (whether unsent or in-flight) until either all requests are finished or a timeout occurs.

Committer Checklist (excluded from commit message)

  • Verify design and implementation
  • Verify test coverage and CI build status
  • Verify documentation (including upgrade notes)

…f there are any requests that are either in-flight or unsent.
@@ -298,7 +298,7 @@ private void sendUnsentRequests(final Timer timer) {
do {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do we need the early return?
If there is a request in in-flight but unSentRequestQueue is empty, we will lose the newest request.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Cool, I think we don't need this early return statement in the current situation because we also need to consider if there are any in-flight requests.
The reason why we have this statement here is because we didn't consider the in-flight requests before, so we could return if there are no unsent requests. 🤔

// in-flight
assertTrue(networkClientDelegate.hasAnyPendingRequests());
assertTrue(networkClientDelegate.unsentRequests().isEmpty());
assertTrue(client.hasInFlightRequests());
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

to cover the whole flow, what about extending the test to ensure that when we get a response, the hasAnyPendingRequests goes false? (there's a prepareFindCoordinatorResponse that should be handy for it)

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This makes sense, I will take a look 😺

do {
networkClientDelegate.poll(timer.remainingMs(), timer.currentTimeMs());
timer.update();
} while (timer.notExpired() && !networkClientDelegate.unsentRequests().isEmpty());
} while (timer.notExpired() && networkClientDelegate.hasAnyPendingRequests());
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We should use while-loop instead of do-while, since it is unnecessary to poll for nothing.

assertTrue(networkClientDelegate.hasAnyPendingRequests());
assertTrue(networkClientDelegate.unsentRequests().isEmpty());
assertTrue(client.hasInFlightRequests());
time.sleep(REQUEST_TIMEOUT_MS);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could we mock response instead of making timeout? For example:

            // in-flight
            assertTrue(networkClientDelegate.hasAnyPendingRequests());
            assertTrue(networkClientDelegate.unsentRequests().isEmpty());
            assertTrue(client.hasInFlightRequests());
            
            // complete request
            client.respond(FindCoordinatorResponse.prepareResponse(Errors.NONE, GROUP_ID, mockNode()));
            networkClientDelegate.poll(0, time.milliseconds());
            assertFalse(networkClientDelegate.hasAnyPendingRequests());
            assertTrue(networkClientDelegate.unsentRequests().isEmpty());
            assertFalse(client.hasInFlightRequests());

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Cool, this works like a charm and is more reasonable. 🥳

@frankvicky frankvicky marked this pull request as ready for review June 2, 2024 16:16
Copy link
Contributor

@chia7712 chia7712 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM

if (networkClientDelegate.unsentRequests().isEmpty())
return;
do {
while (timer.notExpired() && networkClientDelegate.hasAnyPendingRequests()) {
Copy link
Contributor

@lianetm lianetm Jun 3, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

sorry a bit late but one concern: wouldn't this change make that requests that are issued with timeout 0 won't be sent out??? My understanding is that the expected behaviour for requests with timeout 0 is fire-and-forget, so we want to send them at least once.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@lianetm you are right. Maybe we can keep origin code but we replace all networkClientDelegate.unsentRequests().isEmpty() by !networkClientDelegate.hasAnyPendingRequests()?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

exactly, sounds good to me: keeping the early return if to avoid poll, do-while to ensure we send out at least once, both conditions using hasAnyPendingRequests

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hi @lianetm, thank you for pointing out this potential bug, I will fix it. 🙇🏼

@frankvicky
Copy link
Contributor Author

Hi @lianetm and @chia7712,
I made a little change based on your comments. Please take a look 😸

Copy link
Contributor

@lianetm lianetm left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM. Thanks for the nice improvement @frankvicky!

@lianetm
Copy link
Contributor

lianetm commented Jun 4, 2024

Just for the record, there's another PR in-flight (mentioned above) improving testing on this same area too.

@chia7712
Copy link
Contributor

chia7712 commented Jun 4, 2024

Just for the record, there's another PR in-flight (mentioned above) improving testing on this same area too.

nice to know that. I will merge this PR after QA has completed

@chia7712
Copy link
Contributor

chia7712 commented Jun 4, 2024

the failed tests are already fixed by a08db65

@chia7712 chia7712 merged commit 8b3c77c into apache:trunk Jun 4, 2024
1 check failed
chia7712 pushed a commit that referenced this pull request Jun 4, 2024
… task until the shutdown timer is expired. (#16156)

Reviewers: Lianet Magrans <lianetmr@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
apourchet added a commit to apourchet/kafka that referenced this pull request Jun 6, 2024
commit ee834d9
Author: Antoine Pourchet <antoine@responsive.dev>
Date:   Thu Jun 6 15:20:48 2024 -0600

    KAFKA-15045: (KIP-924 pt. 19) Update to new AssignmentConfigs (apache#16219)

    This PR updates all of the streams task assignment code to use the new AssignmentConfigs public class.

    Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>

commit 8a2bc3a
Author: Bruno Cadonna <cadonna@apache.org>
Date:   Thu Jun 6 21:19:52 2024 +0200

    KAFKA-16903: Consider produce error of different task (apache#16222)

    A task does not know anything about a produce error thrown
    by a different task. That might lead to a InvalidTxnStateException
    when a task attempts to do a transactional operation on a producer
    that failed due to a different task.

    This commit stores the produce exception in the streams producer
    on completion of a send instead of the record collector since the
    record collector is on task level whereas the stream producer
    is on stream thread level. Since all tasks use the same streams
    producer the error should be correctly propagated across tasks
    of the same stream thread.

    For EOS alpha, this commit does not change anything because
    each task uses its own producer. The send error is still
    on task level but so is also the transaction.

    Reviewers: Matthias J. Sax <matthias@confluent.io>

commit 7d832cf
Author: David Jacot <djacot@confluent.io>
Date:   Thu Jun 6 21:19:20 2024 +0200

    KAFKA-14701; Move `PartitionAssignor` to new `group-coordinator-api` module (apache#16198)

    This patch moves the `PartitionAssignor` interface and all the related classes to a newly created `group-coordinator/api` module, following the pattern used by the storage and tools modules.

    Reviewers: Ritika Reddy <rreddy@confluent.io>, Jeff Kim <jeff.kim@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>

commit 79ea7d6
Author: Mickael Maison <mimaison@users.noreply.github.com>
Date:   Thu Jun 6 20:28:39 2024 +0200

    MINOR: Various cleanups in clients (apache#16193)

    Reviewers: Chia-Ping Tsai <chia7712@gmail.com>

commit a41f7a4
Author: Murali Basani <muralidhar.basani@aiven.io>
Date:   Thu Jun 6 18:06:25 2024 +0200

    KAFKA-16884 Refactor RemoteLogManagerConfig with AbstractConfig (apache#16199)

    Reviewers: Greg Harris <gharris1727@gmail.com>, Kamal Chandraprakash <kamal.chandraprakash@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>

commit 0ed104c
Author: Kamal Chandraprakash <kchandraprakash@uber.com>
Date:   Thu Jun 6 21:26:08 2024 +0530

    MINOR: Cleanup the storage module unit tests (apache#16202)

    - Use SystemTime instead of MockTime when time is not mocked
    - Use static assertions to reduce the line length
    - Fold the lines if it exceeds the limit
    - rename tp0 to tpId0 when it refers to TopicIdPartition

    Reviewers: Kuan-Po (Cooper) Tseng <brandboat@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>

commit f36a873
Author: Cy <yimck@uci.edu>
Date:   Thu Jun 6 08:46:49 2024 -0700

    MINOR: Added test for ClusterConfig#displayTags (apache#16110)

    Reviewers: Chia-Ping Tsai <chia7712@gmail.com>

commit 226f3c5
Author: Sanskar Jhajharia <122860866+sjhajharia@users.noreply.github.com>
Date:   Thu Jun 6 18:48:23 2024 +0530

    MINOR: Code cleanup in metadata module (apache#16065)

    Reviewers: Mickael Maison <mickael.maison@gmail.com>

commit ebe1e96
Author: Loïc GREFFIER <loic.greffier@michelin.com>
Date:   Thu Jun 6 13:40:31 2024 +0200

    KAFKA-16448: Add ProcessingExceptionHandler interface and implementations (apache#16187)

    This PR is part of KAFKA-16448 which aims to bring a ProcessingExceptionHandler to Kafka Streams in order to deal with exceptions that occur during processing.

    This PR brings ProcessingExceptionHandler interface and default implementations.

    Co-authored-by: Dabz <d.gasparina@gmail.com>
    Co-authored-by: sebastienviale <sebastien.viale@michelin.com>

    Reviewer: Bruno Cadonna <cadonna@apache.org>

commit b74b182
Author: Lianet Magrans <98415067+lianetm@users.noreply.github.com>
Date:   Thu Jun 6 09:45:36 2024 +0200

    KAFKA-16786: Remove old assignment strategy usage in new consumer (apache#16214)

    Remove usage of the partition.assignment.strategy config in the new consumer. This config is deprecated with the new consumer protocol, so the AsyncKafkaConsumer should not use or validate the property.

    Reviewers: Lucas Brutschy <lbrutschy@confluent.io>

commit f880ad6
Author: Alyssa Huang <ahuang@confluent.io>
Date:   Wed Jun 5 23:30:49 2024 -0700

    KAFKA-16530: Fix high-watermark calculation to not assume the leader is in the voter set (apache#16079)

    1. Changing log message from error to info - We may expect the HW calculation to give us a smaller result than the current HW in the case of quorum reconfiguration. We will continue to not allow the HW to actually decrease.
    2. Logic for finding the updated LeaderEndOffset for updateReplicaState is changed as well. We do not assume the leader is in the voter set and check the observer states as well.
    3. updateLocalState now accepts an additional "lastVoterSet" param which allows us to update the leader state with the last known voters. any nodes in this set but not in voterStates will be added to voterStates and removed from observerStates, any nodes not in this set but in voterStates will be removed from voterStates and added to observerStates

    Reviewers: Luke Chen <showuon@gmail.com>, José Armando García Sancio <jsancio@apache.org>

commit 3835515
Author: Okada Haruki <ocadaruma@gmail.com>
Date:   Thu Jun 6 15:10:13 2024 +0900

    KAFKA-16541 Fix potential leader-epoch checkpoint file corruption (apache#15993)

    A patch for KAFKA-15046 got rid of fsync on LeaderEpochFileCache#truncateFromStart/End for performance reason, but it turned out this could cause corrupted leader-epoch checkpoint file on ungraceful OS shutdown, i.e. OS shuts down in the middle when kernel is writing dirty pages back to the device.

    To address this problem, this PR makes below changes: (1) Revert LeaderEpochCheckpoint#write to always fsync
    (2) truncateFromStart/End now call LeaderEpochCheckpoint#write asynchronously on scheduler thread
    (3) UnifiedLog#maybeCreateLeaderEpochCache now loads epoch entries from checkpoint file only when current cache is absent

    Reviewers: Jun Rao <junrao@gmail.com>

commit 7763243
Author: Florin Akermann <florin.akermann@gmail.com>
Date:   Thu Jun 6 00:22:31 2024 +0200

    KAFKA-12317: Update FK-left-join documentation (apache#15689)

    FK left-join was changed via KIP-962. This PR updates the docs accordingly.

    Reviewers: Ayoub Omari <ayoubomari1@outlook.fr>, Matthias J. Sax <matthias@confluent.io>

commit 1134520
Author: Ayoub Omari <ayoubomari1@outlook.fr>
Date:   Thu Jun 6 00:05:04 2024 +0200

    KAFKA-16573: Specify node and store where serdes are needed (apache#15790)

    Reviewers: Matthias J. Sax <matthias@confluent.io>, Bruno Cadonna <bruno@confluent.io>, Anna Sophie Blee-Goldman <ableegoldman@apache.org>

commit 896af1b
Author: Sanskar Jhajharia <122860866+sjhajharia@users.noreply.github.com>
Date:   Thu Jun 6 01:46:59 2024 +0530

    MINOR: Raft module Cleanup (apache#16205)

    Reviewers: Chia-Ping Tsai <chia7712@gmail.com>

commit 0109a3f
Author: Antoine Pourchet <antoine@responsive.dev>
Date:   Wed Jun 5 14:09:37 2024 -0600

    KAFKA-15045: (KIP-924 pt. 17) State store computation fixed (apache#16194)

    Fixed the calculation of the store name list based on the subtopology being accessed.

    Also added a new test to make sure this new functionality works as intended.

    Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>

commit 52514a8
Author: Greg Harris <greg.harris@aiven.io>
Date:   Wed Jun 5 11:35:32 2024 -0700

    KAFKA-16858: Throw DataException from validateValue on array and map schemas without inner schemas (apache#16161)

    Signed-off-by: Greg Harris <greg.harris@aiven.io>
    Reviewers: Chris Egerton <chrise@aiven.io>

commit f2aafcc
Author: Sanskar Jhajharia <122860866+sjhajharia@users.noreply.github.com>
Date:   Wed Jun 5 20:06:01 2024 +0530

    MINOR: Cleanups in Shell Module (apache#16204)

    Reviewers: Chia-Ping Tsai <chia7712@gmail.com>

commit bd9d68f
Author: Abhijeet Kumar <abhijeet.cse.kgp@gmail.com>
Date:   Wed Jun 5 19:12:25 2024 +0530

    KAFKA-15265: Integrate RLMQuotaManager for throttling fetches from remote storage (apache#16071)

    Reviewers: Kamal Chandraprakash<kamal.chandraprakash@gmail.com>, Luke Chen <showuon@gmail.com>, Satish Duggana <satishd@apache.org>

commit 62e5cce
Author: gongxuanzhang <gongxuanzhangmelt@gmail.com>
Date:   Wed Jun 5 18:57:32 2024 +0800

    KAFKA-10787 Update spotless version and remove support JDK8 (apache#16176)

    Reviewers: Chia-Ping Tsai <chia7712@gmail.com>

commit 02c794d
Author: Kamal Chandraprakash <kchandraprakash@uber.com>
Date:   Wed Jun 5 12:12:23 2024 +0530

    KAFKA-15776: Introduce remote.fetch.max.timeout.ms to configure DelayedRemoteFetch timeout (apache#14778)

    KIP-1018, part1, Introduce remote.fetch.max.timeout.ms to configure DelayedRemoteFetch timeout

    Reviewers: Luke Chen <showuon@gmail.com>

commit 7ddfa64
Author: Dongnuo Lyu <139248811+dongnuo123@users.noreply.github.com>
Date:   Wed Jun 5 02:08:38 2024 -0400

    MINOR: Adjust validateOffsetCommit/Fetch in ConsumerGroup to ensure compatibility with classic protocol members (apache#16145)

    During online migration, there could be ConsumerGroup that has members that uses the classic protocol. In the current implementation, `STALE_MEMBER_EPOCH` could be thrown in ConsumerGroup offset fetch/commit validation but it's not supported by the classic protocol. Thus this patch changed `ConsumerGroup#validateOffsetCommit` and `ConsumerGroup#validateOffsetFetch` to ensure compatibility.

    Reviewers: Jeff Kim <jeff.kim@confluent.io>, David Jacot <djacot@confluent.io>

commit 252c1ac
Author: Apoorv Mittal <apoorvmittal10@gmail.com>
Date:   Wed Jun 5 05:55:24 2024 +0100

    KAFKA-16740: Adding skeleton code for Share Fetch and Acknowledge RPC (KIP-932) (apache#16184)

    The PR adds skeleton code for Share Fetch and Acknowledge RPCs. The changes include:

    1. Defining RPCs in KafkaApis.scala
    2. Added new SharePartitionManager class which handles the RPCs handling
    3. Added SharePartition class which manages in-memory record states and for fetched data.

    Reviewers: David Jacot <djacot@confluent.io>, Andrew Schofield <aschofield@confluent.io>, Manikumar Reddy <manikumar.reddy@gmail.com>

commit b89999b
Author: PoAn Yang <payang@apache.org>
Date:   Wed Jun 5 08:02:52 2024 +0800

    KAFKA-16483: Remove preAppendErrors from createPutCacheCallback (apache#16105)

    The method createPutCacheCallback has a input argument preAppendErrors. It is used to keep the "error" happens before appending. However, it is always empty. Also, the pre-append error is handled before createPutCacheCallback by calling responseCallback. Hence, we can remove preAppendErrors.

    Signed-off-by: PoAn Yang <payang@apache.org>

    Reviewers: Luke Chen <showuon@gmail.com>

commit 01e9918
Author: Kuan-Po (Cooper) Tseng <brandboat@gmail.com>
Date:   Wed Jun 5 07:56:18 2024 +0800

    KAFKA-16814 KRaft broker cannot startup when `partition.metadata` is missing (apache#16165)

    When starting up kafka logManager, we'll check stray replicas to avoid some corner cases. But this check might cause broker unable to startup if partition.metadata is missing because when startup kafka, we load log from file, and the topicId of the log is coming from partition.metadata file. So, if partition.metadata is missing, the topicId will be None, and the LogManager#isStrayKraftReplica will fail with no topicID error.

    The partition.metadata missing could be some storage failure, or another possible path is unclean shutdown after topic is created in the replica, but before data is flushed into partition.metadata file. This is possible because we do the flush in async way here.

    When finding a log without topicID, we should treat it as a stray log and then delete it.

    Reviewers: Luke Chen <showuon@gmail.com>, Gaurav Narula <gaurav_narula2@apple.com>

commit d652f5c
Author: TingIāu "Ting" Kì <51072200+frankvicky@users.noreply.github.com>
Date:   Wed Jun 5 07:52:06 2024 +0800

    MINOR: Add topicIds and directoryIds to the return value of the toString method. (apache#16189)

    Add topicIds and directoryIds to the return value of the toString method.

    Reviewers: Luke Chen <showuon@gmail.com>

commit 7e0caad
Author: Igor Soarez <i@soarez.me>
Date:   Tue Jun 4 22:12:33 2024 +0100

    MINOR: Cleanup unused references in core (apache#16192)

    Reviewers: Chia-Ping Tsai <chia7712@gmail.com>

commit 9821aca
Author: PoAn Yang <payang@apache.org>
Date:   Wed Jun 5 05:09:04 2024 +0800

    MINOR: Upgrade gradle from 8.7 to 8.8 (apache#16190)

    Reviewers: Chia-Ping Tsai <chia7712@gmail.com>

commit 9ceed8f
Author: Colin P. McCabe <cmccabe@apache.org>
Date:   Tue Jun 4 14:04:59 2024 -0700

    KAFKA-16535: Implement AddVoter, RemoveVoter, UpdateVoter RPCs

    Implement the add voter, remove voter, and update voter RPCs for
    KIP-853. This is just adding the RPC handling; the current
    implementation in RaftManager just throws UnsupportedVersionException.

    Reviewers: Andrew Schofield <aschofield@confluent.io>, José Armando García Sancio <jsancio@apache.org>

commit 8b3c77c
Author: TingIāu "Ting" Kì <51072200+frankvicky@users.noreply.github.com>
Date:   Wed Jun 5 04:21:20 2024 +0800

    KAFKA-15305 The background thread should try to process the remaining task until the shutdown timer is expired. (apache#16156)

    Reviewers: Lianet Magrans <lianetmr@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>

commit cda2df5
Author: Kamal Chandraprakash <kchandraprakash@uber.com>
Date:   Wed Jun 5 00:41:30 2024 +0530

    KAFKA-16882 Migrate RemoteLogSegmentLifecycleTest to ClusterInstance infra (apache#16180)

    - Removed the RemoteLogSegmentLifecycleManager
    - Removed the TopicBasedRemoteLogMetadataManagerWrapper, RemoteLogMetadataCacheWrapper, TopicBasedRemoteLogMetadataManagerHarness and TopicBasedRemoteLogMetadataManagerWrapperWithHarness

    Reviewers: Kuan-Po (Cooper) Tseng <brandboat@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>

commit 2b47798
Author: Chris Egerton <chrise@aiven.io>
Date:   Tue Jun 4 21:04:34 2024 +0200

    MINOR: Fix return tag on Javadocs for consumer group-related Admin methods (apache#16197)

    Reviewers: Greg Harris <greg.harris@aiven.io>, Chia-Ping Tsai <chia7712@gmail.com>
TaiJuWu pushed a commit to TaiJuWu/kafka that referenced this pull request Jun 8, 2024
… task until the shutdown timer is expired. (apache#16156)

Reviewers: Lianet Magrans <lianetmr@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
gongxuanzhang pushed a commit to gongxuanzhang/kafka that referenced this pull request Jun 12, 2024
… task until the shutdown timer is expired. (apache#16156)

Reviewers: Lianet Magrans <lianetmr@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
4 participants