Commit Graph

3867 Commits

Author SHA1 Message Date
Colin P. Mccabe 5536b1237f KAFKA-5484: Refactor kafkatest docker support
Author: Colin P. Mccabe <cmccabe@confluent.io>

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

Closes #3389 from cmccabe/KAFKA-5484
2017-06-29 13:28:35 -07:00
Ismael Juma 27fdd46dcd MINOR: Improve TransactionIndex.sanityCheck() message
Author: Ismael Juma <ismael@juma.me.uk>

Reviewers: Jason Gustafson <jason@confluent.io>

Closes #3446 from ijuma/improve-transaction-index-exception-sanity-check
2017-06-29 09:51:04 -07:00
Rajini Sivaram dc95456f1d KAFKA-5402; Avoid creating quota related metrics if quotas not enabled
Author: Rajini Sivaram <rajinisivaram@googlemail.com>

Reviewers: Jun Rao <junrao@gmail.com>, Ismael Juma <ismael@juma.me.uk>

Closes #3303 from rajinisivaram/KAFKA-5402
2017-06-29 08:01:12 -04:00
Ismael Juma a05a00e503 MINOR: Implement `toString` in some Validator instances
This is used in the generated config table. Also fix a couple
of typos in the process.

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

Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>

Closes #3451 from ijuma/fix-doc-typos
2017-06-28 20:31:43 +01:00
Jason Gustafson cb0325d484 KAFKA-5490; Cleaner should retain empty batch if needed to preserve producer last sequence
Author: Jason Gustafson <jason@confluent.io>

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

Closes #3406 from hachikuji/KAFKA-5490
2017-06-28 11:35:13 -07:00
Apurva Mehta f05c70b069 MINOR: Enable the TransactionsBounceTest
I'll let this have multiple runs on the branch builder to see if it fails, and investigate if so.

Author: Apurva Mehta <apurva@confluent.io>

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

Closes #3441 from apurvam/MINOR-enable-transactions-bounce-test
2017-06-28 10:22:11 -07:00
Michal Borowiecki efb060c57f KAFKA-5233; KIP-138: Change punctuate semantics
Implementation for KIP-138: Change punctuate semantics

Author: Michal Borowiecki <michal.borowiecki@openbet.com>

Reviewers: Matthias J. Sax <matthias@confluent.io>, Bill Bejeck <bbejeck@gmail.com>, Eno Thereska <eno.thereska@gmail.com>, Damian Guy <damian.guy@gmail.com>

Closes #3055 from mihbor/KIP-138
2017-06-28 11:26:02 +01:00
Ismael Juma a293e1dc0c MINOR: Adjust checkstyle suppression paths to work on Windows
Use the file name whenever possible and replace / with [/\\]
when it's not.

Also remove unnecessary suppresions.

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

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

Closes #3431 from ijuma/fix-checkstyle-suppressions-on-windows
2017-06-28 01:47:00 +01:00
Matthias J. Sax 1d28e25748 MINOR: remove unnecessary null check
Author: Matthias J. Sax <matthias@confluent.io>

Reviewers: Guozhang Wang <wangguoz@gmail.com>

Closes #3445 from mjsax/minor-remove-null-check
2017-06-27 15:29:48 -07:00
Jason Gustafson 031da889bc MINOR: Fix race condition in KafkaConsumer close
We intended to make `KafkaConsumer.close()` idempotent,
but due to the fact that the `closed` variable is
checked without a lock prior to beginning close logic,
it is possible for two or more threads to see
`closed=false` and attempt to close.

Author: Jason Gustafson <jason@confluent.io>

Reviewers: Apurva Mehta <apurva@confluent.io>, Ismael Juma <ismael@juma.me.uk>

Closes #3426 from hachikuji/minor-fix-consumer-idempotent-close
2017-06-27 16:36:45 +01:00
Apurva Mehta f1cc8008e5 KAFKA-5032; Update the docs for message size configs across the board
Before 0.11, we used to have limits for maximum message size on the producer, broker, and consumer side.

From 0.11 onward, these limits apply to record batches as a whole. This patch updates the documentation of the configs to make this explicit.

