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>
Followers should cache the log offset metadata for the start offset of each transaction in order to be able to compute the last stable offset without an offset index lookup. This is needed for follower fetching in KIP-392.
Reviewers: Guozhang Wang <wangguoz@gmail.com>
Include group.instance.id in the describe group result for better visibility.
Reviewers: A. Sophie Blee-Goldman <sophie@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
We recently observed a transient failure of this test case:
```
java.lang.AssertionError: Contents of the map shouldn't change expected:<Map(0 -> (340,340), 5 -> (345,345), 10 -> (350,350), 14 -> (354,354), 1 -> (341,341), 6 -> (346,346), 9 -> (349,349), 13 -> (353,353), 2 -> (342,342), 17 -> (357,357), 12 -> (352,352), 7 -> (347,347), 3 -> (343,343), 18 -> (358,358), 16 -> (356,356), 11 -> (351,351), 8 -> (348,348), 19 -> (359,359), 4 -> (344,344), 15 -> (355,355))> but was:<Map(0 -> (340,340), 5 -> (345,345), 10 -> (350,350), 14 -> (354,354), 1 -> (341,341), 6 -> (346,346), 97 -> (297,297), 9 -> (349,349), 96 -> (296,296), 13 -> (353,353), 2 -> (342,342), 17 -> (357,357), 12 -> (352,352), 7 -> (347,347), 98 -> (298,298), 3 -> (343,343), 18 -> (358,358), 95 -> (295,295), 16 -> (356,356), 11 -> (351,351), 99 -> (299,299), 8 -> (348,348), 19 -> (359,359), 4 -> (344,344), 15 -> (355,355))>
```
The presence of old keys implies that not all old segments had been deleted. We believe the retention check (which runs asynchronously) is executing after the last modified time of all but one of the segments has been changed. This leaves one old segment behind which ultimately causes the above assertion error.
The fix here is to improve the wait condition.Rather than waiting for the number of segments to be 1, we wait for the log start offset to reach the end offset.
Reviewers: David Arthur <mumrah@gmail.com>
`ZkUtils` is not used by the broker, has been deprecated since
2.0.0 and it was never intended as a public API. We should
remove it along with `AdminUtils` methods that rely on it.
Reviewers: David Arthur <mumrah@gmail.com>
* KAFKA-8106:Reducing the allocation and copying of ByteBuffer when logValidator do validation.
* KAFKA-8106:Reducing the allocation and copying of ByteBuffer when logValidator do validation.
* github comments
* use batch.skipKeyValueIterator
* cleanups
* no need to skip kv for uncompressed iterator
* checkstyle fixes
* fix findbugs
* adding unit tests
* reuse decompression buffer; and using streaming iterator
* checkstyle
* add unit tests
* remove reusing buffer supplier
* fix unit tests
* add unit tests
* use streaming iterator
* minor refactoring
* rename
* github comments
* github comments
* reuse buffer at DefaultRecord caller
* some further optimization
* major refactoring
* further refactoring
* update comment
* github comments
* minor fix
* add jmh benchmarks
* update jmh
* github comments
* minor fix
* github comments
It has been deprecated since 0.11.0, it was never meant as a publicly
supported API and people should use
`org.apache.kafka.clients.admin.AdminClient` instead. Its presence
causes confusion and people still use them accidentally at times.
`BrokerApiVersionsCommand` uses one method that is not available
in `org.apache.kafka.clients.admin.AdminClient`, we inline it for now.
Reviewers: David Arthur <mumrah@gmail.com>, Ismael Juma <ismael@juma.me.uk>
The AllReplicas was only printing remote replica ids. This change prints
all ids, including local one.
Reviewers: José Armando García Sancio <jsancio@users.noreply.github.com>, Jason Gustafson <jason@confluent.io>
Kafka should not NPE while loading a deleted partition dir with no log segments. This patch ensures that there will always be at least one segment after initialization.
Co-authored-by: Edoardo Comar <ecomar@uk.ibm.com>
Co-authored-by: Mickael Maison <mickael.maison@gmail.com>
Reviewers: Ismael Juma <ismael@juma.me.uk>, Jason Gustafson <jason@confluent.io>
Cancel PeriodicProducerExpirationCheck when closing a Log instance, to avoid a memory leak. Add a method to KafkaScheduler to make this possible.
Reviewers: Colin P. McCabe <cmccabe@apache.org>, Ismael Juma <ismael@juma.me.uk>
This patch simplifies the controller election API. We were passing `LeaderIsrAndControllerEpoch` into the election utilities even though we just needed `LeaderAndIsr`. We also remove some unneeded collection copies `doElectLeaderForPartitions`.
Reviewers: Ismael Juma <ismael@juma.me.uk>
A `Partition` object contain one or many `Replica` objects. These replica
objects in turn can have the "log" if the replica corresponds to the
local node. All the code in Partition or ReplicaManager peek into
replica object to fetch the log if they need to operate on that. As
replica object can represent a local replica or a remote one, this
lead to a bunch of "if-else" code in log fetch and offset update code.
NOTE: In addition to a "log" that is in use during normal operation, if
an alter log directory command is issued, we also create a future log
object. This object catches up with local log and then we switch the log
directory. So temporarily a Partition can have two local logs. Before
this change both logs are inside replica objects.
This change is an attempt to untangle this relationship. In particular
it moves `Log` from `Replica` into `Partition`. So a partition contains
a local log to which all writes go and possibly a "future log" if the
partition is being moved between directories. Additionally, it maintains
a list of remote replicas for offset and "caught up time" data that it uses
for replication protocol.
Reviewers: José Armando García Sancio <jsancio@users.noreply.github.com>, Jason Gustafson <jason@confluent.io>
Makes HostedPartition a sealed trait and make all of the match cases explicit.
Reviewers: Vikas Singh <soondenana@users.noreply.github.com>, Jason Gustafson <jason@confluent.io>
This PR fixes a bug in static group membership. Previously we limit the `member.id` replacement in JoinGroup to only cases when the group is in Stable. This is error-prone and could potentially allow duplicate consumers reading from the same topic. For example, imagine a case where two unknown members join in the `PrepareRebalance` stage at the same time.
The PR fixes the following things:
1. Replace `member.id` at any time we see a known static member rejoins group with unknown member.id
2. Immediately fence any ongoing join/sync group callback to early terminate the duplicate member.
3. Clearly handle Dead/Empty cases as exceptional.
4. Return old leader id upon static member leader rejoin to avoid trivial member assignment being triggered.
Reviewers: Guozhang Wang <wangguoz@gmail.com>, Jason Gustafson <jason@confluent.io>
Currently we load the high watermark checkpoint separately for every replica that we load. This patch makes this loading logic lazy and caches the loaded map while a LeaderAndIsr request is being handled.
Reviewers: Jun Rao <junrao@gmail.com>
We have seen this failing recently due to the follower error:
```
java.util.NoSuchElementException: None.get
at scala.None$.get(Option.scala:366)
at scala.None$.get(Option.scala:364)
at kafka.admin.PreferredReplicaLeaderElectionCommandTest.getLeader(PreferredReplicaLeaderElectionCommandTest.scala:101)
at kafka.admin.PreferredReplicaLeaderElectionCommandTest.testNoopElection(PreferredReplicaLeaderElectionCommandTest.scala:240)
```
We need to wait for the leader to be available.
Reviewers: David Arthur <mumrah@gmail.com>
We see this failure from time to time:
```
java.lang.AssertionError: expected:<1> but was:<0>
at org.junit.Assert.fail(Assert.java:89)
at org.junit.Assert.failNotEquals(Assert.java:835)
at org.junit.Assert.assertEquals(Assert.java:647)
at org.junit.Assert.assertEquals(Assert.java:633)
at kafka.api.TransactionsTest.testFencingOnTransactionExpiration(TransactionsTest.scala:512)
```
The cause is probably that we are using `consumeRecordsFor` which has no expectation on the number of records to fetch and a timeout of just 1s. This patch changes the code to use `consumeRecords` and the default 15s timeout.
Note we have also fixed a bug in the test case itself, which was using the wrong topic for the second write, which meant it could never have failed in the anticipated way anyway.
Author: Jason Gustafson <jason@confluent.io>
Reviewers: Gwen Shapira
Closes#6905 from hachikuji/fix-flaky-transaction-test
This patch adds test cases for the leader election command added in KIP-460.
Reviewers: Vikas Singh, David Arthur <mumrah@gmail.com>, Jason Gustafson <jason@confluent.io>
This patch checks for errors handling a fetch request before updating follower state. Previously we were unsafely passing the failed `LogReadResult` with most fields set to -1 into `Replica` to update follower state. Additionally, this patch attempts to improve the test coverage for ISR shrinking and expansion logic in `Partition`.
Reviewers: Guozhang Wang <wangguoz@gmail.com>
The Dead state in the coordinator is used for groups which are either pending deletion or migration to a new coordinator. Currently requests received while in this state result in an UNKNOWN_MEMBER_ID which causes consumers to reset the memberId. This is a problem for KIP-345 since it can cause an older member to fence a newer member. This patch changes the error code returned in this state to COORDINATOR_NOT_AVAILABLE, which causes the consumer to rediscover the coordinator, but not reset the memberId.
Reviewers: Guozhang Wang <wangguoz@gmail.com>, Jason Gustafson <jason@confluent.io>
This commit makes three changes:
- Adds a constructor for NewTopic(String, Optional<Integer>, Optional<Short>)
which allows users to specify Optional.empty() for numPartitions or
replicationFactor in order to use the broker default.
- Changes AdminManager to accept -1 as valid options for replication
factor and numPartitions (resolving to broker defaults).
- Makes --partitions and --replication-factor optional arguments when creating
topics using kafka-topics.sh.
- Adds a dependency on scalaJava8Compat library to make it simpler to
convert Scala Option to Java Optional
Reviewers: Ismael Juma <ismael@juma.me.uk>, Ryanne Dolan <ryannedolan@gmail.com>, Jason Gustafson <jason@confluent.io>
The test was failing because other test cases are leaving some metrics in the registry. This patch cleans the registry before and after test runs.
Reviewers: Ismael Juma <ismael@juma.me.uk>