If retrieving metadata during `poll(Duration)` repeatedly takes longer than the
poll timeout, we don't make progress and `waitUntilTrue` eventually times out
causing the test to fail. This behaviour differs from the older `poll(long)` that
would block until metadata retrieval had completed. The flakiness was likely
introduced when we switched from the latter to the former.
Reviewers: Ismael Juma <ismael@juma.me.uk>
Refactors the DescribeDelegationToken to use the generated RPC classes.
Author: Viktor Somogyi-Vass <viktorsomogyi@gmail.com>
Author: Viktor Somogyi <viktorsomogyi@gmail.com>
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Manikumar Reddy <manikumar.reddy@gmail.com>
Closes#7154 from viktorsomogyi/refactor-describe-dt
This patch updates ConsumerGroupCommandTest.scala to use the maximum possible number of AdminClient retries. The test runs will still be bounded by the request timeout. This address flakiness in tests such as testResetOffsetsNotExistingGroup and testResetOffsetsExistingTopic, which was caused by group coordinators being intermittently unavailable.
Reviewers: Ismael Juma <ismael@juma.me.uk>, Jason Gustafson <jason@confluent.io>
Update the javadoc of SockerServer#startup(). SocketServer#startProcessors() does not exist any more and it has been replaced by SocketServer#startDataPlaneProcessors() and SocketServer#startControlPlaneProcessor().
Reviewers: Jason Gustafson <jason@confluent.io>
This patch stores `clusterId` in the `meta.properties` file. During startup, the broker checks that it joins the correct cluster and fails fast otherwise.
The `meta.properties' is versioned. I have decided to not bump the version because 1) the clusterId is null anyway if not present in the file; and 2) bumping it means that rolling back to a previous version won't work.
I have refactored the way the metadata is read and written as it was strongly coupled with the brokerId bits. Now, the metadata is read independently during the startup and used to 1) check the clusterId and 2) get or generate the brokerId (as before).
Reviewers: Stanislav Kozlovski <stanislav_kozlovski@outlook.com>, Jason Gustafson <jason@confluent.io>
The assignReplicasToBrokers method has helpful but a large unformatted
javadoc comment that results in a big blob in generated html. This
change formats the comment so that generated javadoc is nice.
Reviewers: Stanislav Kozlovski <familyguyuser192@windowslive.com>, Jason Gustafson <jason@confluent.io>
Since `Metrics` was constructed with `enableExpiration=false`, this was
not a source of flakiness given the current implementation. This could
change in the future, so good to follow the class contract.
Included a few clean-ups with regards to redundant casts and type parameters
as well as usage of try with resources for inline usage of `Metrics`.
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
We recently introduced a bunch of flaky tests in the AdminClientIntegrationTest.
These tests are failing very frequently. We should ignore the tests in order to
make the build stable until we have a fix.
Reviewers: Ismael Juma <ismael@juma.me.uk>
The introduction of KIP-480: Sticky Producer Partitioner had the
side effect that generateAndProduceMessages can often write
messages to a lower number of partitions to improve batching.
testDescribeLogDirsRequest (and potentially other tests) relies
on the messages being written somewhat uniformly to the topic
partitions. We fix the issue by including a monotonically
increasing key in the produced messages.
I also included a couple of minor clean-ups I noticed while
debugging the issue.
The test failed very frequently when executed locally before the
change and it passed 100 times consecutively after the change.
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
Author: Mickael Maison <mickael.maison@gmail.com>
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>, Viktor Somogyi <viktorsomogyi@gmail.com>
Closes#7038 from mimaison/KAFKA-8598
1. Add onPartitionsLost into the RebalanceListener, which will be triggered when the consumer found that the generation is reset due to fatal errors in response handling.
2. Semantical behavior change: with COOPERATIVE protocol, if the revoked / lost partitions are empty, do not trigger the corresponding callback at all. For added partitions though, even if it is empty we would still trigger the callback as a way to notify the rebalance event; with EAGER protocol, revoked / assigned callbacks are always triggered.
The ordering of the callback would be the following:
a. Callback onPartitionsRevoked / onPartitionsLost triggered.
b. Update the assignment (both revoked and added).
c. Callback onPartitionsAssigned triggered.
In this way we are assured that users can still access the partitions being revoked, whereas they can also access the partitions being added.
3. Semantical behavior change (KAFKA-4600): if the rebalance listener throws an exception, pass it along all the way to the consumer.poll caller, but still completes the rest of the actions. Also, the newly assigned partitions list does not gets affected with exception thrown since it is just for notifying the users.
4. Semantical behavior change: the ConsumerCoordinator would not try to modify assignor's returned assignments, instead it will validate that assignments and set the error code accordingly: if there are overlaps between added / revoked partitions, it is a fatal error and would be communicated to all members to stop; if revoked is not empty, it is an error indicate re-join; otherwise, it is normal.
5. Minor: with the error code removed from the Assignment, ConsumerCoordinator will request re-join if the revoked partitions list is not empty.
6. Updated ConsumerCoordinatorTest accordingly. Also found a minor bug in MetadataUpdate that removed topic would still be retained with null value of num.partitions.
6. Updated a few other flaky tests that are exposed due to this change.
Reviewers: John Roesler <vvcephei@users.noreply.github.com>, A. Sophie Blee-Goldman <sophie@confluent.io>, Jason Gustafson <jason@confluent.io>
Author: Mickael Maison <mickael.maison@gmail.com>
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>, Viktor Somogyi <viktorsomogyi@gmail.com>
Closes#7098 from mimaison/KAFKA-8599
LogConfig.getConfigValue would throw a NoSuchElementException if any log
config was defined without a server default mapping.
Added a unit test for `getConfigValue` and a sanity test for
`toHtml`/`toRst`/`toEnrichedRst`, which were previously not exercised during
the test suite.
Reviewers: Jason Gustafson <jason@confluent.io>, José Armando García Sancio <jsancio@users.noreply.github.com>
[JIRA](https://issues.apache.org/jira/browse/KAFKA-6263)
- Add metrics to provide visibility for how long group metadata and transaction metadata take to load in order to understand some inactivity seen in the consumer groups
- Tests include mocking load times by creating a delay after each are loaded and ensuring the measured JMX metric is as it should be
Author: anatasiavela <anastasiavela@berkeley.edu>
Reviewers: Gwen Shapira, Jason Gustafson
Closes#7045 from anatasiavela/KAFKA-6263
<!--
Is there any breaking changes? If so this is a major release, make sure '#major' is in at least one
commit message to get CI to bump the major. This will prevent automatic down stream dependency
bumping / consuming. For more information about semantic versioning see: https://semver.org/
Suggested PR template: Fill/delete/add sections as needed. Optionally delete any commented block.
-->
What
----
<!--
Briefly describe **what** you have changed and **why**.
Optionally include implementation strategy.
-->
References
----------
[**KIP-412**](https://cwiki.apache.org/confluence/display/KAFKA/KIP-412%3A+Extend+Admin+API+to+support+dynamic+application+log+levels)
[**KAFKA-7800**](https://issues.apache.org/jira/browse/KAFKA-7800)
[**Discussion Thread**](http://mail-archives.apache.org/mod_mbox/kafka-dev/201901.mbox/%3CCANZZNGyeVw8q%3Dx9uOQS-18wL3FEmnOwpBnpJ9x3iMLdXY3gEug%40mail.gmail.com%3E)
[**Vote Thread**](http://mail-archives.apache.org/mod_mbox/kafka-dev/201902.mbox/%3CCANZZNGzpTJg5YX1Gpe5S%3DHSr%3DXGvmxvYLTdA3jWq_qwH-UvorQ%40mail.gmail.com%3E)
<!--
Copy&paste links: to Jira ticket, other PRs, issues, Slack conversations...
For code bumps: link to PR, tag or GitHub `/compare/master...master`
-->
Test&Review
------------
Test cases covered:
* DescribeConfigs
* Alter the log level with and without validateOnly, validate the results with DescribeConfigs
Open questions / Follow ups
--------------------------
If you're a reviewer, I'd appreciate your thoughts on these questions I have open:
1. Should we add synchronization to the Log4jController methods? - Seems like we don't get much value from it
2. Should we instantiate a new Log4jController instead of it having static methods? - All operations are stateless, so I thought static methods would do well
3. A logger which does not have a set value returns "null" (as seen in the unit tests). Should we just return the Root logger's level?
Author: Stanislav Kozlovski <familyguyuser192@windowslive.com>
Reviewers: Gwen Shapira
Closes#6903 from stanislavkozlovski/KAFKA-7800-dynamic-log-levels-admin-ap
Better clarify the auto leader rebalance config documentation to reflect additional leader.imbalance.per.broker.percentage config description.
Reviewers: Stanislav Kozlovski <stanislav_kozlovski@outlook.com>, Jun Rao <junrao@gmail.com>
This patch contains a few small cleanups to make topic describe logic a little clearer. It also fixes a minor inconsistency in the output between the --zookeeper and --bootstrap-server behavior when the leader is unknown. Previously we printed -1 when --zookeeper was used, and now we print "none." The patch consolidates the output logic for describing topics and partitions in order to avoid inconsistencies like this in the future.
Reviewers: José Armando García Sancio <jsancio@users.noreply.github.com>, Guozhang Wang <wangguoz@gmail.com>
Although we currently cache offset metadata for the high watermark and last stable offset, we don't use it when reading from the log. Instead we always look it up from the index. This patch pushes fetch isolation into `Log.read` so that we are able to reuse the cached offset metadata.
Reviewers: Jun Rao <junrao@gmail.com>
Currently the Metadata response returns an empty ISR if there is no active leader. The behavior is inconsistent since other fields such as the replica list and offline replicas are included. This patch changes the behavior to return the current known ISR. This fixes a problem with the topic describe command which fails to report ISR when a leader is offline.
Reviewers: Jason Gustafson <jason@confluent.io>
Add a new exception, NoReassignmentInProgressException. Modify LeaderAndIsrRequest to include the AddingRepicas and RemovingReplicas fields. Add the ListPartitionReassignments and AlterPartitionReassignments RPCs.
Reviewers: Colin P. McCabe <cmccabe@apache.org>, Viktor Somogyi <viktorsomogyi@gmail.com>
And remove redundant call. Closing ZKDatabase is necessary to allow the data
directory to be deleted when running on Windows.
Reviewers: Ismael Juma <ismael@juma.me.uk>
This patch is part of KIP-345. We are aiming to support batch leave group request issued from admin client. This diff is the first effort to bump leave group request version.
Reviewers: Guozhang Wang <wangguoz@gmail.com>, Jason Gustafson <jason@confluent.io>
The bug is that we accidentally used the current state timestamp for the group instead of the real current time. When a group is first loaded, this timestamp is not initialized, so this resulted in a `NoSuchElementException`. Additionally this violated the intended uniqueness of the memberId, which could have broken the group instance fencing. Fix is made and unit test to make sure the timestamp is properly encoded within the returned member.id.
Reviewers: Ismael Juma <ismael@juma.me.uk>, Guozhang Wang <wangguoz@gmail.com>, Jason Gustafson <jason@confluent.io>
Main changes of this PR:
* Deprecate old consumer.internal.PartitionAssignor and add public consumer.ConsumerPartitionAssignor with all OOTB assignors migrated to new interface
* Refactor assignor's assignment/subscription related classes for easier to evolve API
* Removed version number from classes as it is only needed for serialization/deserialization
* Other previously-discussed cleanup included in this PR:
* Remove Assignment.error added in pt 1
* Remove ConsumerCoordinator#adjustAssignment added in pt 2
Reviewers: Boyang Chen <boyang@confluent.io>, Jason Gustafson <jason@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
Minor refactoring of the places where we delete log segments, to ensure we always remove the in-memory metadata of the segment before performing physical deletion.
Reviewers: Ismael Juma <ismael@juma.me.uk>, NIkhil Bhatia <rite2nikhil@gmail.com>, Jun Rao <junrao@gmail.com>
* Clean up one redundant and one misplaced metric
* Clarify the relationship among these metrics to avoid future confusion
Reviewers: Matthias J. Sax <matthias@confluent.io>, Bill Bejeck <bill@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
LogManager#getOrCreateLog() selects a log dir for the new replica from
_liveLogDirs, if disk failure is discovered at this point, before
LogDirFailureHandler finds out, try using other log dirs before failing
the operation.
Reviewers: Anna Povzner <anna@confluent.io>, Jason Gustafson <jason@confluent.io>
* Added removeOldLeaderMetrics in BrokerTopicStats to remove MessagesInPerSec, BytesInPerSec, BytesOutPerSec for any broker that is no longer a leader of any partition for a particular topic
* Modified ReplicaManager to remove the metrics of any topic that the current broker has no leadership (meaning the broker either becomes a follower for all of the partitions in that topic or stops being a replica)
Reviewers: Guozhang Wang <wangguoz@gmail.com>, Jun Rao <junrao@gmail.com>
If there are **no topics** in a cluster, kafka-topics.sh --describe without a --topic option should return empty list, not throw an exception.
Reviewers: Jason Gustafson <jason@confluent.io>
Ensure that Producer#send() throws topic metadata exceptions only for the topic being sent to and not for other topics in the producer's metadata instance. Also removes topics from consumer's metadata when a topic is removed using manual assignment.
Reviewers: Jason Gustafson <jason@confluent.io>, Ismael Juma <ismael@juma.me.uk>
The purpose of this patch is to restrict the paths for updating and accessing the high watermark and the last stable offset. By doing so, we can validate that these offsets always remain within the range of the log. We also ensure that we never expose `LogOffsetMetadata` unless it is fully materialized. Finally, this patch makes a few naming changes. In particular, we remove the `highWatermark_=` and `highWatermarkMetadata_=` which are both misleading and cumbersome to test.
Reviewers: David Arthur <mumrah@gmail.com>
ZooKeeper 3.5.5 is the first stable release in the 3.5.x series. The key new feature
in is TLS support, but there are a few more noteworthy features:
* Dynamic reconfiguration
* Local sessions
* New node types: Container, TTL
* Ability to remove watchers
* Multi-threaded commit processor
* Upgraded to Netty 4.1
See the release notes for more detail:
https://zookeeper.apache.org/doc/r3.5.5/releasenotes.html
In addition to the version bump, we:
* Add `commons-cli` dependency as it's required by `ZooKeeperMain`, but specified as
`provided` in their pom.
* Remove unnecessary `ZooKeeperMainWrapper`, the bug it worked around was fixed
upstream a long time ago.
* Ignore non zero exit in one system test invocation of `ZooKeeperMain`.
`ZooKeeperMainWrapper` always returned `0` and `ZooKeeperService.query` relies
on that for correct behavior.
Reviewers: Jason Gustafson <jason@confluent.io>
We noticed a lot of messages like the following in the broker logs recently:
```
[2019-07-10 02:01:23,946] WARN [ReplicaManager broker=0] While updating the HW for follower -1 for partition connect-storage-topic-connect-cluster-0, the replica could not be found. (kafka.server.ReplicaManager:70)
```
In the KIP-392 PR, we added logic to track the high watermark of followers, but it is invoked even for consumer fetches. This doesn't cause any harm other than all the log noise.
This patch just adds the missing follower check.
Reviewers: David Arthur <mumrah@gmail.com>
We don't allow changing number of partitions for internal topics. To do
so we check if the topic name belongs to the set of internal topics
directly instead of using the "isInternalTopic" method. This breaks the
encapsulation by making client aware of the fact that internal topics
have special names.
This is a simple change to use the method `Topic::isInternalTopic`
method instead of checking it directly in "alterTopic" command. We
also reduce visibility to `Topic::INTERNAL_TOPICS` to avoid
unnecessary reliance on it in the future.
Reviewers: Jason Gustafson <jason@confluent.io>
ZkUtils was removed so we don't need this anymore.
Also:
* Fix ZkSecurityMigrator and ReplicaManagerTest not to
reference ZkClient classes.
* Remove references to zkclient in various `log4j.properties`
and `import-control.xml`.
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>, Stanislav Kozlovski <stanislav_kozlovski@outlook.com>
We recently saw a few failing tests recently due to the static reliance on port 5000. For example:
```
org.apache.kafka.common.KafkaException: Socket server failed to bind to localhost:5000: Address already in use.
at kafka.network.Acceptor.openServerSocket(SocketServer.scala:605)
at kafka.network.Acceptor.<init>(SocketServer.scala:481)
at kafka.network.SocketServer.createAcceptor(SocketServer.scala:253)
at kafka.network.SocketServer.$anonfun$createControlPlaneAcceptorAndProcessor$1(SocketServer.scala:234)
at kafka.network.SocketServer.$anonfun$createControlPlaneAcceptorAndProcessor$1$adapted(SocketServer.scala:232)
at scala.Option.foreach(Option.scala:438)
at kafka.network.SocketServer.createControlPlaneAcceptorAndProcessor(SocketServer.scala:232)
at kafka.network.SocketServer.startup(SocketServer.scala:119)
at kafka.network.SocketServerTest.withTestableServer(SocketServerTest.scala:1139)
at kafka.network.SocketServerTest.testControlPlaneRequest(SocketServerTest.scala:198)
```
This patch fixes the failing tests to dynamically select the port.
Reviewers: Ismael Juma <ismael@juma.me.uk>
Follow on to #6731, this PR adds broker-side support for [KIP-392](https://cwiki.apache.org/confluence/display/KAFKA/KIP-392%3A+Allow+consumers+to+fetch+from+closest+replica) (fetch from followers).
Changes:
* All brokers will handle FetchRequest regardless of leadership
* Leaders can compute a preferred replica to return to the client
* New ReplicaSelector interface for determining the preferred replica
* Incremental fetches will include partitions with no records if the preferred replica has been computed
* Adds new JMX to expose the current preferred read replica of a partition in the consumer
Two new conditions were added for completing a delayed fetch. They both relate to communicating the high watermark to followers without waiting for a timeout:
* For regular fetches, if the high watermark changes within a single fetch request
* For incremental fetch sessions, if the follower's high watermark is lower than the leader
A new JMX attribute `preferred-read-replica` was added to the `kafka.consumer:type=consumer-fetch-manager-metrics,client-id=some-consumer,topic=my-topic,partition=0` object. This was added to support the new system test which verifies that the fetch from follower behavior works end-to-end. This attribute could also be useful in the future when debugging problems with the consumer.
Reviewers: José Armando García Sancio <jsancio@users.noreply.github.com>, Jun Rao <junrao@gmail.com>, Jason Gustafson <jason@confluent.io>
The patch clarifies the exception message for unknown key versions when loading from the group metadata topic. The patch also makes a trivial change in `KafkaAdminClient` to use `Map.computeIfAbsent`.
Reviewers: Viktor Somogyi <viktorsomogyi@gmail.com>, Jason Gustafson <jason@confluent.io>
Leaders should make changes to the assignment and the ISR at the same time as part of processing the LeaderAndIsr requests. The leader should also preserve the order of assignment mainly for consistency with the Controller's code and data representation.
Reviewers: Vikas Singh, David Arthur <mumrah@gmail.com>, Jason Gustafson <jason@confluent.io>
Scala 2.13 support was added to build via #5454. This PR adjusts the code so that
it compiles with 2.11, 2.12 and 2.13.
Changes:
* Add `scala-collection-compat` dependency.
* Import `scala.collection.Seq` in a number of places for consistent behavior between
Scala 2.11, 2.12 and 2.13.
* Remove wildcard imports that were causing the Java classes to have priority over the
Scala ones, related Scala issue: https://github.com/scala/scala/pull/6589.
* Replace parallel collection usage with `Future`. The former is no longer included by
default in the standard library.
* Replace val _: Unit workaround with one that is more concise and works with Scala 2.13
* Replace `filterKeys` with `filter` when we expect a `Map`. `filterKeys` returns a view
that doesn't implement the `Map` trait in Scala 2.13.
* Replace `mapValues` with `map` or add a `toMap` as an additional transformation
when we expect a `Map`. `mapValues` returns a view that doesn't implement the
`Map` trait in Scala 2.13.
* Replace `breakOut` with `iterator` and `to`, `breakOut` was removed in Scala
2.13.
* Replace to() with toMap, toIndexedSeq and toSet
* Replace `mutable.Buffer.--` with `filterNot`.
* ControlException is an abstract class in Scala 2.13.
* Variable arguments can only receive arrays or immutable.Seq in Scala 2.13.
* Use `Factory` instead of `CanBuildFrom` in DecodeJson. `CanBuildFrom` behaves
a bit differently in Scala 2.13 and it's been deprecated. `Factory` has the behavior
we need and it's available via the compat library.
* Fix failing tests due to behavior change in Scala 2.13,
"Map.values.map is not strict in Scala 2.13" (https://github.com/scala/bug/issues/11589).
* Use Java collections instead of Scala ones in StreamResetter (a Java class).
* Adjust CheckpointFile.write to take an `Iterable` instead of `Seq` to avoid
unnecessary collection copies.
* Fix DelayedElectLeader to use a Map instead of Set and avoid `to` call that
doesn't work in Scala 2.13.
* Use unordered map for mapping in SimpleAclAuthorizer, mapping of ordered
maps require an `Ordering` in Scala 2.13 for safety reasons.
* Adapt `ConsumerGroupCommand` to compile with Scala 2.13.
* CoreUtils.min takes an `Iterable` instead of `TraversableOnce`, the latter does
not exist in Scala 2.13.
* Replace `Unit` with `()` in a couple places. Scala 2.13 is stricter when it expects
a value instead of a type.
* Fix bug in CustomQuotaCallbackTest where we did not necessarily set `partitionRatio`
correctly, `forall` can terminate early.
* Add a couple of spotbugs exclusions that are needed by code generated by Scala 2.13
* Remove unused variables, simplify some code and remove procedure syntax in a few
places.
* Remove unused `CoreUtils.JSONEscapeString`.
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>, José Armando García Sancio <jsancio@users.noreply.github.com>
Due to the accidental duplication of `case e: ExecutionException`,
the verification code was not actually running.
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>, David Arthur <mumrah@gmail.com>