A separate patch will have more extensive upgrade notes to tie all the changes together in one narrative.

Author: Apurva Mehta <apurva@confluent.io>

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

Closes #3374 from apurvam/KAFKA-5032-message-size-docs
2017-06-27 15:11:12 +01:00
Ewen Cheslack-Postava e45c767d53 MINOR: Make JmxMixin wait for the monitored process to be listening on the JMX port before launching JmxTool
Author: Ewen Cheslack-Postava <me@ewencp.org>

Reviewers: Apurva Mehta <apurva@confluent.io>, Ismael Juma <ismael@juma.me.uk>

Closes #3437 from ewencp/wait-jmx-listening
2017-06-26 17:06:38 -07:00
Matthias J. Sax 2265834803 KAFKA-5362: Add Streams EOS system test with repartitioning topic
Author: Matthias J. Sax <matthias@confluent.io>

Reviewers: Guozhang Wang <wangguoz@gmail.com>

Closes #3310 from mjsax/kafka-5362-add-eos-system-tests-for-streams-api
2017-06-25 09:34:27 -07:00
Ismael Juma a5c47db138 KAFKA-5506; Fix NPE in OffsetFetchRequest.toString and logging improvements
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
2017-06-24 08:20:55 +01:00
Eno Thereska ee5eac715d KAFKA-5487; upgrade and downgrade streams app system test
-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
2017-06-24 07:48:10 +01:00
Guozhang Wang d5e463b9de KAFKA-4829: Improve log4j on Streams thread / task-level
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
2017-06-23 18:01:41 -07:00
Ismael Juma d65844b527 MINOR: SaslChannelBuilder should be idempotent
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
2017-06-23 11:19:14 -07:00
Eno Thereska b490368735 HOTFIX: Don't check metadata unless you are creating topic
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
2017-06-23 15:56:29 +01:00
ppatierno 9ada0f8169 MINOR: Fixed way how logging methods are used for having a consistent one
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
2017-06-23 14:14:18 +01:00
Jeyhun Karimov 701e318ee1 KAFKA-4653; Improve test coverage of RocksDBStore
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
2017-06-23 11:41:02 +01:00
Jeyhun Karimov 26eea1d71e KAFKA-4656; Improve test coverage of CompositeReadOnlyKeyValueStore
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
2017-06-23 11:32:47 +01:00
Damian Guy 2420491f41 KAFKA-4913; prevent creation of window stores with less than 2 segments
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
2017-06-23 08:59:13 +01:00
Matthias J. Sax ac53979647 MINOR: update AWS test setup guide
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
2017-06-22 16:42:55 -07:00
Matthias J. Sax b62cccd078 MINOR: improve test README
Author: Matthias J. Sax <matthias@confluent.io>

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

Closes #3416 from mjsax/minor-aws
2017-06-22 16:17:27 -07:00
Matthias J. Sax 4baca9172d HOTFIX: reduce log verbosity on commit
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
2017-06-22 23:00:29 +01:00
Onur Karaman 6d2fbfc911 KAFKA-5502; read current brokers from zookeeper upon processing broker change
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
2017-06-22 14:28:03 -07:00
Ewen Cheslack-Postava cd11fd7874 KAFKA-5498: ConfigDef derived from another ConfigDef did not correctly compute parentless configs
Author: Ewen Cheslack-Postava <me@ewencp.org>

Reviewers: Gwen Shapira

Closes #3412 from ewencp/kafka-5498-base-configdef-parentless-configs
2017-06-22 13:00:12 -07:00
Ismael Juma 5d9563d95f MINOR: Switch ZK client logging to INFO
Author: Ismael Juma <ismael@juma.me.uk>

Reviewers: Jun Rao <junrao@gmail.com>

