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-15541: Add oldest-iterator-open-since-ms metric #16041

Merged
merged 2 commits into from
May 28, 2024

Conversation

nicktelford
Copy link
Contributor

@nicktelford nicktelford commented May 23, 2024

Part of KIP-989

This new StateStore metric tracks the timestamp that the oldest surviving Iterator was created.

This timestamp should continue to climb, and closely track the current time, as old iterators are closed and new ones created. If the timestamp remains very low (i.e. old), that suggests an Iterator has leaked, which should enable users to isolate the affected store.

It will report no data when there are no currently open Iterators.

@nicktelford
Copy link
Contributor Author

@mjsax This is the final PR to add the KIP-989 metrics. Although I'm considering opening two, much smaller PRs:

  1. To add some additional tests to ensure we cover all the Iterator constructors
  2. To eliminate some redundancy now all the metrics have landed. Notably, we can replace the AtomicInteger used by num-open-iterators with just counting the cardinality of the Set<MeteredIterator> that we maintain for oldest-iterator-open-since-ms.

@mjsax mjsax added streams kip Requires or implements a KIP labels May 23, 2024
Copy link
Member

@mjsax mjsax left a comment

Choose a reason for hiding this comment

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

A few questions/suggestions.

Doing follow-up PRs sounds great! We can merge these ofter feature freeze too (we can always add more tests :))

