Fixing the build failure caused by the earlier commit 27ea025e33
```
[Error] /Users/satishd/repos/apache-kafka/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala:3526:77: the result type of an implicit conversion must be more specific than Object
[Error] /Users/satishd/repos/apache-kafka/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala:3530:70: the result type of an implicit conversion must be more specific than Object
[Warn] /Users/satishd/repos/apache-kafka/core/src/test/scala/unit/kafka/server/ServerGenerateBrokerIdTest.scala:23:21: imported `QuorumTestHarness` is permanently hidden by definition of object QuorumTestHarness in package server
[Warn] /Users/satishd/repos/apache-kafka/core/src/test/scala/unit/kafka/server/ServerGenerateClusterIdTest.scala:29:21: imported `QuorumTestHarness` is permanently hidden by definition of object QuorumTestHarness in package server
[Error] /Users/satishd/repos/apache-kafka/core/src/test/scala/unit/kafka/utils/TestUtils.scala:1438:15: ambiguous reference to overloaded definition,
both method doReturn in class Mockito of type (x$1: Any, x$2: Object*)org.mockito.stubbing.Stubber
and method doReturn in class Mockito of type (x$1: Any)org.mockito.stubbing.Stubber
match argument types (kafka.log.UnifiedLog)
```
Reviewers: Luke Chen <showuon@gmail.com>
Log4j based loggers use `org.apache.logging.log4j.spi.AbstractLoggerAdapter::getContext`
which invokes StackLocatorUtil to walk the stacktrace. This operation is quite CPU intensive
and is performed each time during instantiation.
To avoid walking the stack often, this change uses a static variable to initialize the logger
for a few classes which seem to be instantiated frequently.
Reviewers: Divij Vaidya <diviv@amazon.com>, Ismael Juma <ismael@juma.me.uk>
This patch implements the existing JoinGroup protocol within the new group coordinator.
Some notable differences:
* Methods return a CoordinatorResult to the runtime framework, which includes records to append to the log as well as a future to complete after the append succeeds/fails.
* The coordinator runtime ensures that only a single thread will be processing a group at any given time, therefore there is no more locking on groups.
* Instead of using on purgatories, we rely on the Timer interface to schedule/cancel delayed operations.
Reviewers: David Jacot <djacot@confluent.io>
* KAFKA-14953: Adding RemoteLogManager metrics
In this PR, I have added the following metrics that are related to tiered storage mentioned in[ KIP-405](https://cwiki.apache.org/confluence/display/KAFKA/KIP-405%3A+Kafka+Tiered+Storage).
|Metric|Description|
|-----------------------------------------|--------------------------------------------------------------|
| RemoteReadRequestsPerSec | Number of remote storage read requests per second |
| RemoteWriteRequestsPerSec | Number of remote storage write requests per second |
| RemoteBytesInPerSec | Number of bytes read from remote storage per second |
| RemoteReadErrorsPerSec | Number of remote storage read errors per second |
| RemoteBytesOutPerSec | Number of bytes copied to remote storage per second |
| RemoteWriteErrorsPerSec | Number of remote storage write errors per second |
| RemoteLogReaderTaskQueueSize | Number of remote storage read tasks pending for execution. |
| RemoteLogReaderAvgIdlePercent | Average idle percent of the remote storage reader thread pool|
| RemoteLogManagerTasksAvgIdlePercent | Average idle percent of RemoteLogManager thread pool |
Added unit tests for all the rate metrics.
Reviewers: Luke Chen <showuon@gmail.com>, Divij Vaidya <diviv@amazon.com>, Kamal Chandraprakash<kamal.chandraprakash@gmail.com>, Jorge Esteban Quilcate Otoya <quilcate.jorge@gmail.com>, Staniel Yao<yaolixinylx@gmail.com>, hudeqi<1217150961@qq.com>, Satish Duggana <satishd@apache.org>
Introduced extra mapping to track verification state.
When verifying, there is a race condition that the add partitions verification response returns that the partition is in the ongoing transaction, but an abort marker is written before we get to append. Therefore, we track any given transaction we are verifying with an object unique to that transaction.
We check this unique state upon the first append to the log. After that, we can rely on currentTransactionFirstOffset. We remove the verification state on appending to the log with a transactional data record or marker.
We will also clean up lingering verification state entries via the producer state entry expiration mechanism. We do not update the the timestamp on retrying a verification for a transaction, so each entry must be verified before producer.id.expiration.ms.
There were a few other fixes:
- Moved the transaction manager handling for failed batch into the future completed exceptionally block to avoid processing it twice (this caused issues in unit tests)
- handle interrupted exceptions encountered when callback thread encountered them
- change handling to throw error if we try to set verification state and leaderLogIfLocal is None.
Reviewers: David Jacot <djacot@confluent.io>, Artem Livshits <alivshits@confluent.io>, Jason Gustafson <jason@confluent.io>
* Adds an exponential backoff to 1m while the controller is waiting for brokers to show up
* Increases one-time logs to INFO
* Adds a summary of the migration records
* Use RecordRedactor for summary of migration batches (TRACE only)
Reviewers: Colin P. McCabe <cmccabe@apache.org>
This patch wires the new group coordinator in BrokerServer (KRaft only). With this, it is now possible to run a cluster with the new group coordinator and to use the ConsumerGroupHeartbeat API by specifying the following two properties:
- group.coordinator.new.enable = true (to enable the new group coordinator)
- unstable.api.versions.enable = true (to enable unreleased APIs)
Note that the new group coordinator does not support all the existing APIs yet.
Reviewers: Jeff Kim <jeff.kim@confluent.io>, Justine Olshan <jolshan@confluent.io>
Update the integration tests to swap the use of the concrete KafkaConsumer class to the generic Consumer interface.
Reviewers: Divij Vaidya <diviv@amazon.com>, Philip Nee <philipnee@gmail.com>, Jun Rao <junrao@gmail.com>
This commit prevents the leak of daemon-bounce-broker thread which was causing test failures for tests which check for thread leak prior to running.
Reviewers: Luke Chen <showuon@gmail.com>, Justine Olshan <jolshan@confluent.io>, Philip Nee <philipnee@gmail.com>
KAFKA-14522 Rewrite and Move of RemoteIndexCache to storage module.
Cleanedup index file suffix usages and other minor cleanups
Reviewers: Jun Rao <junrao@gmail.com>, Ismael Juma <ismael@juma.me.uk>, Luke Chen <showuon@gmail.com>, Divij Vaidya <diviv@amazon.com>, Kamal Chandraprakash<kamal.chandraprakash@gmail.com>, Jorge Esteban Quilcate Otoya <quilcate.jorge@gmail.com>
- Add new unit tests
- Change the on-disk filename from <offset>_<uuid>_.<indexSuffix> to <offset>_<uuid>.<indexSuffix> i.e. remove trailing underscore after
- Fix a small bug where we were parsing offset as Int when reading the file name from disk. Offset is long.
- Perform input validation in RemoteLogSegmentMetadata.
- Remove an extra loop in cleaner thread. Shutdownable thread already performs looping.
Reviewers: Jorge Esteban Quilcate Otoya <jorge.quilcate@aiven.io>, Satish Duggana <satishd@apache.org>
Fixes a bug where we don't send UMR and LISR requests in dual-write mode when new partitions are created. Prior to this patch, KRaftMigrationZkWriter was mutating the internal data-structures of TopicDelta which prevented MigrationPropagator from sending UMR and LISR for the changed partitions.
Reviewers: David Arthur <mumrah@gmail.com>
Fixes a regression introduced in PR #13924 by moving the map from static to a instance specific variable.
---------
Co-authored-by: Deqi Hu <deqi.hu@shopee.com>
This patch rewrites MockTimer in Java and moves it from core to server-common. This continues the work started in https://github.com/apache/kafka/pull/13820.
Reviewers: Divij Vaidya <diviv@amazon.com>
This commit utilizes TestUtils methods to create a topic and adds logs when assertions fail.
Reviewers: Divij Vaidya <diviv@amazon.com>
---------
Co-authored-by: d00791190 <dinglan6@huawei.com>
Use AdminApiDriver class to refresh the metadata and retry the request that failed with retriable errors.
Reviewers: Luke Chen <showuon@gmail.com>, Divij Vaidya <diviv@amazon.com>, Mickael Maison <mmaison@redhat.com>, Dimitar Dimitrov <30328539+dimitarndimitrov@users.noreply.github.com>
It's good for us to add support for Java 20 in preparation for Java 21 - the next LTS.
Given that Scala 2.12 support has been deprecated, a Scala 2.12 variant is not included.
Also remove some branch builds that add load to the CI, but have
low value: JDK 8 & Scala 2.13 (JDK 8 support has been deprecated),
JDK 11 & Scala 2.12 (Scala 2.12 support has been deprecated) and
JDK 17 & Scala 2.12 (Scala 2.12 support has been deprecated).
A newer version of Mockito (4.9.0 -> 4.11.0) is required for Java 20 support, but we
only use it with Scala 2.13+ since it causes compilation errors with Scala 2.12. Similarly,
we upgrade easymock when the Java version is 16 or newer as it's incompatible
with powermock (which doesn't support Java 16 or newer).
Filed KAFKA-15117 for a test that fails with Java 20 (SslTransportLayerTest.testValidEndpointIdentificationCN).
Finally, fixed some lossy conversions that were added after #13582 was submitted.
Reviewers: Ismael Juma <ismael@juma.me.uk>
The choice of using name vs. user as a parameter is because internally the record uses name, all
tests using the StorageTool use name as a parameter, KafkaPrincipals are created with name and
because creating SCRAM credentials is done with --entity-name
Reviewers: Colin P. McCabe <cmccabe@apache.org>
Fixed a regression described in KAFKA-15053 that security.protocol only allows uppercase values like PLAINTEXT, SSL, SASL_PLAINTEXT, SASL_SSL. With this fix, both lower case and upper case values will be supported (e.g. PLAINTEXT, SSL, SASL_PLAINTEXT, SASL_SSL, plaintext, ssl, sasl_plaintext, sasl_ssl)
Reviewers: Chris Egerton <chrise@aiven.io>, Divij Vaidya <diviv@amazon.com>
Adding the following metrics as per kip-890:
VerificationTimeMs – number of milliseconds from adding partition info to the manager to the time the response is sent. This will include the round trip to the transaction coordinator if it is called. This will also account for verifications that fail before the coordinator is called.
VerificationFailureRate – rate of verifications that returned in failure either from the AddPartitionsToTxn response or through errors in the manager.
AddPartitionsToTxnVerification metrics – separating the verification request metrics from the typical add partitions ones similar to how fetch replication and fetch consumer metrics are separated.
Reviewers: Divij Vaidya <diviv@amazon.com>
RPCProducerIdManager initiates an async request to the controller to grab a block of producer IDs and then blocks waiting for a response from the controller.
This is done in the request handler threads while holding a global lock. This means that if many producers are requesting producer IDs and the controller is slow to respond, many threads can get stuck waiting for the lock.
This patch aims to:
* resolve the deadlock scenario mentioned above by not waiting for a new block and returning an error immediately
* remove synchronization usages in RpcProducerIdManager.generateProducerId()
* handle errors returned from generateProducerId() so that KafkaApis does not log unexpected errors
* confirm producers backoff before retrying
* introduce backoff if manager fails to process AllocateProducerIdsResponse
Reviewers: Artem Livshits <alivshits@confluent.io>, Jason Gustafson <jason@confluent.io>
Use thread safe Caffeine to cache indexes fetched from RemoteTier locally. This PR removes a lock contention that led to higher fetch latencies as the IO threads spent time unnecessarily waiting on global cache lock while a single thread fetches the index from remote tier. See PR #13850 for details and rejected alternatives.
Reviewers: Luke Chen <showuon@gmail.com>, Satish Duggana <satishd@apache.org>
Fixes thread leaks by closing the ReplicaManager using try/finally at the end of each test. The leaks were leading to flaky test failures in ReplicaManagerTest.
Reviewers: Justine Olshan <jolshan@confluent.io>, David Jacot <djacot@confluent.io>
The Java rewrite is kept relatively close to the Scala original
to minimize potential newly introduced bugs and to make reviewing
simpler. The following details might be of note:
- The `Logging` trait moved to InterBrokerSendThread with the
rewrite of ShutdownableThread has been similarly moved to any
subclasses that currently use it. InterBrokerSendThread's own
logging has been made to use ShutdownableThread's logger which
mimics the prefix/log identifier that the trait provided.
- The case RequestAndCompletionHandler class has been made a
separate POJO class and the internal-use UnsentRequests class
has been kept as a static nested class.
- The relatively commonly used but internal (not part of the
public API) clients classes that InterBrokerSendThread relies on
have been allowlisted in the server-common import control.
- The accompanying test class has also been moved and rewritten
with one new test added and most of the pre-existing tests made
stricter.
Reviewers: David Jacot <djacot@confluent.io>
This PR expands the scope of ApiVersionManager a bit to include returning the current
MetadataVersion and features that are in effect. This is useful in general because that information
needs to be returned in an ApiVersionsResponse. It also allows us to fix the ApiVersionManager
interface so that all subclasses implement all methods of the interface. Having subclasses that
don't implement some methods is dangerous because they could cause exceptions at runtime in
unexpected scenarios.
On the KRaft controller, we were previously performing a read operation in the QuorumController
thread to get the current metadata version and features. With this PR, we now read a volatile
variable maintained by a separate MetadataVersionContextPublisher object. This will improve
performance and simplify the code. It should not change the guarantees we are providing; in both
the old and new scenarios, we need to be robust against version skew scenarios during updates.
Add a Features class which just has a 3-tuple of metadata version, features, and feature epoch.
Remove MetadataCache.FinalizedFeaturesAndEpoch, since it just duplicates the Features class.
(There are some additional feature-related classes that can be consolidated in in a follow-on PR.)
Create a java class, EndpointReadyFutures, for managing the futures associated with individual
authorizer endpoints. This avoids code duplication between ControllerServer and BrokerServer and
makes this code unit-testable.
Reviewers: David Arthur <mumrah@gmail.com>, dengziming <dengziming1993@gmail.com>, Luke Chen <showuon@gmail.com>
Change Timer.java to implement AutoCloseable because automatic bug finders will flag a warning if an object of a class is marked as AutoCloseable but is not closed properly in the code.
Reviewers: Divij Vaidya <diviv@amazon.com>
This test still fails regularly with the following error:
```
Error
java.util.concurrent.ExecutionException: org.opentest4j.AssertionFailedError: Timed out while awaiting expected assignment Set(topicWithAllPartitionsOnAllRacks-0, topicWithSingleRackPartitions-0). The current assignment is []
Stacktrace
java.util.concurrent.ExecutionException: org.opentest4j.AssertionFailedError: Timed out while awaiting expected assignment Set(topicWithAllPartitionsOnAllRacks-0, topicWithSingleRackPartitions-0). The current assignment is []
at java.base/java.util.concurrent.FutureTask.report(FutureTask.java:122)
at java.base/java.util.concurrent.FutureTask.get(FutureTask.java:205)
at integration.kafka.server.FetchFromFollowerIntegrationTest.$anonfun$testRackAwareRangeAssignor$9(FetchFromFollowerIntegrationTest.scala:211)
at integration.kafka.server.FetchFromFollowerIntegrationTest.$anonfun$testRackAwareRangeAssignor$9$adapted(FetchFromFollowerIntegrationTest.scala:211)
at scala.collection.IterableOnceOps.foreach(IterableOnce.scala:575)
at scala.collection.IterableOnceOps.foreach$(IterableOnce.scala:573)
```
I propose to increase the timeouts to 30 secs to mitigate it. The test already uses 30 secs timeouts in many places. This patch uses 30 secs everywhere. This solution is not optimal but this is better than having a flaky test.
Reviewers: Justine Olshan <jolshan@confluent.io>
I have seen failures like the following one in a few builds:
```
Build / JDK 11 and Scala 2.13 / testDescribeSimpleConsumerGroup() – kafka.admin.DescribeConsumerGroupTest
org.apache.kafka.common.errors.TopicExistsException: Topic '__consumer_offsets' already exists.
```
Many tests still use `TestUtils.createOffsetsTopic(zkClient, servers)` to create the offsets topic. This method does not handle the case where the topic exists (e.g. in the case of a retry). This patch adds this logic.
Reviewers: Divij Vaidya <diviv@amazon.com>, Justine Olshan <jolshan@confluent.io>