Commit Graph

4216 Commits

Author SHA1 Message Date
Ismael Juma 27336192ff MINOR: Include response in request log
It's implemented such that there is no overhead if request logging is
disabled.

Also:
- Reduce metrics computation duplication in `updateRequestMetrics`
- Change a couple of log calls to use string interpolation instead of `format`
- Fix a few compiler warnings related to unused imports and unused default
arguments.

Author: Ismael Juma <ismael@juma.me.uk>

Reviewers: Roger Hoover <roger.hoover@gmail.com>, Jason Gustafson <jason@confluent.io>, Rajini Sivaram <rajinisivaram@googlemail.com>

Closes #3801 from ijuma/log-response-in-request-log
2017-09-08 04:40:24 +01:00
Ismael Juma 07a428e0c8 MINOR: Always specify the keystore type in system tests
Also throw an exception if a null keystore type is seen
in `SecurityStore`. This should never happen.

The default keystore type has changed in Java 9 (
http://openjdk.java.net/jeps/229), so we need to
be explicit to have consistent behaviour across
Java versions.

Author: Ismael Juma <ismael@juma.me.uk>

Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>

Closes #3808 from ijuma/set-jks-explicitly-in-system-tests
2017-09-08 02:29:03 +01:00
Ted Yu a1ec6527db KAFKA-5820: Remove unneeded synchronized keyword in StreamThread
I removed synchronized keyword from 3 methods.

I ran the change thru streams module where test suite passed.

Author: tedyu <yuzhihong@gmail.com>

Reviewers: Viktor Somogyi <viktorsomogyi@gmail.com>, Guozhang Wang <wangguoz@gmail.com>

Closes #3777 from tedyu/trunk
2017-09-07 17:08:45 -07:00
Guozhang Wang 674407908b KAFKA-5698: Sort processor nodes based on its sub-tree size
1. Sort processor nodes within a sub-topology by its sub-tree size: nodes with largest sizes are source nodes and hence printed earlier.

2. Sort sub-topologies by ids; sort global stores by the source topic names.

3. Open for discussion: start newlines for predecessor and successor.

4. Minor: space between processor nodes and stores / topics; maintain `[]` for the topic names.

Author: Guozhang Wang <wangguoz@gmail.com>

Reviewers: Damian Guy <damian.guy@gmail.com>, Matthias J. Sax <matthias@confluent.io>, Ted Yu <yuzhihong@gmail.com>

Closes #3618 from guozhangwang/K5698-topology-description-sorting
2017-09-07 14:36:49 -07:00
Vahid Hashemian 3410f02fec KAFKA-4585: Lower the Minimum Required ACL Permission of OffsetFetch (KIP-163)
Details can be found in the [KIP](https://cwiki.apache.org/confluence/display/KAFKA/KIP-163%3A+Lower+the+Minimum+Required+ACL+Permission+of+OffsetFetch).

Author: Vahid Hashemian <vahidhashemian@us.ibm.com>

Reviewers: Ewen Cheslack-Postava <ewen@confluent.io>

Closes #3661 from vahidhashemian/KAFKA-4585
2017-09-07 11:40:54 -07:00
Colin P. Mccabe 4065ffb3e1 KAFKA-5777; Add ducktape integration for Trogdor
Author: Colin P. Mccabe <cmccabe@confluent.io>

Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>

Closes #3726 from cmccabe/KAFKA-5777
2017-09-07 13:23:03 +01:00
Damian Guy 329d5fa64a KAFKA-5844; add groupBy(selector, serialized) to Ktable
add `KTable#groupBy(KeyValueMapper, Serialized)` and deprecate the overload with `Serde` params

Author: Damian Guy <damian.guy@gmail.com>

Reviewers: Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <wangguoz@gmail.com>, Bill Bejeck <bill@confluent.io>

Closes #3802 from dguy/kip-182-ktable-groupby
2017-09-07 12:35:31 +01:00
Damian Guy 9cbb9f0939 KAFKA-5650; add StateStoreBuilder interface and implementations
Part of KIP-182

- Add `StateStoreBuilder` interface and `WindowStateStoreBuilder`, `KeyValueStateStoreBuilder`, and `SessionStateStoreBuilder` implementations
- Add `StoreSupplier`, `WindowBytesStoreSupplier`, `KeyValueBytesStoreSupplier`, `SessionBytesStoreSupplier` interfaces and implementations
- Add new methods to `Stores` to create the newly added `StoreSupplier` and `StateStoreBuilder` implementations
- Update `Topology` and `InternalTopology` to use the interfaces

Author: Damian Guy <damian.guy@gmail.com>

Reviewers: Bill Bejeck <bill@confluent.io>, Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <wangguoz@gmail.com>

Closes #3767 from dguy/kafka-5650
2017-09-07 09:39:46 +01:00
Damian Guy 667cd60dc6 KAFKA-5816; add Produced class, KStream#to(topic, Produced), and KStream#through(topic, Produced)
Add the `Produced` class and `KStream` overloads that use it:
`KStream#to(String, Produced)`
`KStream#through(String, Produced)`
Deprecate all other to and through methods accept the single param methods that take a topic param

Author: Damian Guy <damian.guy@gmail.com>

Reviewers: Bill Bejeck <bill@confluent.io>, Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <wangguoz@gmail.com>

Closes #3770 from dguy/kafka-5652-produced
2017-09-07 08:54:10 +01:00
Boyang Chen b041c8d87d MINOR: close iterator on doc example
The iterator interface usage has some examples missing explicit close operation after usage. We should remind the user to do so because un-closed iterator will leave the underlying file descriptor open, thus eating up memory.
guozhangwang Ishiihara

Author: Boyang Chen <bychen@pinterest.com>
Author: Boyang Chen <bchen11@outlook.com>

Reviewers: Guozhang Wang <wangguoz@gmail.com>

Closes #3714 from abbccdda/add_iterator_close_on_doc

minor fixes
2017-09-06 22:12:10 -07:00
Guozhang Wang 9b85cf9ed0 MINOR: KIP-138 renaming of string names
Author: Guozhang Wang <wangguoz@gmail.com>

Reviewers: Bill Bejeck <bill@confluent.io>, Matthias J. Sax <matthias@confluent.io>, Damian Guy <damian.guy@gmail.com>

Closes #3796 from guozhangwang/kip-138-minor-renames
2017-09-06 18:39:40 -07:00
Guozhang Wang 3f155eaa23 MINOR: Log encountered exception during rebalance
Some other minor changes:

1. Do not throw the exception form callback as it would only be swallowed by consumer coordinator; remembering it and re-throw in the next loop is good enough.
2. Change Creating to Defining in Stores to avoid confusions that the stores have already been successfully created at that time.
3. Do not need unAssignChangeLogPartitions as the restore consumer will be unassigned already inside changelog reader.

Author: Guozhang Wang <wangguoz@gmail.com>

Reviewers: Matthias J. Sax <matthias@confluent.io>, Damian Guy <damian.guy@gmail.com>

Closes #3769 from guozhangwang/KMinor-logging-before-throwing
2017-09-06 18:38:08 -07:00
Matthias J. Sax 212bce6e3f MINOR: fix typos on web doc's upgrade guide
Author: Matthias J. Sax <matthias@confluent.io>

Reviewers: Guozhang Wang <wangguoz@gmail.com>

Closes #3804 from mjsax/hotfix
2017-09-06 15:42:52 -07:00
Attila Kreiner 23d01c805b KAFKA-5726; KafkaConsumer.subscribe() overload that takes just Pattern
- changed the interface & implementations
- updated tests to use the new method where applicable

Author: Attila Kreiner <attila@kreiner.hu>

Reviewers: Ismael Juma <ismael@juma.me.uk>, Jason Gustafson <jason@confluent.io>

Closes #3669 from attilakreiner/KAFKA-5726
2017-09-06 11:41:33 -07:00
oleg 5102576460 KAFKA-5756; Connect WorkerSourceTask synchronization issue on flush
Author: oleg <oleg@nexla.com>

Reviewers: Randall Hauch <rhauch@gmail.com>, Jason Gustafson <jason@confluent.io>

Closes #3702 from oleg-smith/KAFKA-5756
2017-09-06 11:12:23 -07:00
Damian Guy 45394d52c1 KAFKA-5819; Add Joined class and relevant KStream join overloads
Add the `Joined` class and the overloads to `KStream` that use it.
Deprecate existing methods that have `Serde` params

Author: Damian Guy <damian.guy@gmail.com>

Reviewers: Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <wangguoz@gmail.com>

Closes #3776 from dguy/kip-182-stream-join
2017-09-06 10:55:43 +01:00
Damian Guy b687c06800 KAFKA-5817; Add Serialized class and overloads to KStream#groupBy and KStream#groupByKey
Part of KIP-182
- Add the `Serialized` class
- implement overloads of `KStream#groupByKey` and KStream#groupBy`
- deprecate existing methods that have more than default arguments

Author: Damian Guy <damian.guy@gmail.com>

Reviewers: Bill Bejeck <bill@confluent.io>, Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <wangguoz@gmail.com>

Closes #3772 from dguy/kafka-5817
2017-09-06 10:43:14 +01:00
James Cheng 2fb5664bf4 KAFKA-5597: Autogenerate producer sender metrics
Subtask of https://issues.apache.org/jira/browse/KAFKA-3480

The changes are very similar to what was done for the consumer in https://issues.apache.org/jira/browse/KAFKA-5191 (pull request https://github.com/apache/kafka/pull/2993)

Author: James Cheng <jylcheng@yahoo.com>

Reviewers: Ismael Juma <ismael@juma.me.uk>, Rajini Sivaram <rajinisivaram@googlemail.com>, Guozhang Wang <wangguoz@gmail.com>

Closes #3535 from wushujames/producer_sender_metrics_docs

Fix one minor naming bug
2017-09-05 17:38:58 -07:00
Matthias J. Sax cd59976eea KAFKA-5823: Extend upgrade section for KIP-120
Author: Matthias J. Sax <matthias@confluent.io>

Reviewers: Sharon Liu <sharonliu.cup@gmail.com>, Damian Guy <damian.guy@gmail.com>, Guozhang Wang <wangguoz@gmail.com>

Closes #3787 from mjsax/kafka-5823-kip120-docs
2017-09-05 16:54:00 -07:00
Rajini Sivaram 5d69966195 KAFKA-5837; Set defaults for ReassignPartitionsCommand correctly
Author: Rajini Sivaram <rajinisivaram@googlemail.com>

Reviewers: Dong Lin <lindong28@gmail.com>

Closes #3792 from rajinisivaram/KAFKA-5837
2017-09-06 00:32:59 +01:00
Vladimír Kleštinec 81e789ae3d KAFKA-4860; Allow spaces in paths on windows
When we install kafka on path with spaces, batch files were failing, this PR is trying to fix this issue.

Author: Vladimír Kleštinec <klestinec@gmail.com>

Reviewers: Vahid Hashemian <vahidhashemian@us.ibm.com>, Jason Gustafson <jason@confluent.io>

Closes #2649 from klesta490/trunk
2017-09-05 16:27:08 -07:00
Matthias J. Sax 752e531741 KAFKA-5603; Don't abort TX for zombie tasks
Author: Matthias J. Sax <matthias@confluent.io>

Reviewers: Guozhang Wang <wangguoz@gmail.com>

Closes #3719 from mjsax/kafka-5603-dont-abort-tx-for-zombie-tasks-2
2017-09-05 15:47:31 -07:00
Florian Hussonnois facd2c5a8c KAFKA-4819: Expose states for active tasks to public API
Simple implementation of the feature : [KAFKA-4819](https://issues.apache.org/jira/browse/KAFKA-4819)
 KAFKA-4819

This PR adds a new method `threadStates` to public API of `KafkaStreams` which returns all currently states of running threads and active tasks.

Below is a example for a simple topology consuming from topics; test-p2 and test-p4.

[{"name":"StreamThread-1","state":"RUNNING","activeTasks":[{"id":"0_0", "assignments":["test-p4-0","test-p2-0"], "consumedOffsetsByPartition":[{"topicPartition":"test-p2-0","offset":"test-p2-0"}]}, {"id":"0_2", "assignments":["test-p4-2"], "consumedOffsetsByPartition":[]}]}, {"name":"StreamThread-2","state":"RUNNING","activeTasks":[{"id":"0_1", "assignments":["test-p4-1","test-p2-1"], "consumedOffsetsByPartition":[{"topicPartition":"test-p2-1","offset":"test-p2-1"}]}, {"id":"0_3", "assignments":["test-p4-3"], "consumedOffsetsByPartition":[]}]}]

Author: Florian Hussonnois <florian.hussonnois@gmail.com>

Reviewers: Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <wangguoz@gmail.com>

Closes #2612 from fhussonnois/KAFKA-4819
2017-09-05 14:11:59 -07:00
Damian Guy d78eb03fad MINOR: add mvn-pgp-plugin to sign streams quickstart jars
Author: Damian Guy <damian.guy@gmail.com>

Reviewers: Guozhang Wang <wangguoz@gmail.com>

Closes #3793 from dguy/sign-mvn-jars
2017-09-05 10:51:04 -07:00
Damian Guy caddae19ea MINOR: fix scalaVersion variable in templateData.js
Author: Damian Guy <damian.guy@gmail.com>

Reviewers: Ismael Juma <ismael@juma.me.uk>

Closes #3790 from dguy/fix-js-tempate
2017-09-05 11:45:21 +01:00
Ismael Juma b1871e5adf MINOR: Fix sftp_mkdir in release.py
Author: Ismael Juma <ismael@juma.me.uk>

Reviewers: Damian Guy <damian.guy@gmail.com>

Closes #3789 from ijuma/ftp-release-py-fixes
2017-09-05 10:27:36 +01:00
Eno Thereska 493c2aad59 MINOR: add table of contents
Added a simple table of contents for the developer section.

Author: Eno Thereska <eno.thereska@gmail.com>

Reviewers: Damian Guy <damian.guy@gmail.com>

Closes #3760 from enothereska/minor-docs-toc
2017-09-04 14:28:50 +01:00
Dong Lin adefc8ea07 KAFKA-5694; Add AlterReplicaDirRequest and DescribeReplicaDirRequest (KIP-113 part-1)
Author: Dong Lin <lindong28@gmail.com>

Reviewers: Jun Rao <junrao@gmail.com>, Jiangjie Qin <becket.qin@gmail.com>, Colin P. Mccabe <cmccabe@confluent.io>

Closes #3621 from lindong28/KAFKA-5694
2017-09-02 23:20:13 -07:00
Colin P. Mccabe b2a328daf2 KAFKA-5659; Fix error handling, efficiency issue in AdminClient#describeConfigs
If a request for a broker configuration failed due to a timeout or
the broker not being available, we would fail the futures
associated with the non-broker request instead (and never fail the
broker future, which would be left uncompleted forever).

We would also do an unnecessary request if only broker configs were
requested.

Author: Colin P. Mccabe <cmccabe@confluent.io>

Reviewers: Ismael Juma <ismael@juma.me.uk>

Closes #3585 from cmccabe/KAFKA-5659
2017-09-02 08:54:58 +01:00
Matthias J. Sax 14a7c297ab KAFKA-5818; KafkaStreams state transitions not correct
- need to check that state is CRATED at startup
- some minor test cleanup

Author: Matthias J. Sax <matthias@confluent.io>

Reviewers: Guozhang Wang <wangguoz@gmail.com>, Damian Guy <damian.guy@gmail.com>

Closes #3775 from mjsax/kafka-5818-kafkaStreams-state-transition
2017-09-02 08:24:43 +01:00
Dong Lin bc999989bf KAFKA-5759; Allow user to specify relative path as log directory
Author: Dong Lin <lindong28@gmail.com>

Reviewers: Ismael Juma <ismael@juma.me.uk>, Jiangjie Qin<becket.qin@gmail.com>

Closes #3709 from lindong28/KAFKA-5759
2017-09-01 09:38:27 -07:00
Sebastian Gavril 51af6fb654 MINOR: Allow timestamp parameter in `ProcessorTopologyTestDriver.process`
All current implementations process records using the same timestamp. This makes it difficult to test operations that require time windows, like `KStream-KStream joins`.

This change would allow tests to simulate records created at different times, thus making it possible to test operations like the above mentioned joins.

Author: Sebastian Gavril <sgavril@wehkamp.nl>

Reviewers: Bill Bejeck <bill@confluent.io>, Guozhang Wang <wangguoz@gmail.com>

Closes #3753 from sebigavril/allow-timestamps-in-test-driver
2017-08-31 14:57:54 -07:00
Randall Hauch 75070bdb5d MINOR: Increase timeout of Zookeeper service in system tests
The previous timeout was 10 seconds, but system test failures have occurred when Zookeeper has started after about 11 seconds. Increasing the timeout to 30 seconds, since most of the time this extra time will not be required, and when it is it will prevent a failed system test.

In addition to merging to `trunk`, please backport to the `0.11.x` and `0.10.2.x` branches.

Author: Randall Hauch <rhauch@gmail.com>

Reviewers: Ewen Cheslack-Postava <ewen@confluent.io>

Closes #3774 from rhauch/MINOR-Increase-timeout-of-zookeeper-service-in-system-tests
2017-08-31 14:53:44 -07:00
Colin P. Mccabe ded8741173 KAFKA-5806; Fix transient unit test failure in trogdor coordinator shutdown
In the coordinator, we should check that 'shutdown' is not true before going to sleep waiting for the condition.

Author: Colin P. Mccabe <cmccabe@confluent.io>

Reviewers: Apurva Mehta <apurva@confluent.io>, Jason Gustafson <jason@confluent.io>

Closes #3755 from cmccabe/KAFKA-5806
2017-08-31 13:19:28 -07:00
Rajini Sivaram 47c2753496 MINOR: Test SASL authorization id
Author: Rajini Sivaram <rajinisivaram@googlemail.com>

Reviewers: Ismael Juma <ismael@juma.me.uk>

Closes #3766 from rajinisivaram/MINOR-sasl
2017-08-31 12:42:08 -04:00
Guozhang Wang 9ebc303bb8 KAFKA-5379 follow up: reduce redundant mock processor context
Author: Guozhang Wang <wangguoz@gmail.com>

Reviewers: Matthias J. Sax <matthias@confluent.io>

Closes #3757 from guozhangwang/K5379-follow-up
2017-08-31 08:43:13 -07:00
Manikumar Reddy 9836adc732 KAFKA-2105; Check for null in KafkaProducer.partitionsFor
Author: Manikumar Reddy <manikumar.reddy@gmail.com>

Reviewers: Ismael Juma <ismael@juma.me.uk>

Closes #3697 from omkreddy/KAFKA-2105
2017-08-31 16:30:42 +01:00
Guozhang Wang ce04b14358 MINOR: KIP-160 docs
Author: Guozhang Wang <wangguoz@gmail.com>

Reviewers: Damian Guy <damian.guy@gmail.com>

Closes #3733 from guozhangwang/KMinor-kip160-docs
2017-08-30 11:17:55 -07:00
Guozhang Wang f5310d645c MINOR: doc changes for KIP-138
1. Core concepts (added the stream time definition), upgrade guide and developer guide.
2. Related Java docs changes.

Author: Guozhang Wang <wangguoz@gmail.com>

Reviewers: Bill Bejeck <bill@confluent.io>, Damian Guy <damian.guy@gmail.com>

Closes #3732 from guozhangwang/KMinor-kip138-docs
2017-08-30 09:45:16 -07:00
Guozhang Wang 6682abe4ae KAFKA-5797: Delay checking of partition existence in StoreChangelogReader
1. Remove timeout-based validatePartitionExists from StoreChangelogReader; instead only try to refresh metadata once after all tasks have been created and their topology initialized (hence all stores have been registered).
2. Add the logic to refresh partition metadata at the end of initialization if some restorers needing initialization cannot find their changelogs, hoping that in the next run loop these stores can find their changelogs.

As a result, after `initialize` is called we may not be able to start initializing all the `needsInitializing` ones.

As an optimization, we would not call `consumer#partitionsFor` any more, but only `consumer#listTopics` fetching all the topic metadata; so the only blocking calls left are `listTopics` and `endOffsets`, and we always capture timeout exceptions around these two calls, and delay to retry in the next run loop after refreshing the metadata. By doing this we can also reduce the number of request round trips between consumer and brokers.

Author: Guozhang Wang <wangguoz@gmail.com>

Reviewers: Damian Guy <damian.guy@gmail.com>, Matthias J. Sax <matthias@confluent.io>

Closes #3748 from guozhangwang/K5797-handle-metadata-available
2017-08-30 08:43:22 -07:00
Damian Guy 76c9a6dcbc KAFKA-5804; retain duplicates in ChangeLoggingWindowBytesStore
`ChangeLoggingWindowBytesStore` needs to have the same `retainDuplicates` functionality as `RocksDBWindowStore` else data could be lost upon failover/restoration.

Author: Damian Guy <damian.guy@gmail.com>

Reviewers: Bill Bejeck <bill@confluent.io>, Guozhang Wang <wangguoz@gmail.com>

Closes #3754 from dguy/hotfix-changelog-window-store
2017-08-30 13:52:44 +01:00
Kamal C 10cb534c8b KAFKA-5762; Refactor AdminClient to use LogContext
- client id is part of the log context, so removed ad-hoc usages
- Fixed an issue where the response was not printed correctly,
use `toString(version)` instead of `toString()`
- Capitalized all log statements for consistency
- Fixed a number of double spaces after period

Author: Kamal Chandraprakash <kamal.chandraprakash@gmail.com>

Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>, Ismael Juma <ismael@juma.me.uk>

Closes #3741 from Kamal15/kafka-5762
2017-08-30 12:27:54 +01:00
Eno Thereska bd54d2e3e0 MINOR: stateful docs for aggregates
Author: Eno Thereska <eno.thereska@gmail.com>

Reviewers: Damian Guy <damian.guy@gmail.com>

Closes #3730 from enothereska/minor-docs-aggregates
2017-08-30 10:09:08 +01:00
Colin P. Mccabe 949577ca77 KAFKA-5768; Upgrade to ducktape 0.7.1
Author: Colin P. Mccabe <cmccabe@confluent.io>

Reviewers: Jason Gustafson <jason@confluent.io>

Closes #3721 from cmccabe/KAFKA-5768
2017-08-29 16:41:19 -07:00
Tommy Becker 9ef2b3ce8e KAFKA-5379: ProcessorContext.appConfigs() should return parsed values
Author: Tommy Becker <tobecker@tivo.com>
Author: Tommy Becker <twbecker@gmail.com>

Reviewers: Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <wangguoz@gmail.com>

Closes #3455 from twbecker/kafka-5379
2017-08-29 13:01:57 -07:00
Guozhang Wang 1f659913a6 MINOR: simplify state transition for Kafka Streams and stream threads
1. StreamThread: prevent `PARTITIONS_REVOKED` to transit to itself in `setState` by returning false. And only execute the task suspension logic when `setState(PARTITIONS_REVOKED)` returns true in `onPartitionsRevoked`.

2. StreamThread: minor, renaming `shutdown` to `completeShutdown`, and `close` to `shutdown`, `stillRunning` to `isRunning`, `isInitialized` to `isRunningAndNotRebalancing`.

3. GlobalStreamThread: tighten the transition a bit in `setState`. Force transiting to `PENDING_SHUTDOWN` and `DEAD` when initialization failed.

4. GlobalStreamThread: minor, add logPrefix to StateConsumer. Also removing its state change listener when closing the thread.

5. KafkaStreams: because of 1) above we can now prevent its `REBALANCING` to `REBALANCING`.

6. KafkaStreams: prevent `CREATED` to ever go to `REBALANCING` first to force it transit to `RUNNING` when starting. Also prevent `CREATED` to go to `ERROR`.

7. KafkaStreams: collapse `validateStartOnce` and `checkFirstTimeClosing ` into `setState`.

8. KafkaStreams: in `close` and `start`, only execute the logic when `setState` succeeds.

Author: Guozhang Wang <wangguoz@gmail.com>
Author: Damian Guy <damian.guy@gmail.com>

Reviewers: Damian Guy <damian.guy@gmail.com>, Matthias J. Sax <matthias@confluent.io>

Closes #3713 from guozhangwang/KMinor-set-state
2017-08-29 10:40:42 -07:00
Colin P. Mccabe 7ae02a69c4 KAFKA-5720; Fix AdminClientIntegrationTest#testCallInFlightTimeouts
* When a call is aborted, that should count as a "try" in the failure log message.
* FailureInjectingTimeoutProcessorFactory should fail the first request it is asked about.
* testCallTimeouts should expect the first request it makes to fail because of the timeout we injected.
* FailureInjectingTimeoutProcessorFactory should track how many failures it has injected, and the test should verify that one has been injected.

Author: Colin P. Mccabe <cmccabe@confluent.io>

Reviewers: Ismael Juma <ismael@juma.me.uk>

Closes #3731 from cmccabe/KAFKA-5720
2017-08-27 09:26:38 +01:00
Holden Karau 8d3aa6acad KAFKA-4380; Document the purpose of clean shutdown file in the code.
Remove the previous TODO to remove the clean shutdown file with some of the discussion from https://github.com/apache/kafka/pull/2104.

Author: Holden Karau <holden@us.ibm.com>

Reviewers: Ismael Juma <ismael@juma.me.uk>

Closes #3742 from holdenk/KAFKA-4380-document-clean-shutdown-file
2017-08-27 09:13:05 +01:00
Michal Borowiecki 5648dcc3e5 MINOR: Fix doc typos and grammar
This is contributed by mihbor on various doc fixes including:

https://github.com/apache/kafka/pull/3224
https://github.com/apache/kafka/pull/3226
https://github.com/apache/kafka/pull/3229

Author: Guozhang Wang <wangguoz@gmail.com>

Reviewers: Guozhang Wang <wangguoz@gmail.com>

Closes #3746 from guozhangwang/KMinor-doc-typos
2017-08-26 16:33:33 -07:00
Bill Bejeck b6adb2dc89 KAFKA-3989; MINOR: follow-up: update script to run from kafka root
…h-benchmarks/jmh.sh

Author: bbejeck <bbejeck@gmail.com>

Reviewers: Ismael Juma <ismael@juma.me.uk>, Guozhang Wang <wangguoz@gmail.com>

Closes #2654 from bbejeck/KAFKA-3989_follow_up
2017-08-26 16:15:40 -07:00