@@ -169,6 +172,10 @@ private void registerMetrics() {
iteratorDurationSensor = StateStoreMetrics.iteratorDurationSensor(taskId.toString(), metricsScope, name(), streamsMetrics);
StateStoreMetrics.addNumOpenIteratorsGauge(taskId.toString(), metricsScope, name(), streamsMetrics,
(config, now) -> numOpenIterators.get());
StateStoreMetrics.addOldestOpenIteratorGauge(taskId.toString(), metricsScope, name(), streamsMetrics,
(config, now) -> openIterators.isEmpty() ? null :
openIterators.stream().mapToLong(MeteredIterator::startTimestamp).min().getAsLong()
Copy link
Member

Choose a reason for hiding this comment

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

I don't want to over-engineer (given that we can safely assume that the openIterator set should be small), but wondering if this is the best implementation?

In the end, we only want to track the create ts, not the iterators themselves. And for create ts we could just maintain a list if longs, and we would return list.first() here, and always append to the end of the list when a new iterator is created? Only "remove" would be more expensive, but we could use a sorted tree for the list, and thus remove would be O(log n) not O(n)).

For this case, we also don't need the MeteredIterator helper interface.

Thoughts?

Copy link
Contributor Author

@nicktelford nicktelford May 24, 2024

Choose a reason for hiding this comment

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

Yeah, I considered something like this, but I was concerned that remove could be a problem, in particular for two Iterators with the same timestamp.

An alternative would be to use a ConcurrentSkipListMap<KeyValueIterator<?, ?>, Long>, which would be O(log(n)) for both insert and remove, and eliminate any concerns about duplicate timestamps; whilst still being sorted by timestamp (given the appropriate comparator on construction).

This also solves the problem that this structure should probably be thread-safe, because Iterators may be opened/closed from various Interactive Query threads.

Unfortunately, in Java 8 (the minimum currently supported version), ConcurrentSkipListMap#size() is O(n), and considered "inaccurate". This wasn't addressed until JDK 10, so we will need to keep the AtomicInteger for tracking the num-open-iterators, for the time-being.

Actually, this wouldn't work because the Comparator used by ConcurrentSkipListMap operates on the key, not the value.

🤔

Using ConcurrentSkipListSet instead of HashSet would give us O(1) insert/remove and let us use openIterators.first().startTimestamp() to generate the metric, which is also O(1). It also gives us thread-safety.

Unfortunately, it doesn't let us eliminate the MeteredIterator.

Any better ideas?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I've pushed a commit with the switch to ConcurrentSkipListSet, but I can change it again if we come up with something better.

Copy link
Member

Choose a reason for hiding this comment

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

in particular for two Iterators with the same timestamp.

Yes. We something that allows for duplicates... Not sure if there is any Java standard library impl... Some people implement a custom multi-set on top of tree-map: https://stackoverflow.com/questions/12565587/does-java-have-a-multiset-data-structure-like-the-one-in-c-stl (not sure if it better?) -- seems to be similar to what you propose with ConcurrentSkipListSet?


// now that the first iterator is closed, check that the timestamp has advanced to the still open second iterator
assertThat((Long) oldestIteratorTimestampMetric.metricValue(), equalTo(secondTimestamp));
second.close();
Copy link
Member

Choose a reason for hiding this comment

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

This close() call should go into a finally block IMHO:

KeyValueIterator<String, String> second = null;
final long secondTimestamp;
try {
    try (final KeyValueIterator<String, String> first = metered.all()) {
        ...
    }
    
    // now that the first iterator is closed, check that the timestamp has advanced to the still open second iterator
    assertThat((Long) oldestIteratorTimestampMetric.metricValue(), equalTo(secondTimestamp));
} finally {
    if (second != null) {
        second.close();
    }
}

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Done. I assumed you were suggesting this in all the tests, so I updated them all.

Copy link
Member

@mjsax mjsax left a comment

Choose a reason for hiding this comment

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

I think it would be good to get rid of MeteredIterator, but also don't see a good solution w/o over-engineering...

The current ConcurrentSkipListSet is for sure an improvement over the original version. Happy to merge this. Just a nit on the test code.

@nicktelford nicktelford force-pushed the KIP-989-c branch 2 times, most recently from 39701fb to f05578d Compare May 28, 2024 10:45
@nicktelford
Copy link
Contributor Author

@mjsax Not sure why the build is failing. :streams:compileJava works fine locally. Perhaps the build got confused, can we repeat it?

@mjsax
Copy link
Member

mjsax commented May 28, 2024

[2024-05-28T11:28:17.911Z] > Task :streams:compileJava
[2024-05-28T11:28:17.911Z] /home/jenkins/jenkins-agent/workspace/Kafka_kafka-pr_PR-16041/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredMultiVersionedKeyQueryIterator.java:54: warning: [this-escape] possible 'this' escape before subclass is fully initialized
[2024-05-28T11:28:17.911Z]         openIterators.add(this);
[2024-05-28T11:28:17.911Z]                           ^
[2024-05-28T11:28:17.911Z] error: warnings found and -Werror specified

I don't expect it to go away... Maybe Java 21 has some stricter/smarter checks?

Part of [KIP-989](https://cwiki.apache.org/confluence/x/9KCzDw).

This new `StateStore` metric tracks the timestamp that the oldest
surviving Iterator was created.

This timestamp should continue to climb, and closely track the current
time, as old iterators are closed and new ones created. If the timestamp
remains very low (i.e. old), that suggests an Iterator has leaked, which
should enable users to isolate the affected store.

It will report no data when there are no currently open Iterators.
Being public causes a compile warning when we add instances to the
`openIterators` `Set`.

It's not necessary for this class to be public, and infact is probably a
bug, because users should not be constructing instances of this class
themselves.
@mjsax mjsax merged commit 59ba555 into apache:trunk May 28, 2024
1 check failed
@mjsax
Copy link
Member

mjsax commented May 28, 2024

Thanks for the KIP @nicktelford -- this will be super useful in the future!

apourchet added a commit to apourchet/kafka that referenced this pull request May 29, 2024
commit cc269b0
Author: Antoine Pourchet <[email protected]>
Date:   Wed May 29 14:15:02 2024 -0600

    KAFKA-15045: (KIP-924 pt. 14) Callback to TaskAssignor::onAssignmentComputed (apache#16123)

    This PR adds the logic and wiring necessary to make the callback to
    TaskAssignor::onAssignmentComputed with the necessary parameters.

    We also fixed some log statements in the actual assignment error
    computation, as well as modified the ApplicationState::allTasks method
    to return a Map instead of a Set of TaskInfos.

    Reviewers: Anna Sophie Blee-Goldman <[email protected]>

commit 862ea12
Author: Eugene Mitskevich <[email protected]>
Date:   Wed May 29 16:14:37 2024 -0400

    MINOR: Fix rate metric spikes (apache#15889)

    Rate reports value in the form of sumOrCount/monitoredWindowSize. It has a bug in monitoredWindowSize calculation, which leads to spikes in result values.

    Reviewers: Jun Rao <[email protected]>

commit 0f0c9ec
Author: gongxuanzhang <[email protected]>
Date:   Thu May 30 01:08:17 2024 +0800

    KAFKA-16771 First log directory printed twice when formatting storage (apache#16010)

    Reviewers: Chia-Ping Tsai <[email protected]>

commit 2d9994e
Author: Andrew Schofield <[email protected]>
Date:   Wed May 29 16:31:52 2024 +0100

    KAFKA-16722: Introduce ConsumerGroupPartitionAssignor interface (apache#15998)

    KIP-932 introduces share groups to go alongside consumer groups. Both kinds of group use server-side assignors but it is unlikely that a single assignor class would be suitable for both. As a result, the KIP introduces specific interfaces for consumer group and share group partition assignors.

    This PR introduces only the consumer group interface, `o.a.k.coordinator.group.assignor.ConsumerGroupPartitionAssignor`. The share group interface will come in a later release. The existing implementations of the general `PartitionAssignor` interface have been changed to implement `ConsumerGroupPartitionAssignor` instead and all other code changes are just propagating the change throughout the codebase.

    Note that the code in the group coordinator that actually calculates assignments uses the general `PartitionAssignor` interface so that it can be used with both kinds of group, even though the assignors themselves are specific.

    Reviewers: Apoorv Mittal <[email protected]>, David Jacot <[email protected]>

commit 0b75cf7
Author: gongxuanzhang <[email protected]>
Date:   Wed May 29 22:38:00 2024 +0800

    KAFKA-16705 the flag "started" of RaftClusterInstance is false even though the cluster is started (apache#15946)

    Reviewers: Chia-Ping Tsai <[email protected]>

commit 8d11d95
Author: Loïc GREFFIER <[email protected]>
Date:   Wed May 29 14:09:22 2024 +0200

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

    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 <[email protected]>
    Co-authored-by: sebastienviale <[email protected]>

    Reviewer: Bruno Cadonna <[email protected]>

commit b73f479
Author: Ramin Gharib <[email protected]>
Date:   Wed May 29 13:12:54 2024 +0200

    KAFKA-16362: Fix type-unsafety in KStreamKStreamJoin caused by isLeftSide (apache#15601)

    The introduced changes provide a cleaner definition of the join side in KStreamKStreamJoin. Before, this was done by using a Boolean flag, which led to returning a raw LeftOrRightValue without generic arguments because the generic type arguments depended on the boolean input.

    Reviewers: Greg Harris <[email protected]>, Bruno Cadonna <[email protected]>

commit 897cab2
Author: Luke Chen <[email protected]>
Date:   Wed May 29 15:30:18 2024 +0800

    KAFKA-16399: Add JBOD support in tiered storage (apache#15690)

    After JBOD is supported in KRaft, we should also enable JBOD support in tiered storage. Unit tests and Integration tests are also added.

    Reviewers: Satish Duggana <[email protected]>, Kamal Chandraprakash <[email protected]>, Igor Soarez <[email protected]>, Mickael Maison <[email protected]>

commit eefd114
Author: Dongnuo Lyu <[email protected]>
Date:   Wed May 29 02:21:30 2024 -0400

    KAFKA-16832; LeaveGroup API for upgrading ConsumerGroup (apache#16057)

    This patch implements the LeaveGroup API to the consumer groups that are in the mixed mode.

    Reviewers: Jeff Kim <[email protected]>, David Jacot <[email protected]>

commit 9562143
Author: A. Sophie Blee-Goldman <[email protected]>
Date:   Tue May 28 21:35:02 2024 -0700

    HOTFIX: remove unnecessary list creation (apache#16117)

    Removing a redundant list declaration in the new StickyTaskAssignor implementation

    Reviewers: Antoine Pourchet <[email protected]>

commit d64e3fb
Author: Antoine Pourchet <[email protected]>
Date:   Tue May 28 20:43:30 2024 -0600

    KAFKA-15045: (KIP-924 pt. 13) AssignmentError calculation added (apache#16114)

    This PR adds the post-processing of the TaskAssignment to figure out if the new assignment is valid, and return an AssignmentError otherwise.

    Reviewers: Anna Sophie Blee-Goldman <[email protected]>

commit 8d243df
Author: Antoine Pourchet <[email protected]>
Date:   Tue May 28 19:01:18 2024 -0600

    KAFKA-15045: (KIP-924 pt. 12) Wiring in new assignment configs and logic (apache#16074)

    This PR creates the new public config of KIP-924 in StreamsConfig and uses it to instantiate user-created TaskAssignors. If such a TaskAssignor is found and successfully created we then use that assignor to perform the task assignment, otherwise we revert back to the pre KIP-924 world with the internal task assignors.

    Reviewers: Anna Sophie Blee-Goldman <[email protected]>, Almog Gavra <[email protected]>

commit 56ee139
Author: Antoine Pourchet <[email protected]>
Date:   Tue May 28 18:05:51 2024 -0600

    KAFKA-15045: (KIP-924 pt. 11) Implemented StickyTaskAssignor (apache#16052)

    This PR implements the StickyTaskAssignor with the new KIP 924 API.

    Reviewers: Anna Sophie Blee-Goldman <[email protected]>

commit 59ba555
Author: Nick Telford <[email protected]>
Date:   Wed May 29 00:23:23 2024 +0100

    KAFKA-15541: Add oldest-iterator-open-since-ms metric (apache#16041)

    Part of [KIP-989](https://cwiki.apache.org/confluence/x/9KCzDw).

    This new `StateStore` metric tracks the timestamp that the oldest
    surviving Iterator was created.

    This timestamp should continue to climb, and closely track the current
    time, as old iterators are closed and new ones created. If the timestamp
    remains very low (i.e. old), that suggests an Iterator has leaked, which
    should enable users to isolate the affected store.

    It will report no data when there are no currently open Iterators.

    Reviewers: Matthias J. Sax <[email protected]>

commit 4eb60b5
Author: Frederik Rouleau <[email protected]>
Date:   Tue May 28 23:56:47 2024 +0200

    KAFKA-16507 Add KeyDeserializationException and ValueDeserializationException with record content (apache#15691)

    Implements KIP-1036.

    Add raw ConsumerRecord data to RecordDeserialisationException to make DLQ implementation easier.

    Reviewers: Kirk True <[email protected]>, Andrew Schofield <[email protected]>, Matthias J. Sax <[email protected]>

commit 4d04eb8
Author: PoAn Yang <[email protected]>
Date:   Wed May 29 03:13:33 2024 +0800

    KAFKA-16796 Introduce new org.apache.kafka.tools.api.Decoder to replace kafka.serializer.Decoder (apache#16064)

    Reviewers: Chia-Ping Tsai <[email protected]>

commit a649bc4
Author: Luke Chen <[email protected]>
Date:   Wed May 29 00:05:49 2024 +0800

    KAFKA-16711: Make sure to update highestOffsetInRemoteStorage after log dir change (apache#15947)

    Reviewers: Kamal Chandraprakash<[email protected]>, Satish Duggana <[email protected]>

commit 64f699a
Author: Omnia Ibrahim <[email protected]>
Date:   Tue May 28 15:22:54 2024 +0100

    KAFKA-15853: Move general configs out of KafkaConfig (apache#16040)

    Reviewers: Mickael Maison <[email protected]>, Chia-Ping Tsai <[email protected]>

commit 699438b
Author: Sanskar Jhajharia <[email protected]>
Date:   Tue May 28 16:34:44 2024 +0530

    MINOR: Fix the config name in ProducerFailureHandlingTest (apache#16099)

    When moving from KafkaConfig.ReplicaFetchMaxBytesProp we used ReplicationConfigs.REPLICA_LAG_TIME_MAX_MS_CONFIG instead of ReplicationConfigs.REPLICA_FETCH_MAX_BYTES_CONFIG. This PR patches the same.

    Reviewers: Omnia Ibrahim <[email protected]>, Manikumar Reddy <[email protected]>

commit a57c05b
Author: Ken Huang <[email protected]>
Date:   Tue May 28 17:42:33 2024 +0900

    KAFKA-16805 Stop using a ClosureBackedAction to configure Spotbugs reports (apache#16081)

    Reviewers: Chia-Ping Tsai <[email protected]>

commit 91284d8
Author: Luke Chen <[email protected]>
Date:   Tue May 28 12:23:34 2024 +0800

    KAFKA-16709: abortAndPauseCleaning only when future log is not existed (apache#15951)

    When doing alter replica logDirs, we'll create a future log and pause log cleaning for the partition( here). And this log cleaning pausing will resume after alter replica logDirs completes (here). And when in the resuming log cleaning, we'll decrement 1 for the LogCleaningPaused count. Once the count reached 0, the cleaning pause is really resuming. (here). For more explanation about the logCleaningPaused state can check here.

    But, there's still one factor that could increase the LogCleaningPaused count: leadership change (here). When there's a leadership change, we'll check if there's a future log in this partition, if so, we'll create future log and pauseCleaning (LogCleaningPaused count + 1). So, if during the alter replica logDirs:

    1. alter replica logDirs for tp0 triggered (LogCleaningPaused count = 1)
    2. tp0 leadership changed (LogCleaningPaused count = 2)
    3. alter replica logDirs completes, resuming logCleaning (LogCleaningPaused count = 1)
    4. LogCleaning keeps paused because the count is always >  0

    This PR fixes this issue by only abortAndPauseCleaning when future log is not existed. We did the same check in alterReplicaLogDirs. So this change can make sure there's only 1 abortAndPauseCleaning for either abortAndPauseCleaning or maybeAddLogDirFetchers. Tests also added.

    Reviewers: Chia-Ping Tsai <[email protected]>, Igor Soarez <[email protected]>

commit adab48d
Author: Greg Harris <[email protected]>
Date:   Mon May 27 18:33:01 2024 -0700

    MINOR: Disable JDK 11 and 17 tests on PRs (apache#16051)

    Signed-off-by: Greg Harris <[email protected]>
    Reviewers: Justine Olshan <[email protected]>, David Arthur <[email protected]>, Ismael Juma <[email protected]>, Luke Chen <[email protected]>, Chia-Ping Tsai <[email protected]>

commit bac8df5
Author: Colin P. McCabe <[email protected]>
Date:   Mon May 27 08:53:53 2024 -0700

    MINOR: fix typo in KAFKA-16515

commit da3304e
Author: David Jacot <[email protected]>
Date:   Mon May 27 17:10:37 2024 +0200

    KAFKA-16371; fix lingering pending commit when handling OFFSET_METADATA_TOO_LARGE (apache#16072)

    This patch was initially created in apache#15536.

    When there is a commit for multiple topic partitions and some, but not all, exceed the offset metadata limit, the pending commit is not properly cleaned up leading to UNSTABLE_OFFSET_COMMIT errors when trying to fetch the offsets with read_committed. This change makes it so the invalid commits are not added to the pendingOffsetCommits set.

    Co-authored-by: Kyle Phelps <[email protected]>

    Reviewers: Chia-Ping Tsai <[email protected]>, Justine Olshan <[email protected]>

commit 524ad1e
Author: Kamal Chandraprakash <[email protected]>
Date:   Mon May 27 15:14:23 2024 +0530

    KAFKA-16452: Don't throw OOORE when converting the offset to metadata (apache#15825)

    Don't throw OFFSET_OUT_OF_RANGE error when converting the offset to metadata, and next time the leader should increment the high watermark by itself after receiving fetch requests from followers. This can happen when checkpoint files are missing and being elected as a leader.

    Reviewers: Luke Chen <[email protected]>, Jun Rao <[email protected]>

commit d9ee9c9
Author: Nick Telford <[email protected]>
Date:   Sat May 25 20:22:56 2024 +0100

    KAFKA-15541: Use LongAdder instead of AtomicInteger (apache#16076)

    `LongAdder` performs better than `AtomicInteger` when under contention
    from many threads. Since it's possible that many Interactive Query
    threads could create a large number of `KeyValueIterator`s, we don't
    want contention on a metric to be a performance bottleneck.

    The trade-off is memory, as `LongAdder` uses more memory to space out
    independent counters across different cache lines. In practice, I don't
    expect this to cause too many problems, as we're only constructing 1
    per-store.

    Reviewers: Matthias J. Sax <[email protected]>

commit a8d166c
Author: Ritika Reddy <[email protected]>
Date:   Sat May 25 09:06:15 2024 -0700

    KAFKA-16625; Reverse lookup map from topic partitions to members (apache#15974)

    This patch speeds up the computation of the unassigned partitions by exposing the inverted target assignment. It allows the assignor to check whether a partition is assigned or not.

    Reviewers: Jeff Kim <[email protected]>, David Jacot <[email protected]>

commit d585a49
Author: Jeff Kim <[email protected]>
Date:   Fri May 24 16:33:57 2024 -0400

    KAFKA-16831: CoordinatorRuntime should initialize MemoryRecordsBuilder with max batch size write limit (apache#16059)

    CoordinatorRuntime should initialize MemoryRecordsBuilder with max batch size write limit. Otherwise, we default the write limit to the min buffer size of 16384 for the write limit. This causes the coordinator to threw RecordTooLargeException even when it's under the 1MB max batch size limit.

    Reviewers: David Jacot <[email protected]>

commit 8eea6b8
Author: Edoardo Comar <[email protected]>
Date:   Fri May 24 20:33:00 2024 +0100

    MINOR: mention KAFKA-15905 in docs "Notable changes in 3.7.1" (apache#16070)

    * MINOR: mention KAFKA-15905 in docs "Notable changes in 3.7.1/3.8.0"

    Co-Authored-By: Adrian Preston <[email protected]>

commit 4f55786
Author: Colin P. McCabe <[email protected]>
Date:   Mon May 20 15:41:52 2024 -0700

    KAFKA-16515: Fix the ZK Metadata cache confusion between brokers and controllers

    ZkMetadataCache could theoretically return KRaft controller information from a call to
    ZkMetadataCache.getAliveBrokerNode, which doesn't make sense. KRaft controllers are not part of the
    set of brokers. The only use-case for this functionality was in MetadataCacheControllerNodeProvider
    during ZK migration, where it allowed ZK brokers in migration mode to forward requests to
    kcontrollers when appropriate. This PR changes MetadataCacheControllerNodeProvider to simply
    delegate to quorumControllerNodeProvider in this case.

    Reviewers: José Armando García Sancio <[email protected]>

commit 90892ae
Author: Colin P. McCabe <[email protected]>
Date:   Mon May 20 16:23:27 2024 -0700

    KAFKA-16516: Fix the controller node provider for broker to control channel

    Fix the code in the RaftControllerNodeProvider to query RaftManager to find Node information,
    rather than consulting a static map. Add a RaftManager.voterNode function to supply this
    information. In KRaftClusterTest, add testControllerFailover to get more coverage of controller
    failovers.

    Reviewers: José Armando García Sancio <[email protected]>

commit 2432a18
Author: KrishVora01 <[email protected]>
Date:   Fri May 24 22:21:02 2024 +0530

    KAFKA-16373: KIP-1028:  Adding code to support Apache Kafka Docker Official Images (apache#16027)

    This PR aims to add JVM based Docker Official Image for Apache Kafka as per the following KIP - https://cwiki.apache.org/confluence/display/KAFKA/KIP-1028%3A+Docker+Official+Image+for+Apache+Kafka

    This PR adds the following functionalities:
    Introduces support for Apache Kafka Docker Official Images via:

    GitHub Workflows:

    - Workflow to prepare static source files for Docker images
    - Workflow to build and test Docker official images
    - Scripts to prepare source files and perform Docker image builds and tests

    A new directory for Docker official images, named docker/docker_official_images. This is the new directory to house all Docker Official Image assets.

    Co-authored-by: Vedarth Sharma <[email protected]>

    Reviewers: Manikumar Reddy <[email protected]>, Vedarth Sharma <[email protected]>

commit 0143c72
Author: Lianet Magrans <[email protected]>
Date:   Fri May 24 14:19:43 2024 +0200

    KAFKA-16815: Handle FencedInstanceId in HB response (apache#16047)

    Handle FencedInstanceIdException that a consumer may receive in the heartbeat response. This will be the case when a static consumer is removed from the group by and admin client, and another member joins with the same group.instance.id (allowed in). The first member will receive a FencedInstanceId on its next heartbeat. The expectation is that this should be handled as a fatal error.

    There are no actual changes in logic with this PR, given that without being handled, the FencedInstanceId was being treated as an "unexpected error", which are all treated as fatal errors, so the outcome remains the same. But we're introducing this small change just for accuracy in the logic and the logs: FencedInstanceId is expected during heartbeat, a log line is shown describing the situation and why it happened (and it's treated as a fatal error, just like it was before this PR).

    This PR also improves the test to ensure that the error propagated to the app thread matches the one received in the HB.

    Reviewers: Andrew Schofield <[email protected]>, David Jacot <[email protected]>

commit c5cd190
Author: Gantigmaa Selenge <[email protected]>
Date:   Fri May 24 11:50:47 2024 +0100

    MINOR: Refactor SSL/SASL admin integration tests to not use a custom authorizer (apache#15377)

    Reviewers: Mickael Maison <[email protected]>

commit 520aa86
Author: Jeff Kim <[email protected]>
Date:   Fri May 24 03:51:50 2024 -0400

    KAFKA-16626; Lazily convert subscribed topic names to topic ids (apache#15970)

    This patch aims to remove the data structure that stores the conversion from topic names to topic ids which was taking time similar to the actual assignment computation. Instead, we reuse the already existing ConsumerGroupMember.subscribedTopicNames() and do the conversion to topic ids when the iterator is requested.

    Reviewers: David Jacot <[email protected]>

commit 6941598
Author: Krishna Agarwal <[email protected]>
Date:   Fri May 24 12:16:01 2024 +0530

    KAFKA-16826: Integrate Native Docker Image with github actions (apache#16045)

    This PR integrates the Native docker image with the existing github action jobs for the jvm docker image of AK.

    The integration is done to the following actions:

    docker_build_and_test.yml: Builds the docker image and runs sanity tests and CVE scan
    docker_rc_release.yml: Builds the RC docker image for both amd and arm platform and pushes it to the dockerhub.
    docker_promote.yml: Promotes the RC docker image to the released image tag

    Reviewers: Manikumar Reddy <[email protected]>, Vedarth Sharma <[email protected]>

commit de32028
Author: Kuan-Po (Cooper) Tseng <[email protected]>
Date:   Fri May 24 05:25:53 2024 +0800

    KAFKA-16828 RackAwareTaskAssignorTest failed (apache#16044)

    Reviewers: Chia-Ping Tsai <[email protected]>

commit 11ad5e8
Author: Greg Harris <[email protected]>
Date:   Thu May 23 13:23:18 2024 -0700

    MINOR: Refactor Values class to fix checkstyle, add benchmark, optimize exceptions (apache#15469)

    Signed-off-by: Greg Harris <[email protected]>
    Reviewers: Mickael Maison <[email protected]>
chiacyu pushed a commit to chiacyu/kafka that referenced this pull request Jun 1, 2024
Part of [KIP-989](https://cwiki.apache.org/confluence/x/9KCzDw).

This new `StateStore` metric tracks the timestamp that the oldest
surviving Iterator was created.

This timestamp should continue to climb, and closely track the current
time, as old iterators are closed and new ones created. If the timestamp
remains very low (i.e. old), that suggests an Iterator has leaked, which
should enable users to isolate the affected store.

It will report no data when there are no currently open Iterators.

Reviewers: Matthias J. Sax <[email protected]>
wernerdv pushed a commit to wernerdv/kafka that referenced this pull request Jun 3, 2024
Part of [KIP-989](https://cwiki.apache.org/confluence/x/9KCzDw).

This new `StateStore` metric tracks the timestamp that the oldest
surviving Iterator was created.

This timestamp should continue to climb, and closely track the current
time, as old iterators are closed and new ones created. If the timestamp
remains very low (i.e. old), that suggests an Iterator has leaked, which
should enable users to isolate the affected store.

It will report no data when there are no currently open Iterators.

Reviewers: Matthias J. Sax <[email protected]>
TaiJuWu pushed a commit to TaiJuWu/kafka that referenced this pull request Jun 8, 2024
Part of [KIP-989](https://cwiki.apache.org/confluence/x/9KCzDw).

This new `StateStore` metric tracks the timestamp that the oldest
surviving Iterator was created.

This timestamp should continue to climb, and closely track the current
time, as old iterators are closed and new ones created. If the timestamp
remains very low (i.e. old), that suggests an Iterator has leaked, which
should enable users to isolate the affected store.

It will report no data when there are no currently open Iterators.

Reviewers: Matthias J. Sax <[email protected]>
gongxuanzhang pushed a commit to gongxuanzhang/kafka that referenced this pull request Jun 12, 2024
Part of [KIP-989](https://cwiki.apache.org/confluence/x/9KCzDw).

This new `StateStore` metric tracks the timestamp that the oldest
surviving Iterator was created.

This timestamp should continue to climb, and closely track the current
time, as old iterators are closed and new ones created. If the timestamp
remains very low (i.e. old), that suggests an Iterator has leaked, which
should enable users to isolate the affected store.

It will report no data when there are no currently open Iterators.

Reviewers: Matthias J. Sax <[email protected]>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
kip Requires or implements a KIP streams
Projects
None yet
Development

Successfully merging this pull request may close these issues.

2 participants