NetworkClient's logging improvements:
- Include correlation id in a number of log statements
- Avoid eager toString call in parameter passed to log.debug
- Use node.toString instead of passing a subset of fields to the
logger
- Use requestBuilder instead of clientRequest in one of the log
statements
Author: Ismael Juma <ismael@juma.me.uk>
Reviewers: Damian Guy <damian.guy@gmail.com>, Jason Gustafson <jason@confluent.io>
Closes#3420 from ijuma/kafka-5506-offset-fetch-request-to-string-npe
-Tests for rolling upgrades for a streams app (keeping broker config fixed)
-Tests for rolling upgrades of brokers (keeping streams app config fixed)
Author: Eno Thereska <eno.thereska@gmail.com>
Reviewers: Matthias J. Sax <matthias@confluent.io>, Damian Guy <damian.guy@gmail.com>
Closes#3411 from enothereska/KAFKA-5487-upgrade-test-streams
These are the following improvements I made:
1. On stream thread level, INFO will be demonstrating `Completed xx tasks in yy ms` or `Completed rebalance with xx state in yy ms`,
2. On Stream thread cache level, INFO on `Flushed xx records`.
3. On Stream thread level, DEBUG on internal batched operations like `created xx tasks`, and TRACE on individual operation like `created x task`.
4. Also using `isTraceEnabled` on the critical path to reduce overhead of creating `Object[]`.
5. Minor cleanups in the code.
Author: Guozhang Wang <wangguoz@gmail.com>
Reviewers: Steven Schlansker, Nicolas Fouché, Kamal C, Ismael Juma, Bill Bejeck, Eno Thereska, Matthias J. Sax, Damian Guy
Closes#3354 from guozhangwang/K4829-tasks-log4j
After we call `release`, we should null out the reference so
that we neither use it or release it a second time.
This should fix the following exception that has been reported:
```text
[2017-06-23 03:24:02,485] ERROR stream-thread [...] Failed to close consumer: (org.apache.kafka.streams.processor.internals.StreamThread:1054)
org.apache.kafka.common.KafkaException: Failed to close kafka consumer
at org.apache.kafka.clients.consumer.KafkaConsumer.close(KafkaConsumer.java:1623)
at org.apache.kafka.clients.consumer.KafkaConsumer.close(KafkaConsumer.java:1573)
at org.apache.kafka.clients.consumer.KafkaConsumer.close(KafkaConsumer.java:1549)
at org.apache.kafka.streams.processor.internals.StreamThread.shutdown(StreamThread.java:1052)
at org.apache.kafka.streams.processor.internals.StreamThread.run(StreamThread.java:538)
Caused by: java.lang.IllegalStateException: release called on LoginManager with refCount == 0
at org.apache.kafka.common.security.authenticator.LoginManager.release(LoginManager.java:106)
at org.apache.kafka.common.network.SaslChannelBuilder.close(SaslChannelBuilder.java:125)
at org.apache.kafka.common.network.Selector.close(Selector.java:257)
at org.apache.kafka.clients.NetworkClient.close(NetworkClient.java:505)
at org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.close(ConsumerNetworkClient.java:439)
at org.apache.kafka.clients.ClientUtils.closeQuietly(ClientUtils.java:71)
at org.apache.kafka.clients.consumer.KafkaConsumer.close(KafkaConsumer.java:1613)
```
It's worth noting that it's not clear how `SaslChannelBuilder.close()` is called more than
once and it would be good to understand that as well.
Author: Ismael Juma <ismael@juma.me.uk>
Reviewers: Damian Guy <damian.guy@gmail.com>, Jason Gustafson <jason@confluent.io>
Closes#3422 from ijuma/sasl-channel-builder-idempotent
During a broker rolling upgrade, it's likely we don't have enough brokers ready yet. If streams does not need to create a topic it shouldn't check how many brokers are up.
The system test for this is in a separate PR: https://github.com/apache/kafka/pull/3411
Author: Eno Thereska <eno.thereska@gmail.com>
Reviewers: Damian Guy <damian.guy@gmail.com>
Closes#3418 from enothereska/hotfix-replication
In the stream library there are few cases where we don't leverage on logging methods features (i.e. using {} placeholder instead of string concatenation or passing the exception variable)
Author: ppatierno <ppatierno@live.com>
Reviewers: Damian Guy <damian.guy@gmail.com>
Closes#3419 from ppatierno/streams-consistent-logging
Author: Jeyhun Karimov <je.karimov@gmail.com>
Reviewers: Matthias J. Sax <matthias@confluent.io>, Damian Guy <damian.guy@gmail.com>
Closes#3294 from jeyhunkarimov/KAFKA-4653
Author: Jeyhun Karimov <je.karimov@gmail.com>
Reviewers: Matthias J. Sax <matthias@confluent.io>, Eno Thereska <eno.thereska@gmail.com>, Damian Guy <damian.guy@gmail.com>
Closes#3292 from jeyhunkarimov/KAFKA-4656
Throw IllegalArgumentException when attempting to create a `WindowStore` via `Stores` or directly with `RocksDBWindowStoreSupplier` when it has less than 2 segments.
Author: Damian Guy <damian.guy@gmail.com>
Reviewers: Eno Thereska <eno.thereska@gmail.com>, Matthias J. Sax <matthias@confluent.io>, Bill Bejeck <bbejeck@gmail.com>, Guozhang Wang <wangguoz@gmail.com>
Closes#3410 from dguy/kafka-4913
Author: Matthias J. Sax <matthias@confluent.io>
Reviewers: Joseph Rea <jrea@users.noreply.github.com>, Ewen Cheslack-Postava <ewen@confluent.io>
Closes#2575 from mjsax/minor-update-system-test-readme
Author: Matthias J. Sax <matthias@confluent.io>
Reviewers: Bill Bejeck <bbejeck@gmail.com>, Eno Thereska <eno.thereska@gmail.com>, Ismael Juma <ismael@juma.me.uk>
Closes#3414 from mjsax/hotfix-commit-logging
Dong Lin's testing of the 0.11.0 release revealed a controller-side performance regression in clusters with many brokers and many partitions when bringing up many brokers simultaneously.
The regression is caused by KAFKA-5028: a Watcher receives WatchedEvent notifications from the raw ZooKeeper client EventThread. A WatchedEvent only contains the following information:
- KeeperState
- EventType
- path
Note that it does not actually contain the current data or current set of children associated with the data/child change notification. It is up to the user to do this lookup to see the current data or set of children.
ZkClient is itself a Watcher. When it receives a WatchedEvent, it puts a ZkEvent into its own queue which its own ZkEventThread processes. Users of ZkClient interact with these notifications through listeners (IZkDataListener, IZkChildListener). IZkDataListener actually expects as input the current data of the watched znode, and likewise IZkChildListener actually expects as input the current set of children of the watched znode. In order to provide this information to the listeners, the ZkEventThread, when processing the ZkEvent in its queue, looks up the information (either the current data or current set of children) simultaneously sets up the next watch, and passes the result to the listener.
The regression introduced in KAFKA-5028 is the time at which we lookup the information needed for the event processing.
In the past, the lookup from the ZkEventThread during ZkEvent processing would be passed into the listener which is processed immediately after. For instance in ZkClient.fireChildChangedEvents:
```
List<String> children = getChildren(path);
listener.handleChildChange(path, children);
```
Now, however, there are multiple listeners that pass information looked up by the ZkEventThread into a ControllerEvent which gets processed potentially much later. For instance in BrokerChangeListener:
```
class BrokerChangeListener(controller: KafkaController) extends IZkChildListener with Logging {
override def handleChildChange(parentPath: String, currentChilds: java.util.List[String]): Unit = {
import JavaConverters._
controller.addToControllerEventQueue(controller.BrokerChange(currentChilds.asScala))
}
}
```
In terms of impact, this:
- increases the odds of working with stale information by the time the ControllerEvent gets processed.
- can cause the cluster to take a long time to stabilize if you bring up many brokers simultaneously.
In terms of how to solve it:
- (short term) just ignore the ZkClient's information lookup and repeat the lookup at the start of the ControllerEvent. This is the approach taken in this ticket.
- (long term) try to remove a queue. This basically means getting rid of ZkClient. This is likely the approach that will be taken in KAFKA-5501.
Author: Onur Karaman <okaraman@linkedin.com>
Reviewers: Ismael Juma <ismael@juma.me.uk>, Jun Rao <junrao@gmail.com>
Closes#3413 from onurkaraman/KAFKA-5502
The actual fix for KAFKA-5490 is in
https://github.com/apache/kafka/pull/3406.
This is just the consumer change that will allow the cleaner
to use empty record batches without breaking 0.11.0.0
consumers (assuming that KAFKA-5490 does not make the cut).
This is a safe change even if we decide to go with a different option
for KAFKA-5490 and I'd like to include it in RC2.
Author: Jason Gustafson <jason@confluent.io>
Author: Ismael Juma <ismael@juma.me.uk>
Reviewers: Damian Guy <damian.guy@gmail.com>, Ismael Juma <ismael@juma.me.uk>
Closes#3408 from ijuma/kafka-5490-consumer-should-skip-empty-batches
Author: Jeyhun Karimov <je.karimov@gmail.com>
Reviewers: Matthias J. Sax <matthias@confluent.io>, Damian Guy <damian.guy@gmail.com>
Closes#3293 from jeyhunkarimov/KAFKA-4658
Author: Jeyhun Karimov <je.karimov@gmail.com>
Reviewers: Matthias J. Sax <matthias@confluent.io>, Damian Guy <damian.guy@gmail.com>
Closes#3290 from jeyhunkarimov/KAFKA-4655
It describes the old deprecated clients and it's better to just
remove it.
The contribution is my original work and I license the work to the
project under the project's open source license.
Author: Tom Bentley <tbentley@redhat.com>
Reviewers: Ismael Juma <ismael@juma.me.uk>
Closes#3385 from tombentley/KAFKA-4059
The current config sends org.apache.kafka and any unspecified logger to
stdout. They should go to `server.log` instead.
Author: Ismael Juma <ismael@juma.me.uk>
Reviewers: Damian Guy <damian.guy@gmail.com>
Closes#3402 from ijuma/kafka-5486-org.apache.kafka-logging-server.log
Author: Jeyhun Karimov <je.karimov@gmail.com>
Reviewers: Matthias J. Sax <matthias@confluent.io>, Eno Thereska <eno.thereska@gmail.com>, Bill Bejeck <bbejeck@gmail.com>, Damian Guy <damian.guy@gmail.com>
Closes#3106 from jeyhunkarimov/KAFKA-4785
Author: Guozhang Wang <wangguoz@gmail.com>
Reviewers: Matthias J Sax <matthias@confluent.io>, Bill Bejeck <bbejeck@gmail.com>
Closes#3403 from guozhangwang/KMinor-turn-off-caching-in-demo
Author: Jeyhun Karimov <je.karimov@gmail.com>
Reviewers: Matthias J Sax <matthias@confluent.io>, Guozhang Wang <wangguoz@gmail.com>, Damian Guy <damian.guy@gmail.com>
Closes#3291 from jeyhunkarimov/KAFKA-4659
<strike> - actual fix for `StandbyTask#commit()` </strike>
Additionally (for debugging):
- EOS test, does not report "expected" value correctly
- add `IntegerDecoder` (to be use with `kafka.tools.DumpLogSegments`)
- add test for `StreamTask` to not checkpoint on commit if EOS enabled
Author: Matthias J. Sax <matthias@confluent.io>
Reviewers: Bill Bejeck <bbejeck@gmail.com>, Damian Guy <damian.guy@gmail.com>, Guozhang Wang <wangguoz@gmail.com>
Closes#3375 from mjsax/kafka-5474-eos-standby-task
With this patch, the `ProducePerfomance` tool can create transactions of differing durations.
This patch was used to to collect the initial set of benchmarks for transaction performance, documented here: https://docs.google.com/spreadsheets/d/1dHY6M7qCiX-NFvsgvaE0YoVdNq26uA8608XIh_DUpI4/edit#gid=282787170
Author: Apurva Mehta <apurva@confluent.io>
Reviewers: Jun Rao <junrao@gmail.com>
Closes#3400 from apurvam/MINOR-add-transaction-size-to-producre-perf
This patch lowers the retry backoff when receiving a CONCURRENT_TRANSACTIONS error from an AddPartitions request. The default of 100ms would mean that back to back transactions would be 100ms long at minimum, making things to slow.
Author: Apurva Mehta <apurva@confluent.io>
Reviewers: Guozhang Wang <wangguoz@gmail.com>, Ismael Juma <ismael@juma.me.uk>, Jason Gustafson <jason@confluent.io>
Closes#3377 from apurvam/HOTFIX-lower-retry-for-add-partitions
Author: Matthias J. Sax <matthias@confluent.io>
Reviewers: Bill Bejeck <bbejeck@gmail.com>, Damian Guy <damian.guy@gmail.com>
Closes#3378 from mjsax/minor-producer-naming
the contribution is my original work and I license the work to the project under the project's open source license.
junrao , I had already made the code change before your last comment. I've done pretty much what you said, except that I've not used the current segment because I wasn't sure if it will always be available.
I'm happy to change it if you prefer.
I've run all the unit and integration tests which all passed.
Author: Kelvin Rutt <ruttkelvin@gmail.com>
Author: Kelvin Rutt <kelvin.rutt@sky.com>
Reviewers: Jun Rao <junrao@gmail.com>
Closes#3357 from kelvinrutt/kafka_5413_bugfix
Kafka Connect was adding duplicate group names in the response from the REST API's validation of connector configurations. This fixes the duplicates and maintains the order of the `ConfigDef` objects so that the `ConfigValue` results are in the same order.
This is a blocker and should be merged to 0.11.0.
Author: Randall Hauch <rhauch@gmail.com>
Reviewers: Ewen Cheslack-Postava <ewen@confluent.io>
Closes#3379 from rhauch/KAFKA-5472
Author: Jason Gustafson <jason@confluent.io>
Reviewers: Apurva Mehta <apurva@confluent.io>, Ismael Juma <ismael@juma.me.uk>, Guozhang Wang <wangguoz@gmail.com>
Closes#3388 from hachikuji/KAFKA-5021
Author: Guozhang Wang <wangguoz@gmail.com>
Reviewers: Apurva Mehta <apurva@confluent.io>, Matthias J. Sax <matthias@confluent.io>, Jason Gustafson <jason@confluent.io>
Closes#3345 from guozhangwang/KMinor-streams-eos-docs
based on conversations with vahidhashemian rajinisivaram apurvam
The docs didn't make clear that what gets committed and what gets not may depend on the producer acks.
Author: Edoardo Comar <ecomar@uk.ibm.com>
Reviewers: Vahid Hashemian <vahidhashemian@us.ibm.com>, Apurva Mehta <apurva@confluent.io>, Jason Gustafson <jason@confluent.io>
Closes#3035 from edoardocomar/DOC-clarification-on-committed
See https://github.com/mitchellh/vagrant/issues/7531. The core of the issue is that vagrant rsync uses a fixed set of 1000 possible temp file entries for SSH ControlMaster files to cache SSH connections for rsyncing. A few notes:
* We can't break down the steps further and maintain performance due to various limitations in vagrant/vagrant-aws (rsync is only executed on `vagrant up`/`vagrant reload`/`vagrant rsync`, you can't enable/disable and rsync shared folder only during some of those stages, and provisioning only runs in parallel with vagrant-aws during `vagrant up`).
* We need to isolate each of the serialized rsync calls. (If we assumed `parallel` was available, we actually could get the parallelism back.) This is required because even across calls they could randomly choose the same temporary file.
* If there's a chance multiple instances were running on the same server at the same or nearly the same time, they can conflict since the same temp file entries are used globally. This means anything running on shared CI servers might end up syncing data between different CI jobs (!!), which could lead to some very strange results. Especially weird if they aren't even for the same type of job.
* Provisioning error check needs to be removed because it is catching rsync errors, but those can still happen in the initial `vagrant up` rsync step before the `vagrant up` provisioning step. It seems likely this bug was the cause of missing files anyway so this check might not be as valuable anymore.
Author: Ewen Cheslack-Postava <me@ewencp.org>
Reviewers: Ismael Juma <ismael@juma.me.uk>
Closes#3380 from ewencp/deparallelize-rsync
Author: Apurva Mehta <apurva@confluent.io>
Reviewers: Guozhang Wang <wangguoz@gmail.com>, Jason Gustafson <jason@confluent.io>
Closes#3278 from apurvam/MINOR-add-logging-to-transaction-coordinator-in-all-failure-cases
These are internal methods with no tests and since we now have an `AdminClient`,
we should remove them.
Author: Ismael Juma <ismael@juma.me.uk>
Reviewers: Jason Gustafson <jason@confluent.io>
Closes#3299 from ijuma/remove-unused-admin-utils-methods
This is contributed by Hao Chen.
Author: Guozhang Wang <wangguoz@gmail.com>
Reviewers: Guozhang Wang <wangguoz@gmail.com>, Ismael Juma <ismael@juma.me.uk>
Closes#3366 from guozhangwang/KMinor-fix-streams-link-in-uses
log info typos. should be standby task, not active task.
Author: zqhxuyuan <qihuang.zheng@fraudmetrix.cn>
Reviewers: Guozhang Wang <wangguoz@gmail.com>, Matthias J. Sax <matthias@confluent.io>
Closes#3362 from zqhxuyuan/trunk
Author: Apurva Mehta <apurva@confluent.io>
Reviewers: Ismael Juma <ismael@juma.me.uk>, Jason Gustafson <jason@confluent.io>
Closes#3353 from apurvam/KAFKA-5455-proper-javadocs-eos-clients
Author: Jason Gustafson <jason@confluent.io>
Reviewers: Apurva Mehta <apurva@confluent.io>, Jun Rao <junrao@gmail.com>
Closes#3361 from hachikuji/KAFKA-5435-ALT
- Use ResourceType.toJava instead of ResourceType.fromString. The latter
doesn't work for TransactionalId (or any type with two camel-case
words).
- Replace Throwable with ApiError in response classes.
- Return InvalidRequest instead of Unknown error if ANY or UNKNOWN
are provided during ACL creation.
- Rename `unknown()` to `isUnknown()` in a few places that
were missed previously.
- Add tests.
Author: Ismael Juma <ismael@juma.me.uk>
Reviewers: Jason Gustafson <jason@confluent.io>
Closes#3364 from ijuma/acls-fromString-fixes
Before:
```
pri=TRACE t=Controller-1-to-broker-0-send-thread at=logger Controller 1 epoch 1 received response {error_code=0} for a request sent to broker <ip>:<port> (id: 0 rack: null)
```
After:
```
pri=TRACE t=Controller-1-to-broker-0-send-thread at=logger Controller 1 epoch 1 received response {error_code=0} for a request sent to broker <ip>:<port> (id: 0 rack: us-east-1d)
```
Author: Jeff Chao <jeffchao@me.com>
Reviewers: Onur Karaman <okaraman@linkedin.com>, Ismael Juma <ismael@juma.me.uk>
Closes#3358 from jeffchao/fix-controller-rack-aware-logging