- Rename `encode` to `legacyEncodeAsString`, we
can remove this when we remove `ZkUtils`.
- Introduce `encodeAsString` that uses Jackson.
- Change `encodeAsBytes` to use Jackson.
- Avoid intermediate string when converting
Broker to json bytes.
The methods that use Jackson only support
Java collections unlike `legacyEncodeAsString`.
Tests were added `encodeAsString` and
`encodeAsBytes`.
Author: umesh chaudhary <umesh9794@gmail.com>
Reviewers: Ismael Juma <ismael@juma.me.uk>
Closes#4259 from umesh9794/KAFKA-5631
This is required for ACLs where SSL principals contain
special characters (e.g. comma) that are escaped using
backslash. The strings need to be quoted for JSON to
ensure that the JSON stored in ZK is valid.
Also converted `SslEndToEndAuthorizationTest` to use a
principal with special characters to ensure that this
path is tested.
Author: Rajini Sivaram <rajinisivaram@googlemail.com>
Reviewers: Ismael Juma <ismael@juma.me.uk>
Closes#4303 from rajinisivaram/KAFKA-6319
- Rename `delete()` to `deleteIfExists()` in `LogSegment`, `AbstractIndex`
and `TxnIndex`. Throw exception in case of IO errors for more informative
errors and to make it less likely that errors are ignored, `boolean` is used
for the case where the file does not exist (like `Files.deleteIfExists()`).
- Fix an instance of delete while open (should fix KAFKA-6322 and
KAFKA-6075).
- `LogSegment.deleteIfExists` no longer throws an exception if any of
the files it tries to delete does not exist (fixes KAFKA-6194).
- Remove unnecessary `FileChannel.force(true)` when deleting file.
- Introduce `LogSegment.open()` and use it to improve encapsulation
and reduce duplication.
- Expand functionality of `LogSegment.onBecomeInactiveSegment()`
to reduce duplication and improve encapsulation.
- Use `AbstractIndex.deleteIfExists()` instead of deleting files manually.
- Improve logging when deleting swap files.
- Use CorruptIndexException instead of IllegalArgumentException.
- Simplify `LogCleaner.cleanSegments()` to reduce duplication and
improve encapsulation.
- A few other clean-ups in Log, LogSegment, etc.
Author: Ismael Juma <ismael@juma.me.uk>
Reviewers: Jun Rao <junrao@gmail.com>, Ted Yu <yuzhihong@gmail.com>
Closes#4040 from ijuma/kafka-5829-follow-up
Also:
- Fix bug in result type of `createSequentialPersistentPath`
- Remove duplicated code from `ReplicationUtils`
- Move `propagateIsrChanges` from `ReplicationUtils` to `KafkaZkClient`
- Add tests
- Minor clean-ups
Author: Ismael Juma <ismael@juma.me.uk>
Reviewers: Jun Rao <junrao@gmail.com>, Ted Yu <yuzhihong@gmail.com>, Manikumar Reddy <manikumar.reddy@gmail.com>
Closes#4261 from ijuma/zk-data-improvements
Measures the latency of each request.
Updated existing `ZkUtils` test to use `KafkaZkClient`
instead.
Author: Ismael Juma <ismael@juma.me.uk>
Reviewers: Jun Rao <junrao@gmail.com>
Closes#4265 from ijuma/kafka-6065-async-zk-metrics
- Ensure that `partitionsBeingReassigned` is fully populated before
`removePartitionFromReassignedPartitions` is invoked. This is
necessary to avoid premature deletion of the `reassign_partitions`
znode.
- Modify and add tests to verify the fixes.
- Add documentation.
- Use `info` log message if assignedReplicas == newReplicas and
remove control flow based on exceptions.
- General logging improvements.
- Simplify `initializePartitionAssignment` by relying on logic already
present in `maybeTriggerPartitionReassignment`.
Author: Ismael Juma <ismael@juma.me.uk>
Reviewers: Jun Rao <junrao@gmail.com>, Guozhang Wang <wangguoz@gmail.com>
Closes#4283 from ijuma/kafka-6193-flaky-shouldPerformMultipleReassignmentOperationsOverVariousTopics
* Add maybeThrow method to AsyncResponse
* Update KafkaZkClient to use newly introduced maybeThrow
* Change AsyncResponse from trait to abstract class for
more readable stacktraces (there's no benefit in using a
trait here)
Author: Manikumar Reddy <manikumar.reddy@gmail.com>
Reviewers: Ismael Juma <ismael@juma.me.uk>
Closes#4266 from omkreddy/KAFKAZKCLEINT_EXCEPTION_CLEANUP
It's rare, but it can happen that the initial FindCoordinator request returns before the first Metadata request. Both authorization errors are fine for this test case.
Author: Jason Gustafson <jason@confluent.io>
Reviewers: Guozhang Wang <wangguoz@gmail.com>
Closes#4287 from hachikuji/KAFKA-6118
If the ControllerEventThread is interrupted when a request is
being sent, it may lead to an IllegalStateException being thrown.
This, in turn, can lead to a NullPointerException in
unregisterPartitionReassignmentIsrChangeHandlers,
To avoid these issues, we make the ControllerEventThread
uninterruptable and we shut it down by clearing the queue
and enqueuing a special event.
To make the code more robust, we also set
ReassignedPartitionsContext.reassignIsrChangeHandler
during construction instead of setting it to null first.
Finally, misleading log messages in ephemeral node
creation have been clarified.
For reference, the relevant log lines from the relevant
flaky test:
```text
[2017-11-15 10:30:13,869] ERROR Error while creating ephemeral at /controller with return code: OK (kafka.zk.KafkaZkClient$CheckedEphemeral:101)
[2017-11-15 10:30:14,155] ERROR Haven't been able to send leader and isr requests, current state of the map is Map(101 -> Map(topic1-0 -> PartitionState(controllerEpoch=2, leader=101, leaderEpoch=3, isr=101, zkVersion=3, replicas=100,102,101, isNew=false)), 100 -> Map(topic1-0 -> PartitionState(controllerEpoch=2, leader=101, leaderEpoch=3, isr=101, zkVersion=3, replicas=100,102,101, isNew=false)), 102 -> Map(topic1-0 -> PartitionState(controllerEpoch=2, leader=101, leaderEpoch=3, isr=101, zkVersion=3, replicas=100,102,101, isNew=false))). Exception message: java.lang.InterruptedException (kafka.controller.ControllerBrokerRequestBatch:101)
[2017-11-15 10:30:14,156] ERROR Haven't been able to send metadata update requests to brokers Set(102, 103, 104, 101, 105), current state of the partition info is Map(topic1-0 -> PartitionState(controllerEpoch=1, leader=101, leaderEpoch=2, isr=[101], zkVersion=2, replicas=[100, 102, 101], offlineReplicas=[100])). Exception message: java.lang.InterruptedException (kafka.controller.ControllerBrokerRequestBatch:101)
[2017-11-15 10:30:14,158] ERROR [Controller id=101] Forcing the controller to resign (kafka.controller.KafkaController:101)
[2017-11-15 10:30:14,158] ERROR [Controller id=101] Error completing reassignment of partition topic1-0 (kafka.controller.KafkaController:107)
java.lang.NullPointerException
at kafka.controller.KafkaController$$anonfun$unregisterPartitionReassignmentIsrChangeHandlers$1.apply(KafkaController.scala:784)
at kafka.controller.KafkaController$$anonfun$unregisterPartitionReassignmentIsrChangeHandlers$1.apply(KafkaController.scala:783)
```
Author: Ismael Juma <ismael@juma.me.uk>
Reviewers: Jun Rao <junrao@gmail.com>
Closes#4219 from ijuma/fix-npe-unregister-zk-listener
Only expect responseAsString to be set if request logging is
enabled _and_ responseSend is defined.
Also fixed a couple of issues that would manifest themselves
if trace logging is enabled:
- `MemoryRecords.toString` should not throw exception if data is corrupted
- Generate `responseString` correctly if unsupported api versions request is
received.
Unit tests were added for every issue fixed. Also changed
SocketServerTest to run with trace logging enabled as
request logging breakage has been a common issue.
Author: Ismael Juma <ismael@juma.me.uk>
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>
Closes#4250 from ijuma/fix-issues-when-trace-logging-is-enabled
It's failing often and it seems like there are multiple
reasons. PR #4238 will re-enable it.
Author: Ismael Juma <ismael@juma.me.uk>
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>
Closes#4262 from ijuma/temporarily-disable-test-log-start-offset-checkpoint
Replaced ZkUtils with KafkaZkClient in ReplicaManager and Partition.
Relying on existing tests.
Author: tedyu <yuzhihong@gmail.com>
Reviewers: Jun Rao <junrao@gmail.com>, Manikumar Reddy <manikumar.reddy@gmail.com>, Ismael Juma <ismael@juma.me.uk>
Closes#4254 from tedyu/trunk
- Updated logging to use string interpolation
- Minor refactors
- Fixed a few typos
Author: Mickael Maison <mickael.maison@gmail.com>
Reviewers: Ismael Juma <ismael@juma.me.uk>
Closes#4231 from mimaison/controller_refactor
* Add AdminZkClient class
* Use KafkaZkClient, AdminZkClient in ConfigCommand, TopicCommand
* All the existing tests should work
Author: Manikumar Reddy <manikumar.reddy@gmail.com>
Reviewers: Ismael Juma <ismael@juma.me.uk>, Jun Rao <junrao@gmail.com>
Closes#4194 from omkreddy/KAFKA-5646-ZK-ADMIN-UTILS-DYNAMIC-MANAGER
Use slf4j (via scala-logging) instead. Also:
- Log4jController is only initialised if log4j if in the classpath
- Use FATAL marker to support log4j's FATAL level (as the log4j-slf4j bridge does)
- Removed `Logging.swallow` in favour of CoreUtils.swallow, which logs to the
correct logger
Author: Viktor Somogyi <viktor.somogyi@cloudera.com>
Reviewers: Ismael Juma <ismael@juma.me.uk>
Closes#3477 from viktorsomogyi/KAFKA-1044
Tested with DeleteRecordsRequestTest by Tom Bentley, which is part of
a separate pull request.
Author: tedyu <yuzhihong@gmail.com>
Reviewers: Tom Bentley <tbentley@redhat.com>, Ismael Juma <ismael@juma.me.uk>
Closes#4052 from tedyu/trunk
With the new consumer the "/consumers" path on Zookeeper isn't filled by consumer info. On closing the new consumer, there is some code that is useless to execute for trying to connect to Zookeeper (but the URL is null).
Author: ppatierno <ppatierno@live.com>
Author: Paolo Patierno <ppatierno@live.com>
Reviewers: Ismael Juma <ismael@juma.me.uk>
Closes#3301 from ppatierno/old-consumer-delete-groupid
This is the PR related to the [KIP-204](https://cwiki.apache.org/confluence/display/KAFKA/KIP-204+%3A+Adding+records+deletion+operation+to+the+new+Admin+Client+API) in order to add the `deleteRecords` operation to the new Admin Client (it's already available in the "legacy" one).
Other than that, unit test and integration tests are added as well (such integration tests come from the "legacy" integration tests in order to test the new addition in the same way as the "legacy" one).
Author: Paolo Patierno <ppatierno@live.com>
Reviewers: Colin P. Mccabe <cmccabe@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
Closes#4132 from ppatierno/kafka-5925
rather than using GetDataRequest
Author: Mickael Maison <mickael.maison@gmail.com>
Reviewers: Ismael Juma <ismael@juma.me.uk>
Closes#4217 from mimaison/pathExists
Added unit test for ApiVersion and testApiVersions from
Scala to Java.
Author: Ismael Juma <ismael@juma.me.uk>
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>
Closes#4220 from ijuma/kafka-6210-iae-if-1.0.0-inter-broker-protocol-version
When reviewing https://github.com/apache/kafka/pull/4132, I felt that NOT_ENOUGH_REPLICAS should never happen actually. Hence proposing to remove it from the listed error code as well in the broker-side capture clause.
Testing added in 4132 should have been sufficient.
Author: Guozhang Wang <wangguoz@gmail.com>
Reviewers: Ismael Juma <ismael@juma.me.uk>, Dong Lin <lindong28@gmail.com>
Closes#4208 from guozhangwang/KMinor-delete-records-error-code
Also use ZkVersion.NoVersion instead of -1.
Author: Mickael Maison <mickael.maison@gmail.com>
Reviewers: Ismael Juma <ismael@juma.me.uk>
Closes#4196 from mimaison/zkclient_refactor
Also rename UtilsTest to CoreUtilsTest and note
that `getOrElseUpdate` has the right behaviour
in Scala 2.12.
Author: Ismael Juma <ismael@juma.me.uk>
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>
Closes#4142 from ijuma/use-scala-futures-in-core-utils-test
This patch also adds the a test for test the log deletion after close.
Author: Dong Lin <lindong28@gmail.com>
Reviewers: Jiangjie Qin <becket.qin@gmail.com>, Ismael Juma <ismael@juma.me.uk>
Closes#4179 from lindong28/KAFKA-6175
Author: Onur Karaman <okaraman@linkedin.com>
Reviewers: Viktor Somogyi <viktorsomogyi@gmail.com>, Ismael Juma <ismael@juma.me.uk>
Closes#4198 from onurkaraman/make-controller-helper-methods-private
This memory leak could eventually lead to an OutOfMemoryError. This
was particularly likely in case of down conversions as the leaked
channels would hold on to the record batch (which is only loaded
into the heap in case of down conversions).
Author: Rajini Sivaram <rajinisivaram@googlemail.com>
Reviewers: Ismael Juma <ismael@juma.me.uk>
Closes#4193 from rajinisivaram/KAFKA-6185-oom
Author: Rajini Sivaram <rajinisivaram@googlemail.com>
Reviewers: Viktor Somogyi <viktorsomogyi@gmail.com>, Ismael Juma <ismael@juma.me.uk>
Closes#4181 from rajinisivaram/KAFKA-6164
We currently enqueue a PreferredReplicaLeaderElection controller event in PreferredReplicaElectionHandler's handleCreation, handleDeletion, and handleDataChange. We can just enqueue the event upon znode creation and after preferred replica leader election completes. The processing of this latter enqueue will register the exist watch on PreferredReplicaElectionZNode and perform any pending preferred replica leader election that may have occurred between completion and registration.
Author: Onur Karaman <okaraman@linkedin.com>
Reviewers: Jun Rao <junrao@gmail.com>, Ismael Juma <ismael@juma.me.uk>
Closes#4189 from onurkaraman/KAFKA-6146
- Eliminated all the unnecessary allocations of `TopicPartition` and
`TopicAndPartition` in the Controller. We now use the former
in the Controller (bringing it inline with the rest of the non legacy
code).
- Fixed missed `Listener` -> `Handler` renames for companion
objects.
- More String.format -> String interpolation conversions (the former
is roughly 5 times more expensive).
- Some other minor clean-ups.
Author: Ismael Juma <ismael@juma.me.uk>
Reviewers: Onur Karaman <okaraman@linkedin.com>, Viktor Somogyi <viktorsomogyi@gmail.com>
Closes#4152 from ijuma/controller-topic-partition-and-other-clean-ups
ZooKeeperClient is a zookeeper client that encourages pipelined requests to zookeeper. We want to add the notion of max inflight requests to the client for several reasons:
1. to bound memory overhead associated with async requests on the client.
2. to not overwhelm the zookeeper ensemble with a burst of requests.
Author: Onur Karaman <okaraman@linkedin.com>
Reviewers: Ismael Juma <ismael@juma.me.uk>, Ted Yu <yuzhihong@gmail.com>, Jun Rao <junrao@gmail.com>, Manikumar Reddy <manikumar.reddy@gmail.com>
Closes#3860 from onurkaraman/KAFKA-5894
We only need to generate the event when the znode is created or deleted.
In the former case, we start the reassignment while in the latter we
re-register the watcher (necessary for the Controller to detect future
reassignments).
During Controller failover, we restart the reassignment without generating
an event so it's not affected by this change.
Also use the Controller cache (`ControllerContext.partitionsBeingReassigned`)
in `removePartitionFromReassignedPartitions` instead of reloading the
data from ZooKeeper.
Overall, we would previously load the reassignment data from ZooKeeper twice
per completed partition whereas now as don't do it at all. As an example,
say there were 30k partitions being reassigned, these changes save the
allocation of 900 million `TopicAndPartition` and `Seq[Int]` (replicas)
instances (could easily amount to 20-40 GB depending on the topic name
length). This matters most in cases where the partitions being reassigned
don't have much data allowing the reassignment to complete reasonably
fast for many of the partitions.
Author: Ismael Juma <ismael@juma.me.uk>
Reviewers: Jun Rao <junrao@gmail.com>, Onur Karaman <okaraman@linkedin.com>
Closes#4143 from ijuma/partition-reassignment-ignore-handle-deletion-and-data-change
Author: Dong Lin <lindong28@gmail.com>
Reviewers: Ismael Juma <ismael@juma.me.uk>, Jiangjie Qin <becket.qin@gmail.com>
Closes#4177 from lindong28/KAFKA-6172
Windows directory paths often contain colons which are not allowed in
yammer metrics. Metric tag values with special characters must be
quoted.
Author: huxihx <huxi_2b@hotmail.com>
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>
Closes#4173 from huxihx/KAFKA-6156
This is a followup to #4137
Author: Apurva Mehta <apurva@confluent.io>
Reviewers: Ismael Juma <ismael@juma.me.uk>, Jason Gustafson <jason@confluent.io>
Closes#4146 from apurvam/MINOR-followups-to-bump-epoch-on-expire-patch
Author: Richard Yu <richardyu@Richards-Air.attlocal.net>
Reviewers: Jason Gustafson <jason@confluent.io>
Closes#4110 from ConcurrencyPractitioner/trunk
I kept zkUtils for the call to AdminUtils.createTopic(). AdminUtils can be done in another PR.
Is there a reason why we use TopicAndPartition instead of TopicPartition in KafkaControllerZkUtils ?
Author: Mickael Maison <mickael.maison@gmail.com>
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>, Ismael Juma <ismael@juma.me.uk>, Jun Rao <junrao@gmail.com>
Closes#4111 from mimaison/KAFKA-6073
Failure to close the producer could cause a transient failure, more details
below.
The request timeout was only 2 seconds, exceptions thrown were not
propagated and the producer would not be closed. If the exception
was thrown during `send`, we did not increment `numMessages`
allowing the test to pass.
I have increased the timeout to 10 seconds and made sure that
exceptions are propagated.
Example of the error:
```text
kafka.api.SaslSslAdminClientIntegrationTest > classMethod STARTED
kafka.api.SaslSslAdminClientIntegrationTest > classMethod FAILED
java.lang.AssertionError: Found unexpected threads, allThreads=Set(metrics-meter-tick-thread-2, Signal Dispatcher, main, Reference Handler, scala-execution-context-global-164, kafka-producer-network-thread | producer-1, scala-execution-context-global-166, Test worker, scala-execution-context-global-1249, /0:0:0:0:0:0:0:1:58910 to /0:0:0:0:0:0:0:1:43025 workers Thread 2, Finalizer, /0:0:0:0:0:0:0:1:58910 to /0:0:0:0:0:0:0:1:43025 workers Thread 3, scala-execution-context-global-163, metrics-meter-tick-thread-1)
```
Author: Ismael Juma <ismael@juma.me.uk>
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>
Closes#4144 from ijuma/ensure-producer-is-closed-test-alter-replica-log-dirs
Author: Manikumar Reddy <manikumar.reddy@gmail.com>
Reviewers: Tom Bentley <tbentley@redhat.com>, Ismael Juma <ismael@juma.me.uk>
Closes#3951 from omkreddy/KIP-103-DOCS
Author: Vahid Hashemian <vahidhashemian@us.ibm.com>
Reviewers: Ismael Juma <ismael@juma.me.uk>
Closes#4139 from vahidhashemian/minor/indentation_fix_1710
A description of the problem is in the JIRA. I have added an integration test which reproduces the original scenario, and also added unit test cases.
Author: Apurva Mehta <apurva@confluent.io>
Reviewers: Jason Gustafson <jason@confluent.io>, Ted Yu <yuzhihong@gmail.com>, Guozhang Wang <wangguoz@gmail.com>
Closes#4137 from apurvam/KAFKA-6119-bump-epoch-when-expiring-transactions
Author: Rajini Sivaram <rajinisivaram@googlemail.com>
Reviewers: Jason Gustafson <jason@confluent.io>, Guozhang Wang <wangguoz@gmail.com>, Ismael Juma <ismael@juma.me.uk>
Closes#4140 from rajinisivaram/KAFKA-6131-txn-concurrentmap
Even though this class is internal, it's widely
used by other projects and it's better to avoid
breaking them until we have a publicly supported
test library.
Author: Ismael Juma <ismael@juma.me.uk>
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>
Closes#4138 from ijuma/revert-embedded-zookeeper-rename
- kafka.controller.ZookeeperClient -> kafka.zookeeper.ZooKeeperClient
- kafka.controller.ControllerZkUtils -> kafka.zk.KafkaZkClient
- kafka.controller.ZkData -> kafka.zk.ZkData
- Renamed various fields to match new names and for consistency
- A few clean-ups in ZkData
- Document intent
Author: Ismael Juma <ismael@juma.me.uk>
Reviewers: Onur Karaman <okaraman@linkedin.com>, Manikumar Reddy <manikumar.reddy@gmail.com>, Jun Rao <junrao@gmail.com>
Closes#4112 from ijuma/rename-zookeeper-client-and-move-to-zookeper-package
* Fix issue in `retryRequestsUntilConnected` where the same response
could appear multiple times (implies that we are lacking test coverage)
* Introduce type member in AsyncRequest for the AsyncResponse
type and refactor the code to eliminate most downcasts
* Remove a number of unnecessary collection copies in
`retryRequestsUntilConnected`
* Move ControllerContext to its own file
* Rename getACL/setACL to getAcl/setAcl to match Kafka naming
convention
* Replace tuple of 3 elements with case class in one place (we
should do this in other places too)
* Extract `send` and `shouldWatch` from
`ZooKeeperClient.handleRequests`
* Use pattern matching instead of if/else chains in a few places (we
should do it in more places)
* A couple of renames to avoid overloads and hence benefit from
better type inference
* Use Option and default arguments instead of passing null in
some places
* `Expired` is no longer a case class since it has no parameters,
but it has state
* Various minor clean-ups
Author: Ismael Juma <ismael@juma.me.uk>
Reviewers: Jun Rao <junrao@gmail.com>, Onur Karaman <okaraman@linkedin.com>
Closes#4088 from ijuma/async-zkclient-cleanups