Closes #3409 from ijuma/tweak-log-config
2017-06-22 16:56:06 +01:00
Jason Gustafson fc58ac594f KAFKA-5490; Skip empty record batches in the consumer
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
2017-06-22 16:54:28 +01:00
Jeyhun Karimov 1744a9b4c2 KAFKA-4658; Improve test coverage InMemoryKeyValueLoggedStore
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
2017-06-22 15:04:29 +01:00
Jeyhun Karimov adfaa11611 KAFKA-4655; Improve test coverage of CompositeReadOnlySessionStore
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
2017-06-22 15:00:41 +01:00
Kevin Sweeney 785d8e20ca MINOR: Provide link to ZooKeeper within Quickstart
Author: Kevin Sweeney <restlessdesign@users.noreply.github.com>

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

Closes #3372 from restlessdesign/patch-1
2017-06-22 14:03:39 +01:00
Tom Bentley a6799f4e14 KAFKA-4059; API Design section under Implementation is out of date
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
2017-06-22 13:42:40 +01:00
Ismael Juma a4794b11b2 KAFKA-5486: org.apache.kafka logging should go to server.log
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
2017-06-22 12:48:53 +01:00
Jeyhun Karimov cb5e1f0a40 KAFKA-4785; Records from internal repartitioning topics should always use RecordMetadataTimestampExtractor
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
2017-06-22 12:23:58 +01:00
Guozhang Wang 7f4feda959 MINOR: Turn off caching in demos for more understandable outputs
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
2017-06-22 08:53:32 +01:00
Jeyhun Karimov 4e8797f54e KAFKA-4659; Improve test coverage of CachingKeyValueStore
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
2017-06-22 08:40:54 +01:00
Matthias J. Sax 914e42a282 KAFKA-5474: Streams StandbyTask should no checkpoint on commit if EOS is enabled
<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
2017-06-21 17:16:48 -07:00
Apurva Mehta bc47e9d6ca KAFKA-5491; Enable transactions in ProducerPerformance Tool
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
2017-06-21 14:41:51 -07:00
Ewen Cheslack-Postava 96587f4b1f KAFKA-5475: Connector config validation should include fields for defined transformation aliases
Author: Ewen Cheslack-Postava <me@ewencp.org>

Reviewers: Konstantine Karantasis <konstantine@confluent.io>, Jason Gustafson <jason@confluent.io>

Closes #3399 from ewencp/kafka-5475-validation-transformations
2017-06-21 14:20:48 -07:00
Jason Gustafson e6e2631743 MINOR: Detail message/batch size implications for conversion between old and new formats
Author: Jason Gustafson <jason@confluent.io>

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

Closes #3373 from hachikuji/fetch-size-upgrade-notes
2017-06-21 14:04:19 -07:00
Guozhang Wang f848e2cd68 Revert "MINOR: make flush no-op as we don't need to call flush on commit."
This reverts commit 90b2a2bf66.
2017-06-21 13:05:54 -07:00
Apurva Mehta 254add9534 KAFKA-5477; Lower retry backoff for first AddPartitions in transaction
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
2017-06-21 11:00:08 -07:00
Eno Thereska 55a90938a1 MINOR: add Yahoo benchmark to nightly runs
Author: Eno Thereska <eno.thereska@gmail.com>

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

Closes #3289 from enothereska/yahoo-benchmark
2017-06-21 11:46:59 +01:00
Matthias J. Sax cae5977ed0 MINOR: explain producer naming within Streams
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
2017-06-21 09:32:46 +01:00
Kelvin Rutt 76f6e14b07 KAFKA-5413; Log cleaner fails due to large offset in segment file
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
2017-06-20 18:00:41 -07:00
Randall Hauch de982ba3fb KAFKA-5472: Eliminated duplicate group names when validating connector results
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
2017-06-20 17:48:32 -07:00
Damian Guy 84de7f1756 MINOR: remove unused hitRatio field in NamedCache
Author: Damian Guy <damian.guy@gmail.com>

Reviewers: Guozhang Wang <wangguoz@gmail.com>

Closes #3384 from dguy/remove-unused-field
2017-06-20 17:38:56 -07:00
Jason Gustafson cb3952a4fd KAFKA-5021; Update delivery semantics documentation for EoS (KIP-98)
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
2017-06-20 16:56:38 -07:00
Guozhang Wang f28fc1100b MINOR: Add Processing Guarantees to Streams docs
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
2017-06-20 16:29:08 -07:00