This patch ensures that truncation to an empty segment forces resizing of the index file in order to prevent premature rolling.
I have added unit tests which verify that appends are permitted following truncation to an empty segment. Without the fix, this test case reproduces the failure in which the rolled segment matches the current active segment.
Author: Jason Gustafson <jason@confluent.io>
Reviewers: Ismael Juma <ismael@juma.me.uk>, Jiangjie (Becket) Qin <becket.qin@gmail.com>
Closes#4498 from hachikuji/KAFKA-6492
Remove caching of brokerId in DynamicBrokerConfig constructor and delay initialization until brokerId is set in KafkaConfig.
Reviewers: Jason Gustafson <jason@confluent.io>
Dynamic metrics reporter updates described in KIP-226. This includes:
- Addition and removal of metrics reporters
- Reconfiguration of custom metrics reporter configs
- Tests for metrics reporter updates at default cluster-level and as per-broker config for testing
Reviewers: Jason Gustafson <jason@confluent.io>
Dynamic resize of broker thread pools as described in KIP-226:
- num.network.threads
- num.io.threads
- num.replica.fetchers
- num.recovery.threads.per.data.dir
- background.threads
Reviewers: Jason Gustafson <jason@confluent.io>
The cause for compilation error in JDK 9.0 was an ambiguity issue in scalac:
```
both method putAll in class Properties of type (x$1: java.util.Map[_, _])Unit
and method putAll in class Hashtable of type (x$1: java.util.Map[_ <: Object, _ <: Object])Unit
match argument types (java.util.Properties)
newProps.putAll(props)
```
Author: RichardYuSTUG <yohan.richard.yu2@gmail.com>
Reviewers: Manikumar Reddy O <manikumar.reddy@gmail.com>, Rajini Sivaram <rajinisivaram@googlemail.com>
Closes#4482 from ConcurrencyPractitioner/trunk
In MetadataResponse deserialization, if the partition leader key is set
to -1, the leader is set to null. The MetadataResponse#toStruct code
should handle this correctly as well.
Also fix a case in KafkaApis where we were not taking into account the
possibility of the leader being null.
RequestResponseTest should test this as well.
Reviewers: Ismael Juma <ismael@juma.me.uk>, Jason Gustafson <jason@confluent.io>
- Use newly added pause method in LogCleaner and ControllerChannelManager classes
- Remove LogCleaner, Cleaner exclusions from findbugs-exclude.xml
Reviewers: Ismael Juma <ismael@juma.me.uk>, Jason Gustafson <jason@confluent.io>
…for dirty non-active segments only when `log.cleaner.min.compaction.lag.ms` is greater than 0
With `log.cleaner.min.compaction.lag.ms`'s default value of 0, there is no need to hold heap objects for those dirty non-active segments. This could reduce the heap size and also avoid the unnecessary monitor lock retrieval.
With this patch, simple consumer groups which only use Kafka for offset storage will be viewable using the `--list` option in consumer-groups.sh. In addition, this patch fixes a bug in the offset loading logic which caused us to lose the protocol type of empty groups on coordinator failover. I also did some cleanup of the various consumer group command test cases.
For testing, I have added new integration tests which cover listing and describing simple consumer groups. I also added unit tests to cover loading empty groups with assertions on the protocol type.
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>
Enable dynamic broker configuration (see KIP-226 for details). Includes
- Base implementation to allow specific broker configs and custom configs to be dynamically updated
- Extend DescribeConfigsRequest/Response to return all synonym configs and their sources in the order of precedence
- Extend AdminClient to alter dynamic broker configs
- Dynamic update of SSL keystores
Reviewers: Ted Yu <yuzhihong@gmail.com>, Jason Gustafson <jason@confluent.io>
add check to KafkaApis, add unit test specific to follower fetch
developed with @mimaison
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Ismael Juma <ismael@juma.me.uk>, Rajini Sivaram <rajinisivaram@googlemail.com>
- Add capability to create delegation token
- Add authentication based on delegation token.
- Add capability to renew/expire delegation tokens.
- Add units tests and integration tests
Author: Manikumar Reddy <manikumar.reddy@gmail.com>
Reviewers: Jun Rao <junrao@gmail.com>
Closes#3616 from omkreddy/KAFKA-4541
This is the implementation of KIP-225.
It marks the previous metrics as deprecated in the documentation and adds new metrics using tags.
Testing verifies that both the new and the old metric report the same value.
Author: cmolter <cmolter@apple.com>
Reviewers: Jiangjie (Becket) Qin <becket.qin@gmail.com>
Closes#4362 from lahabana/kafka-5890
- Add missing locking/volatile in MetadataCache.aliveEndPoint
- Fix topic metadata not to throw BrokerNotAvailableException
when listeners are inconsistent. Add test verifying the fix. As
part of this fix, renamed Broker methods to follow Map
convention where the `get` version returns `Option`.
Reviewers: Jason Gustafson <jason@confluent.io>
- Fix zk session state and session change rate metric names: type
should be SessionExpireListener instead of KafkaHealthCheck. Test
verifying the fix was included.
- Handle missing controller in controlled shutdown in the same way as if
the broker is not registered (i.e. retry after backoff).
- Restructure BrokerInfo to reduce duplication. It now contains a
Broker instance and the JSON serde is done in BrokerIdZNode
since `Broker` does not contain all the fields.
- Remove dead code from `ZooKeeperClient.initialize` and remove
redundant `close` calls.
- Move ACL handling and persistent paths definition from ZkUtils to
ZkData (and call ZkData from ZkUtils).
- Remove ZooKeeperClientWrapper and ZooKeeperClientMetrics from
ZkUtils (avoids metrics clash if third party users create a ZkUtils
instance in the same process as the broker).
- Introduce factory method in KafkaZkClient that creates
ZooKeeperClient and remove metric name defaults from
ZooKeeperClient.
- Fix a few instances where ZooKeeperClient was not closed in tests.
- Update a few TestUtils methods to use KafkaZkClient instead of
ZkUtils.
- Add test verifying SessionState metric.
- Various clean-ups.
Testing: mostly relying on existing tests, but added a couple
of new tests as mentioned above.
Author: Ismael Juma <ismael@juma.me.uk>
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>, Jun Rao <junrao@gmail.com>
Closes#4359 from ijuma/kafka-6320-kafka-health-zk-metrics-follow-up
Updates:
- Gradle, gradle plugins and maven artifact updated
- Bug fix updates for ZooKeeper, Jackson, EasyMock and Snappy
Not updated:
- RocksDB as it often causes issues, so better done separately
- args4j as our test coverage is weak and the update was a
feature release
Also fixed scala-reflect version to match scala-library.
Release notes for ZooKeeper 3.4.11:
https://zookeeper.apache.org/doc/r3.4.11/releasenotes.html
A notable fix is improved handling of UnknownHostException:
https://issues.apache.org/jira/browse/ZOOKEEPER-2614
Manually tested that IntelliJ import and build still works.
Relying on existing test suite otherwise.
Reviewers: Jun Rao <junrao@gmail.com>
* Moved metrics in KafkaHealthCheck to ZookeeperClient.
* Converted remaining ZkUtils usage in KafkaServer to ZookeeperClient and removed ZkUtils from KafkaServer.
* Made the re-creation of ZooKeeper during ZK session expiration with infinite retries.
* Added unit tests for all new methods in KafkaZkClient.
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>, Ismael Juma <ismael@juma.me.uk>
* Use KafkaZkClient in ReassignPartitionsCommand
* Use KafkaZkClient in PreferredReplicaLeaderElectionCommand
* Updated test classes to use new methods
* All existing tests should pass
Author: Manikumar Reddy <manikumar.reddy@gmail.com>
Reviewers: Jun Rao <junrao@gmail.com>
Closes#4260 from omkreddy/KAFKA-5647-ADMINCOMMANDS
1. Create default internal topic configs in StreamsConfig, especially for repartition topics change the segment size and time to smaller value.
2. Consolidate the default internal topic settings to InternalTopicManager and simplify InternalTopicConfig correspondingly.
3. Add an integration test for purging data.
4. MINOR: change TopologyBuilderException to IllegalStateException in StreamPartitionAssignor (part of https://issues.apache.org/jira/browse/KAFKA-5660).
Here are a few public facing APIs that get added:
1. AbstractConfig#originalsWithPrefix(String prefix, boolean strip): this for simplify the logic of passing admin and topic prefixed configs to consumer properties.
2. KafkaStreams constructor with Time object for convienent mocking in tests.
Will update KIP-204 accordingly if people re-votes these changes.
Author: Guozhang Wang <wangguoz@gmail.com>
Author: Matthias J. Sax <matthias@confluent.io>
Reviewers: Matthias J. Sax <matthias@confluent.io>, Bill Bejeck <bill@confluent.io>, Damian Guy <damian.guy@gmail.com>
Closes#4315 from guozhangwang/K6150-segment-size
(WIP: this commit isn't ready to be reviewed yet. I was checking the travis-ci build with the configuration changes in my account and opened the PR prematurely against trunk. I will make it consistent with Contribution guidelines once it's well tested.)
https://issues.apache.org/jira/browse/KAFKA-5473
Design:
`zookeeper.connection.retry.timeout.ms` => this determines how long to wait before triggering the shutdown. The default is 60000ms.
Currently the implementation only handles the `handleSessionEstablishmentError` by waiting for the sessionTimeout.
Author: Prasanna Gautam <prasannagautam@gmail.com>
Reviewers: Ismael Juma <ismael@juma.me.uk>, Jun Rao <junrao@gmail.com>
Closes#3990 from prasincs/KAFKA-5473
The `--describe` option of ConsumerGroupCommand is expanded, as proposed in [KIP-175](https://cwiki.apache.org/confluence/display/KAFKA/KIP-175%3A+Additional+%27--describe%27+views+for+ConsumerGroupCommand), to support:
* `--describe` or `--describe --offsets`: listing of current group offsets
* `--describe --members` or `--describe --members --verbose`: listing of group members
* `--describe --state`: group status
Example: With a single partition topic `test1` and a double partition topic `test2`, consumers `consumer1` and `consumer11` subscribed to `test`, consumers `consumer2` and `consumer22` and `consumer222` subscribed to `test2`, and all consumers belonging to group `test-group`, this is an output example of the new options above for `test-group`:
```
--describe, or --describe --offsets:
TOPIC PARTITION CURRENT-OFFSET LOG-END-OFFSET LAG CONSUMER-ID HOST CLIENT-ID
test2 0 0 0 0 consumer2-bad9496d-0889-47ab-98ff-af17d9460382 /127.0.0.1 consumer2
test2 1 0 0 0 consumer22-c45e6ee2-0c7d-44a3-94a8-9627f63fb411 /127.0.0.1 consumer22
test1 0 0 0 0 consumer1-d51b0345-3194-4305-80db-81a68fa6c5bf /127.0.0.1 consumer1
```
```
--describe --members
CONSUMER-ID HOST CLIENT-ID #PARTITIONS
consumer2-bad9496d-0889-47ab-98ff-af17d9460382 /127.0.0.1 consumer2 1
consumer222-ed2108cd-d368-41f1-8514-5b72aa835bcc /127.0.0.1 consumer222 0
consumer11-dc8295d7-8f3f-4438-9b11-7270bab46760 /127.0.0.1 consumer11 0
consumer22-c45e6ee2-0c7d-44a3-94a8-9627f63fb411 /127.0.0.1 consumer22 1
consumer1-d51b0345-3194-4305-80db-81a68fa6c5bf /127.0.0.1 consumer1 1
```
```
--describe --members --verbose
CONSUMER-ID HOST CLIENT-ID #PARTITIONS ASSIGNMENT
consumer2-bad9496d-0889-47ab-98ff-af17d9460382 /127.0.0.1 consumer2 1 test2(0)
consumer222-ed2108cd-d368-41f1-8514-5b72aa835bcc /127.0.0.1 consumer222 0 -
consumer11-dc8295d7-8f3f-4438-9b11-7270bab46760 /127.0.0.1 consumer11 0 -
consumer22-c45e6ee2-0c7d-44a3-94a8-9627f63fb411 /127.0.0.1 consumer22 1 test2(1)
consumer1-d51b0345-3194-4305-80db-81a68fa6c5bf /127.0.0.1 consumer1 1 test1(0)
```
```
--describe --state
COORDINATOR (ID) ASSIGNMENT-STRATEGY STATE #MEMBERS
localhost:9092 (0) range Stable 5
```
Note that this PR also addresses the issue reported in [KAFKA-6158](https://issues.apache.org/jira/browse/KAFKA-6158) by dynamically setting the width of columns `TOPIC`, `CONSUMER-ID`, `HOST`, `CLIENT-ID` and `COORDINATOR (ID)`. This avoid truncation of column values when they go over the current fixed width of these columns.
The code has been restructured to better support testing of individual values and also the console output. Unit tests have been updated and extended to take advantage of this restructuring.
Author: Vahid Hashemian <vahidhashemian@us.ibm.com>
Reviewers: Jason Gustafson <jason@confluent.io>
Closes#4271 from vahidhashemian/KAFKA-5526
- 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