The group coordinator expects the instance ID to always be sent when
leaving the group in a static membership configuration, see
ea94507679/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java (L814)
The failure was silent, because the group coordinator does not log
failed requests and the consumer doesn't wait for the heartbeat response
during close.
Reviewers: Matthias J. Sax <matthias@confluent.io>, Kirk True <ktrue@confluent.io>, Bruno Cadonna <cadonna@apache.org>
The consumer keeps a poll timer, which is used to ensure liveness of the application thread. The poll timer automatically updates while the Consumer.poll(Duration) method is blocked, while the newer consumer only updates the poll timer when a new call to Consumer.poll(Duration) is issued. This means that the kafka-console-consumer.sh tools, which uses a very long timeout by default, works differently with the new consumer, with the consumer proactively rejoining the group during long poll timeouts.
This change solves the problem by (a) repeatedly sending PollApplicationEvents to the background thread, not just on the first call of poll and (b) making sure that the application thread doesn't block for so long that it runs out of max.poll.interval.
An integration test is added to make sure that we do not rejoin the group when a long poll timeout is used with a low max.poll.interval.
Reviewers: Lianet Magrans <lianetmr@gmail.com>, Andrew Schofield <aschofield@confluent.io>, Bruno Cadonna <cadonna@apache.org>
The assertion to check BytesOut doesn't include replication was performed before replication occurred.
This PR fixed the position of the assertion.
Reviewers: Luke Chen <showuon@gmail.com>
This PR cleans up: metrics, migration, network, raft, security, serializer, tools, utils, and zookeeper package classes
Mark methods and fields private where possible
Annotate public methods and fields
Remove unused classes and methods
Make sure Arrays are not printed with .toString
Optimize minor warnings
Reviewers: Mickael Maison <mickael.maison@gmail.com>
The wake-up mechanism in the new consumer is preventing from committing within a rebalance listener callback. The reason is that we are trying to register two wake-uppable actions at the same time.
The fix is to register the wake-uppable action more closely to where we are in fact blocking on it, so that the action is not registered when we execute rebalance listeneners and callback listeners.
Reviewers: Bruno Cadonna <cadonna@apache.org>
This integration test is now passing, presumably based on recent related changes. Re-enabling to ensure it is included in the test suite to catch any regressions.
Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
The main bug causing this test to fail as described in the ticket was already fixed.
The test is still flaky if unchanged, because in the new consumer, the assignment can
change in between two polls. Interceptors are only executed inside poll (and have to be,
since they must run as part of the application thread), so we need to modify the
integration test to call poll once after observing that the assignment changed.
Reviewers: Bruno Cadonna <bruno@confluent.io>
Fix the flakiness of LogDirFailureTest by setting a separate metadata.log.dir for brokers in KRAFT mode.
The test was flaky because as we call causeLogDirFailure some times we impact the first log.dir which also is KafkaConfig.metadataLogDir as we don't have metadata.log.dir. So to fix the flakiness we need to explicitly set metadata.log.dir to diff log dir than the ones we could potentially fail for the tests.
This is part 1 of the fixes. Delivering them separately as the other issues were not as clear cut.
Reviewers: Gaurav Narula <gaurav_narula2@apple.com>, Justine Olshan <jolshan@confluent.io>, Greg Harris <greg.harris@aiven.io>
This PR is part of #14471
Is contains some of ConsoleGroupCommand tests rewritten in java.
Intention of separate PR is to reduce changes and simplify review.
Reviewers: Luke Chen <showuon@gmail.com>
- Removing ! and Unused Imports
- Put a space after the control structure's defining keyword.
- remove unnecessary whitespace a space after the method name in higher-order function invocations.
Reviewers: Divij Vaidya <diviv@amazon.com>
This patch adds the support for filtering groups by types (Classic or Consumer) to both the old and the new group coordinators.
Reviewers: David Jacot <djacot@confluent.io>
In Kraft mode, the broker fails to handle topic recreation correctly with broken disks. This is because ReplicaManager tracks HostedPartitions which are on an offline disk but it doesn't associate TopicId information with them.
This change updates HostedPartition.Offline to associate topic id information. We also update the log creation logic in Partition::createLogInAssignedDirectoryId to not just rely on targetLogDirectoryId == DirectoryId.UNASSIGNED to determine if the log to be created is "new".
Please refer to the comments in https://issues.apache.org/jira/browse/KAFKA-16157 for more information.
Reviewers: Luke Chen <showuon@gmail.com>, Omnia Ibrahim <o.g.h.ibrahim@gmail.com>, Gaurav Narula <gaurav_narula2@apple.com>
While migrating from ZK mode to KRaft mode, the broker passes through a "hybrid" phase, in which it
receives LeaderAndIsrRequest and UpdateMetadataRequest RPCs from the KRaft controller. For the most
part, these RPCs can be handled just like their traditional equivalents from a ZK-based controller.
However, there is one thing that is different: the way topic deletions are handled.
In ZK mode, there is a "deleting" state which topics enter prior to being completely removed.
Partitions stay in this state until they are removed from the disks of all replicas. And partitions
associated with these deleting topics show up in the UMR and LAIR as having a leader of -2 (which
is not a valid broker ID, of course, because it's negative). When brokers receive these RPCs, they
know to remove the associated partitions from their metadata caches, and disks. When a full UMR or
ISR is sent, deleting partitions are included as well.
In hybrid mode, in contrast, there is no "deleting" state. Topic deletion happens immediately. We
can do this because we know that we have topic IDs that are never reused. This means that we can
always tell the difference between a broker that had an old version of some topic, and a broker
that has a new version that was re-created with the same name. To make this work, when handling a
full UMR or LAIR, hybrid brokers must compare the full state that was sent over the wire to their
own local state, and adjust accordingly.
Prior to this PR, the code for handling those adjustments had several major flaws. The biggest flaw
is that it did not correctly handle the "re-creation" case where a topic named FOO appears in the
RPC, but with a different ID than the broker's local FOO. Another flaw is that a problem with a
single partition would prevent handling the whole request.
In ZkMetadataCache.scala, we handle full UMR requests from KRaft controllers by rewriting the UMR
so that it contains the implied deletions. I fixed this code so that deletions always appear at the
start of the list of topic states. This is important for the re-creation case since it means that a
single request can both delete the old FOO and add a new FOO to the cache. Also, rather than
modifying the requesst in-place, as the previous code did, I build a whole new request with the
desired list of topic states. This is much safer because it avoids unforseen interactions with
other parts of the code that deal with requests (like request logging). While this new copy may
sound expensive, it should actually not be. We are doing a "shallow copy" which references the
previous list topic state entries.
I also reworked ZkMetadataCache.updateMetadata so that if a partition is re-created, it does not
appear in the returned set of deleted TopicPartitions. Since this set is used only by the group
manager, this seemed appropriate. (If I was in the consumer group for the previous iteration of
FOO, I should still be in the consumer group for the new iteration.)
On the ReplicaManager.scala side, we handle full LAIR requests by treating anything which does not
appear in them as a "stray replica." (But we do not rewrite the request objects as we do with UMR.)
I moved the logic for finding stray replicas from ReplicaManager into LogManager. It makes more
sense there, since the information about what is on-disk is managed in LogManager. Also, the stray
replica detection logic for KRaft mode is there, so it makes sense to put the stray replica
detection logic for hybrid mode there as well.
Since the stray replica detection is now in LogManager, I moved the unit tests there as well.
Previously some of those tests had been in BrokerMetadataPublisherTest for historical reasons.
The main advantage of the new LAIR logic is that it takes topic ID into account. A replica can be a
stray even if the LAIR contains a topic of the given name, but a different ID. I also moved the
stray replica handling earlier in the becomeLeaderOrFollower function, so that we could correctly
handle the "delete and re-create FOO" case.
Reviewers: David Arthur <mumrah@gmail.com>
In KRaft mode, or on ZK brokers that are migrating to KRaft, we have a local __cluster_metadata
log. This log is stored in a single log directory which is configured via metadata.log.dir. If
there is no metadata.log.dir given, it defaults to the first entry in log.dirs. In the future we
may support multiple metadata log directories, but we don't yet. For now, we must abort the
process when this log directory fails.
In ZK mode, it is not necessary to abort the process when this directory fails, since there is no
__cluster_metadata log there. This PR changes the logic so that we check for whether we're in ZK
mode and do not abort in that scenario (unless we lost the final remaining log directory. of
course.)
Reviewers: Luke Chen <showuon@gmail.com>, Colin P. McCabe <cmccabe@apache.org>, Omnia G H Ibrahim <o.g.h.ibrahim@gmail.com>, Proven Provenzano <pprovenzano@confluent.io>
During migration from ZK mode to KRaft mode, there is a step where the kcontrollers load all of the
data from ZK into the metadata log. Previously, we were using a batch size of 1000 for this, but
200 seems better. This PR also adds an internal configuration to control this batch size, for
testing purposes.
Reviewers: Colin P. McCabe <cmccabe@apache.org>
This patch extends the Admin client to support describing new consumer groups with the ConsumerGroupDescribe API introduced in KIP-848. Users will continue to use the `Admin#describeConsumerGroups` API. The admin client does all the magic. Basically, the admin client always tries to describe the requested groups with the ConsumerGroupDescribe API to start with. If all the groups are there, great, the job is done. If there are groups unresolved groups due to a UNSUPPORTED_VERSION or GROUP_ID_NOT_FOUND error, the admin client tries with the DescribeGroups API. The patch also adds fields to the data structure returned by `Admin#describeConsumerGroups` as stated in the KIP.
Reviewers: Andrew Schofield <aschofield@confluent.io>, Bruno Cadonna <bruno@confluent.io>
* MINOR: Clean up core server classes
Mark methods and fields private where possible
Annotate public methods and fields
Remove unused classes and methods
Make sure Arrays are not printed with .toString
Optimize minor warnings
Remove unused apply method
Signed-off-by: Josep Prat <josep.prat@aiven.io>
Reviewers: Mickael Maison <mickael.maison@gmail.com>
This patch removes the extra hop via the request thread when the new group coordinator verifies a transaction. Prior to it, the ReplicaManager would automatically re-schedule the callback to a request thread. However, the new group coordinator does not need this as it already schedules the write into its own thread. With this patch, the decision to re-schedule on a request thread or not is left to the caller.
Reviewers: Artem Livshits <alivshits@confluent.io>, Justine Olshan <jolshan@confluent.io>
We update metadata update handler to resend broker registration when
metadata has been updated to >= 3.7IV2 so that the controller becomes
aware of the log directories in the broker.
We also update DirectoryId::isOnline to return true on an empty list of
log directories while the controller awaits broker registration.
Co-authored-by: Proven Provenzano <pprovenzano@confluent.io>
Reviewers: Omnia G H Ibrahim <o.g.h.ibrahim@gmail.com>, Luke Chen <showuon@gmail.com>, Colin P. McCabe <cmccabe@apache.org>
Because of lack of implicit conversions, boolean properties need to be
passed as Strings
This is done in other parts of the code already
Signed-off-by: Josep Prat <josep.prat@aiven.io>
When a broker is down, and a topic is deleted, this will result in that broker seeing "stray
replicas" the next time it starts up. These replicas contain data that used to be important, but
which now needs to be deleted. Stray replica deletion is handled during the initial metadata
publishing step on the broker.
Previously, we deleted these stray replicas after starting up BOTH LogManager and ReplicaManager.
However, this wasn't quite correct. The presence of the stray replicas confused ReplicaManager.
Instead, we should delete the stray replicas BEFORE starting ReplicaManager.
This bug triggered when a topic was deleted and re-created while a broker was down, and some of the
replicas of the re-created topic landed on that broker. The impact was that the stray replicas were
deleted, but the new replicas for the next iteration of the topic never got created. This, in turn,
led to persistent under-replication until the next time the broker was restarted.
Reviewers: Luke Chen <showuon@gmail.com>, Omnia G H Ibrahim <o.g.h.ibrahim@gmail.com>, Gaurav Narula <gaurav_narula2@apple.com>
This patch causes the active KRaftMigrationDriver to reload the /migration ZK state after electing
itself as the leader in ZK. This closes a race condition where the previous active controller could
make an update to /migration after the new leader was elected. The update race was not actually a
problem regarding the data since both controllers would be syncing the same state from KRaft to ZK,
but the change to the znode causes the new controller to fail on the zk version check on
/migration.
This patch also fixes a as-yet-unseen bug where the active controllers failing to elect itself via
claimControllerLeadership would not retry.
Reviewers: Colin P. McCabe <cmccabe@apache.org>
We need to make sure to call the consumer interceptor and test its integration.
This is adding the required call in commitSync and commitAsync. The calls in commitAsync are executed using the same mechanism as commit callbacks, to ensure that we are calling the interceptors from a single thread, as is intended in the original KIP.
The interceptors also need to be invoked on auto-commits which are executed in the commit request manager. For this purpose, we share the OffsetCommitCallbackInvoker class with the background thread (it is already accessed implicitly from the background thread through a future lambda). This is done analogous to the RebalanceListenerInvoker.
Co-authored-by: John Doe zh2725284321@gmail.com
Reviewers: Bruno Cadonna <bruno@confluent.io>, Andrew Schofield <aschofield@confluent.io>, Philip Nee <pnee@confluent.io>
While using —list —state the current accepted values correspond to the classic group type states. This patch adds the new states introduced by KIP-848. It also make the matching on the server case insensitive.
Co-authored-by: d00791190 <dinglan6@huawei.com>
Reviewers: Ritika Reddy <rreddy@confluent.io>, David Jacot <djacot@confluent.io>
In BrokerTopicMetrics group, we'll provide not only the metric for per topic, but also the all topic aggregated metric value. The beanName is like this:
kafka.server:type=BrokerTopicMetrics,name=RemoteCopyLagSegments
kafka.server:type=BrokerTopicMetrics,name=RemoteCopyLagSegments,topic=Leader
This PR is to add the missing all topic aggregated metric value for tiered storage, specifically for gauge type metrics.
Reviewers: Divij Vaidya <divijvaidya13@gmail.com>, Kamal Chandraprakash<kamal.chandraprakash@gmail.com>, Christo Lolov <lolovc@amazon.com>
I was using the ZERO_UUID topicId instead of the actual topicId in the testFetchResponseContainsNewLeaderOnNotLeaderOrFollower introduced in #14444, updating as the actual topicId is more correct.
Reviewers: Justine Olshan <jolshan@confluent.io>
This PR is part of #14471
Is contains single test rewritten in java.
Intention of separate PR is to reduce changes and simplify review.
Reviewers: Justine Olshan <jolshan@confluent.io>
I originally did some refactors in #14774, but we decided to keep the changes minimal since the ticket was a blocker. Here are those refactors:
* Removed separate append paths so that produce, group coordinator, and other append paths all call appendRecords
* AppendRecords has been simplified
* Removed unneeded error conversions in verification code since group coordinator and produce path convert errors differently, removed test for that
* Fixed incorrect capital param name in KafkaRequestHandler
* Updated ReplicaManager test to handle produce appends separately when transactions are used.
Reviewers: David Jacot <djacot@confluent.io>, Jason Gustafson <jason@confluent.io>
* MINOR: Clean up core api, cluster, common, log, admin, controller and coordinator classes
Mark methods and fields private where possible
Annotate public methods and fields
Remove unused classes and methods
Signed-off-by: Josep Prat <josep.prat@aiven.io>
Reviewers: Luke Chen <showuon@gmail.com>
After this #13107 PR, an if-else block became unreachable. We need remove it and make the code clean.
Reviewers: Luke Chen <showuon@gmail.com>, Divij Vaidya <diviv@amazon.com>
This patch implements the new DescribeTopicPartitions RPC as defined in KIP-966 (ELR). Additionally, this patch adds a broker config "max.request.partition.size.limit" which limits the number of partitions returned by the new RPC.
Reviewers: Artem Livshits <alivshits@confluent.io>, Jason Gustafson <jason@confluent.io>, David Arthur <mumrah@gmail.com>
This ensures that no records are fetched, or positions initialized, while the onPartitionsAssigned callback completes in the new async consumer Application thread. This is achieved using an internal mark in the subscription state, so that the partitions are not considered fetchable or requiring initializing positions until the callback completes.
Reviewers: David Jacot <djacot@confluent.io>
KIP-714 requires client instance cache in broker which should also have a time-based eviction policy where client instances which are not actively sending metrics should be evicted. KIP mentions This client instance specific state is maintained in broker memory up to MAX(60*1000, PushIntervalMs * 3) milliseconds.
Reviewers: Andrew Schofield <aschofield@confluent.io>, Jun Rao <junrao@gmail.com>
This originally was #14489 which covered 2 aspects -- reloading on partition epoch changes where leader epoch did not change and reloading when leader epoch changed but we were already the leader.
I've cut out the second part of the change since the first part is much simpler.
Redefining the TopicDelta fields to better distinguish when a leader is elected (leader epoch bump) vs when a leader has isr/replica changes (partition epoch bump). There are some cases where we bump the partition epoch but not the leader epoch. We do not need to do operations that only care about the leader epoch bump. (ie -- onElect callbacks)
Reviewers: Artem Livshits <alivshits@confluent.io>, José Armando García Sancio <jsancio@apache.org>
When transactional offset commits are eventually committed, we must always keep the most recent committed when we have a mix of transactional and regular offset commits. We achieve this by storing the offset of the offset commit record along side the committed offset in memory. Without preserving information of the commit record offset, compaction of the __consumer_offsets topic itself may result in the wrong offset commit being materialized.
Reviewers: Jeff Kim <jeff.kim@confluent.io>, Justine Olshan <jolshan@confluent.io>
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Ismael Juma <ismael@juma.me.uk> , David Jacot <djacot@confluent.io>, Nikolay <NIzhikov@gmail.com>
PR adds support to capture client socket port information in Request Context. The port from request context is used as matching criteria in filtering clients and shall be used by metrics plugin to fetch port from request context.
Reviewers: Andrew Schofield <aschofield@confluent.io>, Jun Rao <junrao@gmail.com>
We reevaluated the integration tests that were disabled for the new consumer group protocol which should be supported. The evaluation was to run the PlaintextConsumerTest suite ten times and see which tests passed and which failed.
Based on that evaluation, the following test can now be enabled:
testAutoCommitOnClose
testAutoCommitOnRebalance
testExpandingTopicSubscriptions
testMultiConsumerSessionTimeoutOnClose
testMultiConsumerSessionTimeoutOnStopPolling
testShrinkingTopicSubscriptions
There are three tests which consistently failed. For each, a dedicated Jira was created to track and fix. Those that failed:
testAutoCommitOnCloseAfterWakeup (KAFKA-16167)
testPerPartitionLagMetricsCleanUpWithSubscribe (local failure rate 100%, KAFKA-16150)
testPerPartitionLeadMetricsCleanUpWithSubscribe (local failure rate: 70%, KAFKA-16151)
testStaticConsumerDetectsNewPartitionCreatedAfterRestart (local failure rate: 100%, KAFKA-16152)
See KAFKA-16104 for more details.
Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
The current load summary exposes the time from when the partition load operation is scheduled to when the load completes. We are missing the information of how long the scheduled operation stays in the scheduler. Log that information.
Reviewers: David Jacot <djacot@confluent.io>
This PR adds a test case for follower fetch when segments are archived and expired from remote storage. This test case verifies the following scenario (from comment):
1. Leader is archiving to tiered storage and has a follower.
2. Follower has caught up to offset X (exclusive).
3. While follower is offline, leader moves X to tiered storage and expires data locally till Y, such that, Y = leaderLocalLogStartOffset and leaderLocalLogStartOffset > X. Meanwhile, X has been expired from tiered storage as well. Hence, X < globalLogStartOffset.
4. Follower comes online and tries to fetch X from leader.
Reviewers: Luke Chen <showuon@gmail.com>, Kamal Chandraprakash <kamal.chandraprakash@gmail.com>
This PR creates MetadataVersion.latestTesting to represent the highest metadata version (which may be unstable) and MetadataVersion.latestProduction to represent the latest version that should be used in production. It fixes a few cases where the broker was advertising that it supported the testing versions even when unstable metadata versions had not been configured.
Reviewers: Colin P. McCabe <cmccabe@apache.org>, Ismael Juma <ismael@juma.me.uk>
Some kcontroller dynamic configurations may fail to apply at startup. This happens because there is
a race between registering the reconfigurables to the DynamicBrokerConfig class, and receiving the
first update from the metadata publisher. We can fix this by registering the reconfigurables first.
This seems to have been introduced by the "MINOR: Install ControllerServer metadata publishers
sooner" change.
Reviewers: Ron Dagostino <rdagostino@confluent.io>
This fixes an issue with the time boundaries used for the auto-commit performed when partitions are revoked.
Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
When we are migrating from ZK mode to KRaft mode, the brokers pass through a phase where they are
running in ZK mode, but the controller is in KRaft mode (aka a kcontroller). This is called "hybrid
mode." In hybrid mode, the KRaft controllers send old-style controller RPCs to the remaining ZK
mode brokers. (StopReplicaRequest, LeaderAndIsrRequest, UpdateMetadataRequest, etc.)
To complete partition reassignment, the kcontroller must send a StopReplicaRequest to any brokers
that no longer host the partition in question. Previously, it was sending this StopReplicaRequest
with delete = false. This led to stray partitions, because the partition data was never removed as
it should have been. This PR fixes it to set delete = true. This fixes KAFKA-16120.
There is one additional problem with partition reassignment in hybrid mode, tracked as KAFKA-16121.
The issue is that in ZK mode, brokers ignore any LeaderAndIsr request where the partition leader
epoch is less than or equal to the current partition leader epoch. However, when in hybrid mode,
just as in KRaft mode, we do not bump the leader epoch when starting a new reassignment, see:
`triggerLeaderEpochBumpIfNeeded`. This PR resolves this problem by adding a special case on the
broker side when isKRaftController = true.
Reviewers: Akhilesh Chaganti <akhileshchg@users.noreply.github.com>, Colin P. McCabe <cmccabe@apache.org>
Fetching from remote log segment implementation does not handle the topics that had retention policy as compact earlier and changed to delete. It always assumes record batch will exist in the required segment for the requested offset. But there is a possibility where the requested offset is the last offset of the segment and has been removed due to log compaction. Then it requires iterating over the next higher segment for further data as it has been done for local segment fetch request.
This change partially addresses the above problem by iterating through the remote log segments to find the respective segment for the target offset.
Reviewers: Satish Duggana <satishd@apache.org>, Kamal Chandraprakash<kamal.chandraprakash@gmail.com>, Divij Vaidya <diviv@amazon.com>, Christo Lolov <lolovc@amazon.com>
This patch wires the transaction verification in the new group coordinator. It basically calls the verification path before scheduling the write operation. If the verification fails, the error is returned to the caller.
Note that the patch uses `appendForGroup`. I suppose that we will move away from using it when https://github.com/apache/kafka/pull/15087 is merged.
Reviewers: Justine Olshan <jolshan@confluent.io>
Following @dajac 's finding in #15063, I found we also create new RemoteLogManager in ReplicaManagerTest, but didn't close them.
While investigating ReplicaManagerTest, I also found there are other threads leaking:
1. remote fetch reaper thread. It's because we create a reaper thread in test, which is not expected. We should create a mocked one like other purgatory instance.
2. Throttle threads. We created a quotaManager to feed into the replicaManager, but didn't close it. Actually, we have created a global quotaManager instance and will close it on AfterEach. We should re-use it.
3. replicaManager and logManager didn't invoke close after test.
Reviewers: Divij Vaidya <divijvaidya13@gmail.com>, Satish Duggana <satishd@apache.org>, Justine Olshan <jolshan@confluent.io>
Migrates functionality provided by utility to Kafka core. This wrapper will be used to generate property files and format storage when invoked from docker container.
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Viktor Somogyi-Vass <viktorsomogyi@gmail.com>, Manikumar Reddy <manikumar.reddy@gmail.com>
This patch moves the `RaftIOThread` implementation into Java. I changed the name to `KafkaRaftClientDriver` since the main thing it does is drive the calls to `poll()`. There shouldn't be any changes to the logic.
Reviewers: José Armando García Sancio <jsancio@apache.org>
Throw UnknownTopicIdException instead of InvalidTopicException when no name is found for the topic ID.
Similar to #6124 for describeTopics using a topic name. MockAdminClient already makes use of UnknownTopicIdException for this case.
Reviewers: Justine Olshan <jolshan@confluent.io>, Ashwin Pankaj <apankaj@confluent.io>
updated GroupCoordinatorIntegrationTest.testGroupCoordinatorPropagatesOffsetsTopicCompressionCodec to support KRaft
Reviewers: Justine Olshan <jolshan@confluent.io>
Mockito will keep the invocation history in the test suite and cause the huge heap usage. Since the mock replicaManager is only used to bypass the replicaManager constructor without verifying/mocking anything, we create a real dummy replicaManager to avoid the mockito invocation history in memory.
Reviewers: Luke Chen <showuon@gmail.com>, Justine Olshan <jolshan@confluent.io>
Co-authored-by: Luke Chen <showuon@gmail.com>
It seems like this PR (https://github.com/apache/kafka/pull/8768) duplicated the implementation to QuotaUtils, but didn't remove this implementation and private methods that is using
Reviewers: Justine Olshan <jolshan@confluent.io>
The controllerApi will create some resources, including the reaper threads. In ControllerApisTest, we created it on many test cases, but didn't close it. This commit doesn't change anything in the business logic of the test, it just adds try/finally to close the controllerApi instance.
Reviewers: Divij Vaidya <diviv@amazon.com>
This commit closes the KDC server properly in `CustomQuotaCallbackTest` and `AclAuthorizerWithZkSaslTest`.
Reviewers: Justine Olshan <jolshan@confluent.io>
related to KAFKA-15818
This is a bug in the AsyncKafkaConsumer poll loop that it does not send an event to the network thread to acknowledge user poll. This causes a few issues:
Autocommit won't work without user setting the timer
the member will just leave the group after rebalance timeout and never able to rejoin.
In this PR, a few subtle changes are made to address this issue
Hook up poll event to the AsyncKafkaConsumer#poll. It is only fired once per invocation
Upon entering staled state, we need to reset HeartbeatState otherwise we will get an invalid request
We will clear and current assignment and remove all assigned partitions once the heartbeat is sent. See changes in onHeartbeatRequestSent
Reviewers: David Jacot <djacot@confluent.io>, Bruno Cadonna <cadonna@apache.org>, Andrew Schofield <aschofield@confluent.io>
I was investigating a build which failed with "exit 1". In the logs of the broker, I was that the first call to exist was caught. However, a second one was not. See the logs below. The issue seems to be that we must first shutdown the cluster before reseting the exit catcher. Otherwise, there is still a change for the broker to call exit.
```
[2023-12-21 13:52:59,310] ERROR Shutdown broker because all log dirs in /tmp/kafka-2594137463116889965 have failed (kafka.log.LogManager:143)
[2023-12-21 13:52:59,312] ERROR test error (kafka.server.epoch.EpochDrivenReplicationProtocolAcceptanceWithIbp26Test:76)
java.lang.RuntimeException: halt(1, null) called!
at kafka.server.QuorumTestHarness.$anonfun$setUp$4(QuorumTestHarness.scala:273)
at org.apache.kafka.common.utils.Exit.halt(Exit.java:63)
at kafka.utils.Exit$.halt(Exit.scala:33)
at kafka.log.LogManager.handleLogDirFailure(LogManager.scala:224)
at kafka.server.ReplicaManager.handleLogDirFailure(ReplicaManager.scala:2600)
at kafka.server.ReplicaManager$LogDirFailureHandler.doWork(ReplicaManager.scala:324)
at org.apache.kafka.server.util.ShutdownableThread.run(ShutdownableThread.java:131)
```
```
[2023-12-21 13:53:05,797] ERROR Shutdown broker because all log dirs in /tmp/kafka-7355495604650755405 have failed (kafka.log.LogManager:143)
```
Reviewers: Luke Chen <showuon@gmail.com>
This patch ensures that the RemoteLogManager is closed in RemoteLogManagerTest.
Reviewers: Divij Vaidya <diviv@amazon.com>, Lucas Brutschy <lbrutschy@confluent.io>
These tests are removed in this commit because they are flaky.
After investigation, the causes are:
1. remoteLogSizeComputationTime: It failed with Expected to find 1000 for RemoteLogSizeComputationTime metric value, but found 0. The reason is because if the verification thread is too slow, and the 2nd run of RLMTask started, then it'll reset the value back to 0. Fix it by adding latch to wait for verification.
2. remoteFetchExpiresPerSec: It failed with The ExpiresPerSec value is not incremented. Current value is: 0. The reason is because the remoteFetchExpiresPerSec metric is a static metric. And we remove all metrics after each test completed in tearDown method. So once remoteFetchExpiresPerSec is removed, it won't be created again like other metrics. And that's why it failed sometimes in Jenkins because if there is a previous test have expired remote fetch, then this metric will be created and removed forever. Fix it by only removing it in afterAll.
Reviewers: Kamal Chandraprakash <kamal.chandraprakash@gmail.com>, Satish Duggana <satishd@apache.org>, Christo Lolov <lolovc@amazon.com>
This pull request aims to implement RemoteLogSizeBytes from KIP-963.
Reviewers: Kamal Chandraprakash <kamal.chandraprakash@gmail.com>, Satish Duggana <satishd@apache.org>, Luke Chen <showuon@gmail.com>
People has raised concerned about using `Generic` as a name to designate the old rebalance protocol. We considered using `Legacy` but discarded it because there are still applications, such as Connect, using the old protocol. We settled on using `Classic` for the `Classic Rebalance Protocol`.
The changes in this patch are extremely mechanical. It basically replaces the occurrences of `generic` by `classic`.
Reviewers: Divij Vaidya <diviv@amazon.com>, Lucas Brutschy <lbrutschy@confluent.io>
This patch wires the handling of makers written by the transaction coordinator via the WriteTxnMarkers API. In the old group coordinator, the markers are written to the logs and the group coordinator is informed to materialize the changes as a second step if the writes were successful. This approach does not really work with the new group coordinator for mainly two reasons: 1) The second step would actually fail while the coordinator is loading and there is no guarantee that the loading has picked up the write or not; 2) It does not fit well with the new memory model where the state is snapshotted by offset. In both cases, it seems that having a single writer to the `__consumer_offsets` partitions is more robust and preferable.
Reviewers: Jeff Kim <jeff.kim@confluent.io>, Justine Olshan <jolshan@confluent.io>
The new coordinator stops loading if the partition goes offline during load. However, the partition is still considered active. Instead, we should return NOT_LEADER_OR_FOLLOWER exception during load.
Another change is that we only want to invoke CoordinatorPlayback#updateLastCommittedOffset if the current offset (last written offset) is greater than or equal to the current high watermark. This is to ensure that in the case the high watermark is ahead of the current offset, we don't clear snapshots prematurely.
Reviewers: David Jacot <djacot@confluent.io>
We are not properly closing Closeable resources in the code base at multiple places especially when we have an exception. This code change fixes multiple of these leaks.
Reviewers: Ismael Juma <ismael@juma.me.uk>, Luke Chen <showuon@gmail.com>, Mickael Maison <mickael.maison@gmail.com>
This patch adds the group.coordinator.rebalance.protocols configuration which accepts a list of protocols to enable. At the moment, only generic and consumer are supported and it is not possible to disable generic yet. When consumer is enabled, the new consumer rebalance protocol (KIP-848) is enabled alongside the new group coordinator. This patch also publishes all the new configurations introduced by KIP-848.
Reviewers: Jeff Kim <jeff.kim@confluent.io>, Stanislav Kozlovski <stanislav@confluent.io>
This pull request aims to implement RemoteCopyLagSegments, RemoteDeleteLagBytes and RemoteDeleteLagSegments from KIP-963.
Reviewers: Luke Chen <showuon@gmail.com>, Kamal Chandraprakash <kamal.chandraprakash@gmail.com>
Breakdown of this PR:
* Extend the generator to support deprecated api versions
* Set deprecated api versions via the request json files
* Expose the information via metrics and the request log
The relevant section of the KIP:
> * Introduce metric `kafka.network:type=RequestMetrics,name=DeprecatedRequestsPerSec,request=(api-name),version=(api-version),clientSoftwareName=(client-software-name),clientSoftwareVersion=(client-software-version)`
> * Add boolean field `requestApiVersionDeprecated` to the request
header section of the request log (alongside `requestApiKey` ,
`requestApiVersion`, `requestApiKeyName` , etc.).
Unit tests were added to verify the new generator functionality,
the new metric and the new request log attribute.
Reviewers: Jason Gustafson <jason@confluent.io>
We drive the consumer closing via events, and rely on the still-lived network thread to complete these operations.
This ticket encompasses several different tickets:
KAFKA-15696/KAFKA-15548
When closing the consumer, we need to perform a few tasks. And here is the top level overview:
We want to keep the network thread alive until we are ready to shut down, i.e., no more requests need to be sent out. To achieve so, I implemented a method, signalClose() to signal the managers to prepare for shutdown. Once we signal the network thread to close, the manager will prepare for the request to be sent out on the next event loop. The network thread can then be closed after issuing these events. The application thread's task is pretty straightforward, 1. Tell the background thread to perform n events and 2. Block on certain events until succeed or the timer runs out. Once all requests are sent out, we close the network thread and other components as usual.
Here I outline the changes in detail
AsyncKafkaConsumer: Shutdown procedures, and several utility functions to ensure proper exceptions are thrown during shutdown
AsyncKafkaConsumerTest: I examine each individual test and fix ones that are blocking for too long or logging errors
CommitRequestManager: signalClose()
FetchRequestManagerTest: changes due to change in pollOnClose()
ApplicationEventProcessor: handle CommitOnClose and LeaveGroupOnClose. Latter, it triggers leaveGroup() which should be completed on the next heartbeat (or we time out on the application thread)
Reviewers: Lucas Brutschy <lbrutschy@confluent.io>, Kirk True <ktrue@confluent.io>
Producers and consumers could be leaked in the AuthorizerIntegrationTest. In the teardown logic, `removeAllClientAcls()` is called before calling the super teardown method. If `removeAllClientAcls()` fails, the super method does not have a change to close the producers and consumers. Example of such failure [here](https://ci-builds.apache.org/blue/organizations/jenkins/Kafka%2Fkafka-pr/detail/PR-14925/11/tests/).
As a new cluster is created for each test anyway, calling `removeAllClientAcls()` does not seem necessary. This patch removes it.
Reviewers: Jason Gustafson <jason@confluent.io>
This PR implements part of KIP-963, specifically for adding new metrics.
The metrics added in this PR are:
RemoteDeleteRequestsPerSec (emitted when expired log segments on remote storage being deleted)
RemoteDeleteErrorsPerSec (emitted when failed to delete expired log segments on remote storage)
BuildRemoteLogAuxStateRequestsPerSec (emitted when building remote log aux state for replica fetchers)
BuildRemoteLogAuxStateErrorsPerSec (emitted when failed to build remote log aux state for replica fetchers)
Reviewers: Luke Chen <showuon@gmail.com>, Nikhil Ramakrishnan <ramakrishnan.nikhil@gmail.com>, Christo Lolov <lolovc@amazon.com>, Kamal Chandraprakash <kamal.chandraprakash@gmail.com>, Divij Vaidya <diviv@amazon.com>, Satish Duggana <satishd@apache.org>
This reverts commit ed7ad6d.
We have been seeing a lot of failures of TransactionsWithTieredStoreTest.testTransactionsWithCompression on trunk, and it seems to start with this PR. I see how this PR can influence the test via the change in TestUtils. The bad part is that sometimes seems to kill the Gradle Executors completely. So I'd suggest reverting the change before investigating further to stabilize CI.
Reviewers: Bruno Cadonna <cadonna@apache.org>
I observed several failed tests in PR builds. Let's first disable them and try to find a different way to test the async consumer with these tests.
Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
I ran this test 40 times without KAFKA-15653 with and without compression enabled.
With compression it failed 39/40 times and without it passed 40/40 times.
With the KAFKA-15653 and compression it passed 40/40 times locally
Reviewers: Jason Gustafson <jason@confluent.io>
The consumer integration tests were experimentally disabled for the new `AsyncKafkaConsumer` variant with the aim of improving build stability. Several improvements have been made to the consumer code and other tests which seem to have made a difference. This patch re-enables the tests.
Reviewers: David Jacot <djacot@confluent.io>
Add metric for the number of expired remote fetches per second, and corresponding unit test to verify that the metric is marked on expiration.
kafka.server:type=DelayedRemoteFetchMetrics,name=ExpiresPerSec
Reviewers: Luke Chen <showuon@gmail.com>, Satish Duggana <satishd@apache.org>, Kamal Chandraprakash<kamal.chandraprakash@gmail.com>, Christo Lolov <lolovc@amazon.com>
This patch adds the logic for coordinating the invocation of the `ConsumerRebalanceListener` callback invocations between the background thread (in `MembershipManagerImpl`) and the application thread (`AsyncKafkaConsumer`) and back again. It allowed us to enable more tests from `PlaintextConsumerTest` to exercise the code herein.
Reviewers: David Jacot <djacot@confluent.io>
Moves ELR from MetadataVersion IBP_3_7_IV3 into the new IBP_3_8_IV0 because the ELR feature was not completed before 3.7 reached feature freeze. Leaves IBP_3_7_IV3 empty -- it is a no-op and is not reused for anything. Adds the new MetadataVersion IBP_3_7_IV4 for the FETCH request changes from KIP-951, which were mistakenly never associated with a MetadataVersion. Updates the LATEST_PRODUCTION MetadataVersion to IBP_3_7_IV4 to declare both KRaft JBOD and the KIP-951 changes ready for production use.
Reviewers: Omnia G H Ibrahim <o.g.h.ibrahim@gmail.com>, Ron Dagostino <rdagostino@confluent.io>, Ismael Juma <ismael@juma.me.uk>, José Armando García Sancio <jsancio@apache.org>, Justine Olshan <jolshan@confluent.io>
Rewrote the verification flow to pass a callback to execute after verification completes.
For the TxnOffsetCommit, we will call doTxnCommitOffsets. This allows us to do offset validations post verification.
I've reorganized the verification code and group coordinator code to make these code paths clearer. The followup refactor (https://issues.apache.org/jira/browse/KAFKA-15987) will further clean up the produce verification code.
Reviewers: Artem Livshits <alivshits@confluent.io>, Jason Gustafson <jason@confluent.io>, David Jacot <djacot@confluent.io>, Jun Rao <junrao@gmail.com>
This pull request implements the first in the list of metrics in KIP-963: Additional metrics in Tiered Storage.
Since each partition of a topic will be serviced by its own RLMTask we need an aggregator object for a topic. The aggregator object in this pull request is BrokerTopicAggregatedMetric. Since the RemoteCopyLagBytes is a gauge I have introduced a new GaugeWrapper. The GaugeWrapper is used by the metrics collection system to interact with the BrokerTopicAggregatedMetric. The RemoteLogManager interacts with the BrokerTopicAggregatedMetric directly.
Reviewers: Luke Chen <showuon@gmail.com>, Satish Duggana <satishd@apache.org>, Kamal Chandraprakash<kamal.chandraprakash@gmail.com>
This patch ensure that `offset.commit.timeout.ms` is enforced. It does so by adding a timeout to the CoordinatorWriteEvent.
Reviewers: David Jacot <djacot@confluent.io>
Implement Consumer.listTopics and Consumer.partitionsFor in the new consumer. The topic metadata request manager already existed so this PR adds expiration to requests, removes some redundant state checking and adds tests.
Reviewers: Lucas Brutschy <lucasbru@apache.org>
Currently, poll interval is not being respected during consumer#poll. When the user stops polling the consumer, we should assume either the consumer is too slow to respond or is already dead. In either case, we should let the group coordinator kick the member out of the group and reassign its partition after the rebalance timeout expires.
If the consumer comes back alive, we should send a heartbeat and the member will be fenced and rejoin. (and the partitions will be revoked).
This is the same behavior as the current implementation.
Reviewers: Lucas Brutschy <lucasbru@apache.org>, Bruno Cadonna <cadonna@apache.org>, Lianet Magrans <lianetmr@gmail.com>
Session expiration in ZkClient can lead to a thread leak, and does fail CI on master.
This is happening in testNoDescribeProduceOrConsumeWithoutTopicDescribeAcl, and possibly other tests.
Use try-with-resources to close ZkClient if this happens.
This does not fix the underlying session expiration in ZK.
Reviewers: David Jacot <djacot@confluent.io>
In the new consumer, the commit request manager and the membership manager are separate components. The commit request manager is initialised with group information that it uses to construct `OffsetCommit` requests. However, the initial value of the member ID is `""` in some cases. When the consumer joins the group, it receives a `ConsumerGroupHeartbeat` response which tells it the member ID. The member ID was not being passed to the commit request manager, so it sent invalid `OffsetCommit` requests that failed with `UNKNOWN_MEMBER_ID`.
Reviewers: Bruno Cadonna <cadonna@apache.org>, David Jacot <djacot@confluent.io>
The support for regular expressions has not been implemented yet in the new consumer group protocol. This patch removes the `SubscribedTopicRegex` from the `ConsumerGroupHeartbeatRequest` in preparation for 3.7. It seems better to bump the version and add it back when we implement the feature, as part of https://issues.apache.org/jira/browse/KAFKA-14517, instead of having an unused field in the request.
Reviewers: Sagar Rao <sagarmeansocean@gmail.com>, Justine Olshan <jolshan@confluent.io>
- Add proper start & stop for AssignmentsManager's event loop
- Dedupe queued duplicate assignments
- Fix bug where directory ID is resolved too late
Co-authored-by: Gaurav Narula <gaurav_narula2@apple.com>
Reviewers: Colin P. McCabe <cmccabe@apache.org>
Allow using JBOD during ZK migration if MetadataVersion is at or above 3.7-IV2.
Reviewers: Ron Dagostino <rndgstn@gmail.com>, Colin P. McCabe <cmccabe@apache.org>, Proven Provenzano <pprovenzano@confluent.io>
KAFKA-15361 (#14838) introduced a check for non empty directory list on brokerregistration requests
from MetadataVersion.IBP_3_7_IV2 or later, which enables directory assignment. However, ZK brokers
weren't yet registering yet with a directory list. This patch addresses that. We also make the
directory list non-optional in BrokerLifecycleManager.
Reviewers: Ron Dagostino <rndgstn@gmail.com>, Colin P. McCabe <cmccabe@apache.org>, Proven Provenzano <pprovenzano@confluent.io>
This new integration test verifies that a static member who temporary left the group is removed after the session timeout expires. It also verifies that a new static member with the same instance id can't join the group until the previous static member is expired.
Reviewers: David Jacot <djacot@confluent.io>
DelegationTokenEndToEndAuthorizationWithOwnerTest can leak a thread, causing problems with many tests.
This is due to an admin client that isn't being closed when a (flaky) test fails. Using the Scala util Using to close the auto-closable admin client in case the validation fails.
Reviewers: David Jacot <djacot@confluent.io>, Bruno Cadonna <cadonna@apache.org>
Handle AssignReplicasToDirs requests, persist metadata changes
with new directory assignments and possible leader elections.
Reviewers: Proven Provenzano <pprovenzano@confluent.io>, Ron Dagostino <rndgstn@gmail.com>, Colin P. McCabe <cmccabe@apache.org>
Add the CurrentControllerId metric as described in KIP-1001. This gives us an easy way to identify the current controller by looking at the metrics of any Kafka node (broker or controller).
Reviewers: David Arthur <mumrah@gmail.com>
Improvement for KIP-1000 to list client metrics resources in KafkaApis.scala. Using functionality exposed by KIP-1000 to support describe all metrics operations for KIP-714.
Reviewers: Andrew Schofield <aschofield@confluent.io>, Jun Rao <junrao@gmail.com>
The Kafka consumer makes a variety of requests to brokers such as fetching committed offsets and updating metadata. In the LegacyKafkaConsumer, the approach is typically to prepare RPC requests and then poll the network to wait for responses. In the AsyncKafkaConsumer, the approach is to enqueue an ApplicationEvent for processing by one of the request managers on the background thread. However, it is still important to wait for responses rather than spinning enqueuing events for the request managers before they have had a chance to respond.
In general, the behaviour will not be changed by this code. The PlaintextConsumerTest.testSeek test was flaky because operations such as KafkaConsumer.position were not properly waiting for a response which meant that subsequent operations were being attempted in the wrong state. This test is no longer flaky.
Reviewers: Kirk True <ktrue@confluent.io>, Lianet Magrans <lianetmr@gmail.com>, Bruno Cadonna <cadonna@apache.org>
After the new coordinator loads a __consumer_offsets partition, it logs the following exception when making a read operation (fetch/list groups, etc):
```
java.lang.RuntimeException: No in-memory snapshot for epoch 740745. Snapshot epochs are:
at org.apache.kafka.timeline.SnapshotRegistry.getSnapshot(SnapshotRegistry.java:178)
at org.apache.kafka.timeline.SnapshottableHashTable.snapshottableIterator(SnapshottableHashTable.java:407)
at org.apache.kafka.timeline.TimelineHashMap$ValueIterator.<init>(TimelineHashMap.java:283)
at org.apache.kafka.timeline.TimelineHashMap$Values.iterator(TimelineHashMap.java:271)
```
This happens because we don't have a snapshot at the last updated high watermark after loading. We cannot generate a snapshot at the high watermark after loading all batches because it may contain records that have not yet been committed. We also don't know where the high watermark will advance up to so we need to generate a snapshot for each offset the loader observes to be greater than the current high watermark. Then once we add the high watermark listener and update the high watermark we can delete all of the older snapshots.
Reviewers: David Jacot <djacot@confluent.io>
* Validate the client’s configuration for server-side assignor selection defined in config group.remote.assignor
* Include the assignor taken from config in the ConsumerGroupHeartbeat request, in the ServerAssignor field
* Properly handle UNSUPPORTED_ASSIGNOR errors that may be returned to the HB response if the server does not support the assignor defined by the consumer.
Includes a simple integration tests for sending an invalid assignor to the broker, and for using the range assignor with a single consumer.
Reviewers: David Jacot <djacot@confluent.io>, Lianet Magrans <lianetmr@gmail.com>, Bruno Cadonna <cadonna@apache.org>
DELETE_RECORDS API can move the log-start-offset beyond the highest-copied-remote-offset. In such cases, we should allow deletion of local-log segments since they won't be eligible for upload to remote storage.
Reviewers: Satish Duggana <satishd@apache.org>, Luke Chen <showuon@gmail.com>
This adds the new ListClientMetricsResources RPC to the Kafka protocol and puts support
into the Kafka admin client. The broker-side implementation in this PR is just to return an empty
list. A future PR will obtain the list from the config store.
Includes a few unit tests for what is a very simple RPC. There are additional tests already written and
waiting for the PR that delivers the kafka-client-metrics.sh tool which builds on this PR.
Reviewers: Jun Rao <junrao@gmail.com>
This patch adds an integration test which verifies that a static member gets back its previous assignment back when rejoining.
Reviewers: David Jacot <djacot@confluent.io>
This patch adds the `Uniform` assignor to the default list of supported assignors. It also do small changes in the code.
Reviewers: Justine Olshan <jolshan@confluent.io>
This is a follow-up to https://github.com/apache/kafka/pull/14687 as we found out that some parameterized tests do not include the test method name in their name. For the context, the JUnit XML report does not include the name of the method by default but only rely on the display name provided.
Reviewers: David Arthur <mumrah@gmail.com>
This patch adds a ThreadLocal with a GrowableBufferSupplier so that each writing thread can reuse the same buffer instead of allocating a new one for each write. The patch relies on existing tests.
Reviewers: Jeff Kim <jeff.kim@confluent.io>, Justine Olshan <jolshan@confluent.io>
In this [buid](https://ci-builds.apache.org/blue/organizations/jenkins/Kafka%2Fkafka-pr/detail/PR-14826/11/pipeline/12/), the following test hang forever.
```
Gradle Test Run :core:test > Gradle Test Executor 93 > PlaintextConsumerTest > testSeek(String, String) > testSeek(String, String).quorum=kraft+kip848.groupProtocol=consumer STARTED
```
As the new consumer is not extremely stable yet, we should add a Timeout to all those integration tests to ensure that builds are not blocked unnecessarily.
Reviewers: Andrew Schofield <aschofield@confluent.io>, Justine Olshan <jolshan@confluent.io>
The PR adds handling of telemetry APIs in KafkaAPIs.scala which calls the respective manager to handle the API calls. Also the telemetry plugin which if registered in configs get registered for exporting client metrics.
Reviewers: Jun Rao <junrao@gmail.com>
This is part of the investigation into recent build instability. It simply turns off the consumer integration tests that use the new AsyncKafkaConsumer to see whether the build runs smoothly.
Reviewers: David Jacot <djacot@confluent.io>
Verifies that the group ID passed into the async consumer is valid. That is, if the group ID is not null, it is not empty or it does not consist of only whitespaces.
This change stores the group ID in the group metadata because KAFKA-15281 about the group metadata API will build on that.
Reviewers: Lucas Brutschy <lbrutschy@confluent.io>, Kirk True <ktrue@confluent.io>
Test startup does not assure that all brokers are registered. In flaky failures,
the `DescribeCluster` API does not return a complete list of brokers. To fix
the issue, we add a call to `ensureConsistentKRaftMetadata()` to ensure that all
brokers are registered and have caught up to current metadata.
Reviewers: David Jacot <djacot@confluent.io>
This PR fixes some details of the interface to KafkaConsumer.committed which were different between the existing consumer and the new consumer.
Adds a unit test that validates the behaviour is the same for both consumer implementations.
Reviewers: Kirk True <ktrue@confluent.io>, Bruno Cadonna <cadonna@apache.org>
`FetchFromFollowerIntegrationTest.testRackAwareRangeAssignor` is extremely flaky and we have never been able to fix it. This patch disables it until we find a solution to make it reliable with https://issues.apache.org/jira/browse/KAFKA-15020.
Reviewers: Stanislav Kozlovski <stanislav@confluent.io>
When creating partition registrations directories must always be defined.
If creating a partition from a PartitionRecord or PartitionChangeRecord from an older version that
does not support directory assignments, then DirectoryId.MIGRATING is assumed.
If creating a new partition, or triggering a change in assignment, DirectoryId.UNASSIGNED should be
specified, unless the target broker has a single online directory registered, in which case the
replica should be assigned directly to that single directory.
Reviewers: Colin P. McCabe <cmccabe@apache.org>
Assign MetadataVersion.IBP_3_7_IV2 to JBOD.
Move KIP-966 support to MetadataVersion.IBP_3_7_IV3.
Create MetadataVersion.LATEST_PRODUCTION as the latest metadata version that can be used when formatting a
new cluster, or upgrading a cluster using kafka-features.sh. This will allow us to clearly distinguish between stable
and unstable metadata versions for the first time.
Reviewers: Igor Soarez <soarez@apple.com>, Ron Dagostino <rndgstn@gmail.com>, Calvin Liu <caliu@confluent.io>, Proven Provenzano <pprovenzano@confluent.io>
With `AbstractResponse.maybeSetThrottleTimeMs`, we don't need to use a callback to build the response with the respective throttle.
Reviewers: David Jacot <djacot@confluent.io>
While any blocking operation under holding the UnifiedLog.lock could lead to serious performance (even availability) issues, currently there are several paths that calls fsync(2) inside the lock
In the meantime the lock is held, all subsequent produces against the partition may block
This easily causes all request-handlers to be busy on bad disk performance
Even worse, when a disk experiences tens of seconds of glitch (it's not rare in spinning drives), it makes the broker to unable to process any requests with unfenced from the cluster (i.e. "zombie" like status)
This PR gets rid of 4 cases of essentially-unnecessary fsync(2) calls performed under the lock:
(1) ProducerStateManager.takeSnapshot at UnifiedLog.roll
I moved fsync(2) call to the scheduler thread as part of existing "flush-log" job (before incrementing recovery point)
Since it's still ensured that the snapshot is flushed before incrementing recovery point, this change shouldn't cause any problem
(2) ProducerStateManager.removeAndMarkSnapshotForDeletion as part of log segment deletion
This method calls Utils.atomicMoveWithFallback with needFlushParentDir = true internally, which calls fsync.
I changed it to call Utils.atomicMoveWithFallback with needFlushParentDir = false (which is consistent behavior with index files deletion. index files deletion also doesn't flush parent dir)
This change shouldn't cause problems neither.
(3) LeaderEpochFileCache.truncateFromStart when incrementing log-start-offset
This path is called from deleteRecords on request-handler threads.
Here, we don't need fsync(2) either actually.
On unclean shutdown, few leader epochs might be remained in the file but it will be handled by LogLoader on start-up so not a problem
(4) LeaderEpochFileCache.truncateFromEnd as part of log truncation
Likewise, we don't need fsync(2) here, since any epochs which are untruncated on unclean shutdown will be handled on log loading procedure
Reviewers: Luke Chen <showuon@gmail.com>, Divij Vaidya <diviv@amazon.com>, Justine Olshan <jolshan@confluent.io>, Jun Rao <junrao@gmail.com>
The PR provide implementation for client metrics manager along with other classes. Manager is responsible to support 3 operations:
UpdateSubscription - From kafka-configs.sh and reload from metadata cache.
Process Get Telemetry Request - From KafkaApis.scala
Process Push Telemetry Request - From KafkaApis.scala
Manager maintains an in-memory cache to keep track of client instances against their instance id.
Reviewers: Andrew Schofield <aschofield@confluent.io>, Jun Rao <junrao@gmail.com>
This patch adds support for transactional writes to the CoordinatorRuntime framework. This mainly consists in adding CoordinatorRuntime#scheduleTransactionalWriteOperation and in adding the producerId and producerEpoch to various interfaces. The patch also extends the CoordinatorLoaderImpl and the CoordinatorPartitionWriter accordingly.
Reviewers: Justine Olshan <jolshan@confluent.io>
Only add directory.id to meta.properties when migrating to kraft mode, or already in
kraft mode. This prevents incompatibilities with older Kafka releases, which checked
that each directory in a JBOD ensemble had the same meta.properties values.
Reviewers: Colin P. McCabe <cmccabe@apache.org>
This commit parameterizes the consumer integration tests so they can be run against
the existing "generic" group protocol and the new "consumer" group protocol
introduced in KIP-848.
The KIP-848 client code is under construction so some of the tests do not run on
both variants to start with, but the idea is that the tests can be enabled as the gaps
in functionality are closed.
Reviewers: Lucas Brutschy <lbrutschy@confluent.io>, Kirk True <ktrue@confluent.io>
As described in KAFKA-9470, testBlockOnRequestCompletionFromStateChangeHandler
will block for hours occasionally.
If it passes, it takes 0.5 seconds, so a minute timeout should be safe.
This is not a fix for KAFKA-9470, it's just aiming to make the CI more stable.
Reviewers: David Jacot <djacot@confluent.io>, Matthias J. Sax <matthias@confluent.io>
The PR adds support of alter/describe configs for client-metrics as defined in KIP-714
Reviewers: Andrew Schofield <aschofield@confluent.io>, Jun Rao <junrao@gmail.com>
Roll the active segment and offload it to remote storage once it breaches the retention time policy.
A segment is eligible for deletion once it gets uploaded to the remote storage. We have checks to allow only the passive segments to be uploaded, so the active segment never gets removed at all even if breaches the retention time. For low-throughput/stale topics, the active segment can hold the data beyond the configured retention time by the user.
Reviewers: Satish Duggana <satishd@apache.org>, Christo Lolov <lolovc@amazon.com>
The other call sites for KafkaRaftManager (SharedServer, TestRaftServer, MetadataShell) appear to shutdown
the KafkaRaftManager when shutting down themselves. The call-site in ZK-mode KafkaServer should behave
the same way.
Reviewers: Colin P. McCabe <cmccabe@apache.org>
"findHighestRemoteOffset" does not take into account the leader-epoch end offset. This can cause log divergence between the local and remote log segments when there is unclean leader election.
To handle it correctly, the logic to find the highest remote offset can be updated to:
find-highest-remote-offset = min(end-offset-for-epoch-in-the-checkpoint, highest-remote-offset-for-epoch)
Discussion thread: https://github.com/apache/kafka/pull/14004#discussion_r1266864272
Reviewers: Satish Duggana <satishd@apache.org>, Christo Lolov <lolovc@amazon.com>
This PR implements KIP-978: Allow dynamic reloading of certificates with different DN / SANs. It adds two new options ssl.allow.dn.changes and ssl.allow.san.changes that can be used to enable dynamic reloading of certificates even if their DN / SANs change. They both default to false to maintain the current behavior by default.
Reviewers: Mickael Maison <mimaison@apache.org>, Viktor Somogyi-Vass <viktorsomogyi@gmail.com>
This patch updates the testing framework to support running tests with kraft and the new group coordinator introduced in the context of KIP-848. This can be done by using `kraft+kip-848` as a quorum. Note that this is temporary until we make it the default and only option in 4.0. To verify this, this patch also enables kraft and kraft+kip-848 in PlaintextConsumerTest and its parent classes.
Reviewers: David Jacot <djacot@confluent.io>
The metadata cache now considers registered log directories
and directory assignments when determining offline replicas.
Reviewers: Colin P. McCabe <cmccabe@apache.org>, Proven Provenzano <pprovenzano@confluent.io>
We plan to ship an early access of KIP-848 in AK 3.7. Therefore, we need to mark the ConsumerGroupHeartbeat API (v1), OffsetCommit API (v9) and OffsetFetch API (v9) as stable.
Reviewers: Andrew Schofield <aschofield@confluent.io>, Justine Olshan <jolshan@confluent.io>
This patch copy over existing metrics and add new consumer group metrics to the new GroupCoordinatorService.
Now that each coordinator is responsible for a topic partition, this patch introduces a GroupCoordinatorMetrics that records gauges for global metrics such as the number of generic groups in PreparingRebalance state, etc. For GroupCoordinatorShard specific metrics, GroupCoordinatorMetrics will activate new GroupCoordinatorMetricsShards that will be responsible for incrementing/decrementing TimelineLong objects and then aggregate the total amount across all shards.
As the CoordinatorRuntime/CoordinatorShard does not care about group metadata, we have introduced a CoordinatorMetrics.java/CoordinatorMetricsShard.java so that in the future transaction coordinator metrics can also be onboarded in a similar fashion.
Main files to look at:
GroupCoordinatorMetrics.java
GroupCoordinatorMetricsShard.java
CoordinatorMetrics.java
CoordinatorMetricsShard.java
CoordinatorRuntime.java
Metrics to add after #14408 is merged:
offset deletions sensor (OffsetDeletions); Meter(offset-deletion-rate, offset-deletion-count)
Metrics to add after https://issues.apache.org/jira/browse/KAFKA-14987 is merged:
offset expired sensor (OffsetExpired); Meter(offset-expiration-rate, offset-expiration-count)
Reviewers: Justine Olshan <jolshan@confluent.io>
Instead of only sending failed log directory UUIDs in the heartbeat
request until a successful response is received, the broker sends
the full cumulative set of failed directories since startup time.
This aims to simplify the handling of log directory failure in the
controller side, considering overload mode handling of heartbeat
requests, which returns an undifferentiated reply.
Reviewers: Colin P. McCabe <cmccabe@apache.org>, Proven Provenzano <pprovenzano@confluent.io>
This patch contains a few small clean-ups in LogValidator and associated classes:
1. Set shallowOffsetOfMaxTimestamp consistently as the last offset in the
batch for v2 compressed and non-compressed data.
2. Rename `RecordConversionStats` to `RecordValidationStats` since one of its
fields `temporaryMemoryBytes` does not depend on conversion.
3. Rename `batchIndex` in `recordIndex` in loops over the records in each batch
inside `LogValidator`.
Reviewers: Qichao Chu <5326144+ex172000@users.noreply.github.com>, Jun Rao <junrao@gmail.com>
The commit covers a few important points:
- Exception handling: We should be thrown RetriableCommitException when the commit exception is retriable. We should throw FencedIdException on commit and poll similar to the current implementation. Other errors should be thrown as it is.
- Callback invocation: The callbacks need to be invoked on the main/application thread; however, the future is completed in the background thread. To achieve this, I created an Invoker class with a queue, so that this callback can be invoked during the consumer.poll()
Note: One change I made is to remove the DefaultOffsetCommit callback. Since the callback is purely for logging, I think it is reasonable for us to move the logging to the background thread instead of relying on the application thread to trigger the logging.
Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
We only move Java classes that have minimal or no dependencies on Scala classes in this PR.
Details:
* Configured `server` module in build files.
* Changed `ControllerRequestCompletionHandler` to be an interface since it has no implementations.
* Cleaned up various import control files.
* Minor build clean-ups for `server-common`.
* Disabled `testAssignmentAggregation` when executed with Java 8, this is an existing issue (see #14794).
For broader context on this change, please check:
* KAFKA-15852: Move server code from `core` to `server` module
Reviewers: Divij Vaidya <diviv@amazon.com>
https://github.com/apache/kafka/pull/14369 introduced a compilation error in ReplicaManagerConcurrencyTest for Scala 2.12.
Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
A new AssignmentsManager accumulates, batches, and sends KIP-858
assignment events to the Controller. Assignments are sent via
AssignReplicasToDirs requests.
Move QuorumTestHarness.formatDirectories into TestUtils so it can be
used in other test contexts.
Fix a bug in ControllerRegistration.java where the wrong version of the
record was being generated in ControllerRegistration.toRecord.
Reviewers: Colin P. McCabe <cmccabe@apache.org>, Proven Provenzano <pprovenzano@confluent.io>, Omnia G H Ibrahim <o.g.h.ibrahim@gmail.com>
This KIP-951 commit was reverted to investigate the org.apache.kafka.tiered.storage.integration.ReassignReplicaShrinkTest test failure (#14738).
A fix for that was merged in #14757, hence unreverting this change.
This reverts commit a98bd7d.
Reviewers: Justine Olshan <jolshan@confluent.io>, Mayank Shekhar Narula <mayanks.narula@gmail.com>
This patch adds the concept of a "Full" UpdateMetadataRequest, similar to what is used in
LeaderAndIsr. A new tagged field is added to UpdateMetadataRequest at version 8 which allows the
KRaft controller to indicate if a UMR contains all the metadata or not. Since UMR is implicitly
treated as incremental by the ZK broker, we needed a way to detect topic deletions when the KRaft
broker sends a metadata snapshot to the ZK broker. By sending a "Full" flag, the broker can now
compare existing topic IDs to incoming topic IDs and calculate which topics should be removed from
the MetadataCache.
This patch only removes deleted topics from the MetadataCache. Partition/log management was
implemented in KAFKA-15605.
Reviewers: Colin P. McCabe <cmccabe@apache.org>
The consumer refactoring project introduced another `Consumer` implementation, creating two different, coexisting implementations of the `Consumer` interface:
* `KafkaConsumer` (AKA "existing", "legacy" consumer)
* `PrototypeAsyncConsumer` (AKA "new", "refactored" consumer)
The goal of this task is to refactor the code via the delegation pattern so that we can keep a top-level `KafkaConsumer` but then delegate to another implementation under the covers. There will be two delegates at first:
* `LegacyKafkaConsumer`
* `AsyncKafkaConsumer`
`LegacyKafkaConsumer` is essentially a renamed `KafkaConsumer`. That implementation handles the existing group protocol. `AsyncKafkaConsumer` is renamed from `PrototypeAsyncConsumer` and will implement the new consumer group protocol from KIP-848. Both of those implementations will live in the `internals` sub-package to discourage their use.
This task is part of the work to implement support for the new KIP-848 consumer group protocol.
Reviewers: Philip Nee <pnee@confluent.io>, Andrew Schofield <aschofield@confluent.io>, David Jacot <djacot@confluent.io>
In 91fa196, I accidentally removed the action queue paramater that was added in 7d147cf. I also renamed the actionQueue as to not confuse this in the future.
I don't think this broke anything since we don't use verification for group coordinator commits, but I should fix it to be as it was before.
Reviewers: Artem Livshits <alivshits@confluent.io>, Jason Gustafson <jason@confluent.io>
This reverts commit f38b0d8.
Trying to find the root cause of org.apache.kafka.tiered.storage.integration.ReassignReplicaShrinkTest failing in CI.
Reviewers: Justine Olshan <jolshan@confluent.io>
This patch fixes a compilation error introduced by https://github.com/apache/kafka/pull/14392 for Scala 2.12.
```
> Task :core:compileScala
[Error] /home/jenkins/workspace/Kafka_kafka-pr_PR-14392/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala:305:49: value incl is not a member of scala.collection.immutable.Set[org.apache.kafka.common.Uuid]
```
Reviewers: Luke Chen <showuon@gmail.com>
BrokerLifecycleManager should send the offline log directories in the BrokerHeartbeatRequests it
sends. Also, when handling BrokerHeartbeatResponses, do so by enqueing a BrokerLifecycleManager
event, rather than trying to do the handling directly in the callback.
Reviewers: Colin P. McCabe <cmccabe@apache.org>, Proven Provenzano <pprovenzano@confluent.io>
meta.properties files are used by Kafka to identify log directories within the filesystem.
Previously, the code for handling them was in BrokerMetadataCheckpoint.scala. This PR rewrites the
code for handling them as Java and moves it to the apache.kafka.metadata.properties namespace. It
also gets rid of the separate types for v0 and v1 meta.properties objects. Having separate types
wasn't so bad back when we had a strict rule that zk clusters used v0 and kraft clusters used v1.
But ZK migration has blurred the lines. Now, a zk cluster may have either v0 or v1, if it is
migrating, and a kraft cluster may have either v0 or v1, at any time.
The new code distinguishes between an individual meta.properties file, which is represented by
MetaProperties, and a collection of meta.properties files, which is represented by
MetaPropertiesEnsemble. It is useful to have this distinction, because in JBOD mode, even if some
log directories are inaccessible, we can still use the ensemble to extract needed information like
the cluster ID. (Of course, even when not in JBOD mode, KRaft servers have always been able to
configure a metadata log directory separate from the main log directory.)
Since we recently added a unique directory.id to each meta.properties file, the previous convention
of passing a "canonical" MetaProperties object for the cluster around to various places in the code
needs to be revisited. After all, we can no longer assume all of the meta.properties files are the
same. This PR fixes these parts of the code. For example, it fixes the constructors of
ControllerApis and RaftManager to just take a cluster ID, rather than a MetaProperties object. It
fixes some other parts of the code, like the constructor of SharedServer, to take a
MetaPropertiesEnsemble object.
Another goal of this PR was to centralize meta.properties validation a bit more and make it
unit-testable. For this purpose, the PR adds MetaPropertiesEnsemble.verify, and a few other
verification methods. These enforce invariants like "the metadata directory must be readable," and
so on.
Reviewers: Igor Soarez <soarez@apple.com>, David Arthur <mumrah@gmail.com>, Divij Vaidya <diviv@amazon.com>, Proven Provenzano <pprovenzano@confluent.io>
With the new callback mechanism we were accidentally passing context with the wrong request local. Now include a RequestLocal as an explicit argument to the callback.
Also make the arguments passed through the callback clearer by separating the method out.
Added a test to ensure we use the request handler's request local and not the one passed in when the callback is executed via the request handler.
Reviewers: Ismael Juma <ismael@juma.me.uk>, Divij Vaidya <diviv@amazon.com>, David Jacot <djacot@confluent.io>, Jason Gustafson <jason@confluent.io>, Artem Livshits <alivshits@confluent.io>, Jun Rao <junrao@gmail.com>,
Only advance the HWM for a partition if the ISR set is equal to or above the min ISR config. This patch also sets an upper bound on the min ISR config so it cannot exceed the number of replicas.
Reviewers: David Arthur <mumrah@gmail.com>
TestUtils.createTopicWithAdmin calls waitForAllPartitionsMetadata which waits for partition(s) to be present in each brokers' metadata cache. This is a sufficient check in ZK mode because the controller sends an LISR request before sending an UpdateMetadataRequest which means that the partition in the ReplicaManager will be updated before the metadata cache.
In KRaft mode, the metadata cache is updated first, so the check may return before partitions and other metadata listeners are fully initialized.
Testing:
Insert a Thread.sleep(100) in BrokerMetadataPublisher.onMetadataUpdate after
// Publish the new metadata image to the metadata cache.
metadataCache.setImage(newImage)
and run EdgeCaseRequestTest.testProduceRequestWithNullClientId and the test will fail locally nearly deterministically. After the change(s), the test no longer fails.
Reviewers: Justine Olshan <jolshan@confluent.io>
This PR adds resources to store and handle client metrics needed for KIP-714.
Changes include:
Adding CLIENT_METRICS to resource type
Corresponding DYNAMIC client configurations in resources.
Changes to support dynamic loading of configuration on changes.
Changes to support API calls to fetch data stored against the new resource.
Test cases for the changes.
Reviewers: Andrew Schofield <andrew_schofield@uk.ibm.com>, Philip Nee <pnee@confluent.io>, Jun Rao <junrao@gmail.com>
Since we have added directory.id to MetaProperties, it is no longer safe
to assume that all directories on a node contain the same MetaProperties.
Therefore, we should get rid of places where we are using a single
MetaProperties object to represent the settings of an entire cluster.
This PR removes a few such cases. In each case, it is sufficient just to
pass cluster ID.
The second part of this change refactors KafkaClusterTestKit so that we
convert paths to absolute before creating BrokerNode and ControllerNode
objects, rather than after. This prepares the way for storing an
ensemble of MetaProperties objects in BrokerNode and ControllerNode,
which we will do in a follow-up change.
Reviewers: Ron Dagostino <rndgstn@gmail.com>
Reviewers: Christo Lolov <lolovc@amazon.com>, Colin P. McCabe <cmccabe@apache.org>, Proven Provenzano <pprovenzano@confluent.io>, Ron Dagostino <rdagostino@confluent.io>
This patch adds integration tests for four group coordinator APIs with new group coordinator and new protocol, new group coordinator and old protocol, and old group coordinator and old protocols for the following APIs:
- DescribeGroups
- DeleteGroups
- OffsetDelete
- ListGroups
Reviewers: Ritika Reddy <rreddy@confluent.io>, David Jacot <djacot@confluent.io>
This PR moves PartitionMetadataFile to the storage module.
Existing unit tests in UnifiedLogTest like testLogFlushesPartitionMetadataOnAppend should suffice.
Reviewers: Ismael Juma <ismael@juma.me.uk>, Jun Rao <junrao@gmail.com>
This PR contains changes required to move PartitionReassignmentState class to java code.
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Justine Olshan <jolshan@confluent.io>, Federico Valeri <fedevaleri@gmail.com>, Taras Ledkov Taras Ledkov <tledkov@apache.org>, Kamal Chandraprakash<kamal.chandraprakash@gmail.com>,
In `KafkaApis.scala`, we build the API response differently if exceptions are thrown during the API execution. Since the new group coordinator only populates the response with error code instead of throwing an exception when an error occurs, there may be different behavior between the existing group coordinator and the new one.
This patch:
- Fixes the response building in `KafkaApis.scala` for the two APIs affected by such difference -- OffsetFetch and OffsetDelete.
- In `GroupCoordinatorService.java`, returns a response with error code instead of a failed future when the coordinator is not active.
Reviewers: David Jacot <djacot@confluent.io>
Straightforward refactoring to extract an inner class and methods related to `ConsumerRebalanceListener` for reuse in the KIP-848 implementation of the consumer group protocol. Also using `Optional` to explicitly mark when a `ConsumerRebalanceListener` is in use or not, allowing us to make some (forthcoming) optimizations when there is no listener to invoke.
Reviewers: David Jacot <djacot@confluent.io>
After a late discussion in the voting thread for KIP-858 we
decided to improve the names for the designated reserved
log directory UUID values.
Reviewers: Christo Lolov <lolovc@amazon.com>, Ismael Juma <ismael@juma.me.uk>, Ziming Deng <dengziming1993@gmail.com>.
This field was missed by the initial KIP-919 PR(s). The result is that migrations can't begin since
the controllers will never become ready. This patch fixes that as well as pulls over some fixes
from the 3.6 branch.
Reviewers: Colin P. McCabe <cmccabe@apache.org>
This patch adds reconciliation logic to migrating ZK brokers to deal with pending topic deletions as well as missed StopReplicas.
During the hybrid mode of the ZK migration, the KRaft controller is asynchronously sending UMR and LISR to the ZK brokers to propagate metadata. Since this process is essentially "best effort" it is possible for a broker to miss a StopReplicas. The new logic lets the ZK broker examine its local logs compared with the full set of replicas in a "Full" LISR. Any local logs which are not present in the set of replicas in the request are removed from ReplicaManager and marked as "stray".
To avoid inadvertent data loss with this new behavior, the brokers do not delete the "stray" partitions. They will rename the directories and log warning messages during log recovery. It will be up to the operator to manually delete the stray partitions. We can possibly enhance this in the future to clean up old stray logs.
This patch makes use of the previously unused Type field on LeaderAndIsrRequest. This was added as part of KIP-516 but never implemented. Since its introduction, an implicit 0 was sent in all LISR. The KRaft controller will now send a value of 2 to indicate a full LISR (as specified by the KIP). The presence of this value acts as a trigger for the ZK broker to perform the log reconciliation.
Reviewers: Colin P. McCabe <cmccabe@apache.org>
ConsumerGroupCommand contains code duplications for table row format.
This PR reduces code duplication and make it more clear and easy to understand.
Reviewers: Luke Chen <showuon@gmail.com>, hudeqi <1217150961@qq.com>
RemoteIndexCache has a concurrency bug which leads to IOException while fetching data from remote tier.
The bug could be reproduced as per the following order of events:-
Thread 1 (cache thread): invalidates the entry, removalListener is invoked async, so the files have not been renamed to "deleted" suffix yet.
Thread 2: (fetch thread): tries to find entry in cache, doesn't find it because it has been removed by 1, fetches the entry from S3, writes it to existing file (using replace existing)
Thread 1: async removalListener is invoked, acquires a lock on old entry (which has been removed from cache), it renames the file to "deleted" and starts deleting it
Thread 2: Tries to create in-memory/mmapped index, but doesn't find the file and hence, creates a new file of size 2GB in AbstractIndex constructor. JVM returns an error as it won't allow creation of 2GB random access file.
This commit fixes the bug by using EvictionListener instead of RemovalListener to perform the eviction atomically with the file rename. It handles the manual removal (not handled by EvictionListener) by using computeIfAbsent() and enforcing atomic cache removal & file rename.
Reviewers: Luke Chen <showuon@gmail.com>, Divij Vaidya <diviv@amazon.com>, Arpit Goyal
<goyal.arpit.91@gmail.com>, Kamal Chandraprakash <kamal.chandraprakash@gmail.com>
Fix test FetchRequestTest.testLastFetchedEpochValidation for KRaft mode
The test fails due to unexpected error (OFFSET_OUT_OF_RANGE) when enabled with KRaft mode.
The reason it takes longer to set the leader epoch in KRaft mode is because of the way the topic partitions are created differently than Zookeeper. In Zookeeper mode, we create the topic partitions directly with Zookeeper therefore seem to take less time to create the logs and set leader epoch on broker. In KRaft mode, we use Admin client to create topic partitions. Even though the test waits for topic partitions to get created and appear in metadata cache, it doesn’t seem to be sufficient time for leader epoch to get set on the brokers.
Reviewers: Luke Chen <showuon@gmail.com>, dengziming <dengziming1993@gmail.com>
I've added a new class with an incrementing atomic long to represent the verification guard. Upon creation of verification guard, we will increment this value and assign it to the guard.
The expected behavior is the same as the object guard, but with better debuggability with the string value and type safety (I found a type safety issue in the current code when implementing this)
Reviewers: Ismael Juma <ismael@juma.me.uk>, Artem Livshits <alivshits@confluent.io>
In ConsumerGroupCommand, there are two methods: getLogEndOffsets and getLogStartOffsets, the first parameter groupId is not used, so remove it.
Reviewers: Luke Chen <showuon@gmail.com>
Fixed some of the failing tests in FetchRequestTest.
testFetchWithPartitionsWithIdError and testCreateIncrementalFetchWithPartitionsInErrorV12 fail with the following error when enabled with KRaft mode. These tests only fail sometimes when running locally but consistently failed when running in the Jenkins Pipeline.
Tests will call the utility function TestUtils.waitUntilLeaderIsKnown after creating the topic partitions so that they wait for the logs to be created on the leader before sending fetch requests.
Enabled all tests except checkLastFetchedEpochValidation with KRaft mode.
Looking at the build history in Jenkins, all the other tests except these 2 tests and checkLastFetchedEpochValidation were passing when they were enabled with KRaft mode. Therefore enabled them with KRaft mode again but left checkLastFetchedEpochValidation to be investigated further.
Reviewers: Luke Chen <showuon@gmail.com>, dengziming <dengziming1993@gmail.com>
This patch introduces preliminary changes for Eligible Leader Replicas (KIP-966)
* New MetadataVersion 16 (3.7-IV1)
* New record versions for PartitionRecord and PartitionChangeRecord
* New tagged fields on PartitionRecord and PartitionChangeRecord
* New static config "eligible.leader.replicas.enable" to gate the whole feature
Reviewers: Artem Livshits <alivshits@confluent.io>, David Arthur <mumrah@gmail.com>, Colin P. McCabe <cmccabe@apache.org>
The PR includes:
* Added a new class of CleanShutdownFile which helps write and read from a clean shutdown file.
* Updated the BrokerRegistration API.
* Client side handling for the broker epoch.
* Minimum work on the controller side.
Reviewers: Jun Rao <junrao@gmail.com>
Implements the following metrics:
kafka.server:type=group-coordinator-metrics,name=num-partitions,state=loading
kafka.server:type=group-coordinator-metrics,name=num-partitions,state=active
kafka.server:type=group-coordinator-metrics,name=num-partitions,state=failed
kafka.server:type=group-coordinator-metrics,name=event-queue-size
kafka.server:type=group-coordinator-metrics,name=partition-load-time-max
kafka.server:type=group-coordinator-metrics,name=partition-load-time-avg
kafka.server:type=group-coordinator-metrics,name=thread-idle-ratio-min
kafka.server:type=group-coordinator-metrics,name=thread-idle-ratio-avg
The PR makes these metrics generic so that in the future the transaction coordinator runtime can implement the same metrics in a similar fashion.
Also, CoordinatorLoaderImpl#load will now return LoadSummary which encapsulates the start time, end time, number of records/bytes.
Co-authored-by: David Jacot <djacot@confluent.io>
Reviewers: Ritika Reddy <rreddy@confluent.io>, Calvin Liu <caliu@confluent.io>, David Jacot <djacot@confluent.io>, Justine Olshan <jolshan@confluent.io>
This is now possible since `InterBrokerSend` was moved from `core` to `server-common`.
Also rewrite/move `KafkaNetworkChannelTest`.
The scala version of `KafkaNetworkChannelTest` passed with the changes here (before I
deleted it).
Reviewers: Justine Olshan <jolshan@confluent.io>, José Armando García Sancio <jsancio@users.noreply.github.com>
Do not return fenced brokers from metadataCache.getPartitionReplicaEndpoints, since that could lead to
them getting used as preferred read replicas.
Reviewers: Colin P. McCabe <cmccabe@apache.org>
A few notes:
* Delete a few methods from `UnifiedLog` that were simply invoking the related method in `LogFileUtils`
* Fix `CoreUtils.swallow` to use the passed in `logging`
* Fix `LogCleanerParameterizedIntegrationTest` to close `log` before reopening
* Minor tweaks in `LogSegment` for readability
For broader context on this change, please check:
* KAFKA-14470: Move log layer to storage module
Reviewers: Divij Vaidya <diviv@amazon.com>, Satish Duggana <satishd@apache.org>
Spotbugs was temporarily disabled as part of KAFKA-15485 to support Kafka build with JDK 21. This PR upgrades the spotbugs version to 4.8.0 which adds support for JDK 21 and enables it's usage on build again.
Reviewers: Divij Vaidya <diviv@amazon.com>