Reviewers: Viktor Somogyi <viktorsomogyi@gmail.com>, Vahid Hashemian <vahidhashemian@us.ibm.com>, Manikumar Reddy <manikumar.reddy@gmail.com>, Ismael Juma <ismael@juma.me.uk>
- Removed internal kafka.admin.AdminClient.deleteRecordsBefore since it's
no longer used.
- Removed redundant tests and rewrote non redundant ones to use the Java
AdminClient.
Reviewers: Viktor Somogyi <viktor.somogyi@cloudera.com>, Manikumar Reddy <manikumar.reddy@gmail.com>, Ismael Juma <ismael@juma.me.uk>
This patch contains a few follow-up improvements/cleanup for KIP-266:
- Add upgrade notes
- Add missing `commitSync(Duration)` API
- Improve timeout messages and fix some naming inconsistencies
- Various small cleanups
Reviewers: John Roesler <john@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
This implements KIP-219, where a broker returns a response with throttle time on
quota violation immediately after processing the corresponding request. After
the response is sent out, the broker will keep the channel muted until the
throttle time is over. Also, on receiving a response with throttle time, client
will block outgoing communication to the broker for the specified throttle time.
See PR 4830, 5064 and 5094 for all the review history
Author: Jon Lee <jonlee@jonlee-ld1.linkedin.biz>
Reviewers: Jun Rao <junrao@gmail.com>, Rajini Sivaram <rajinisivaram@googlemail.com>, Ismael Juma <ismael@juma.me.uk>, Dong Lin <lindong28@gmail.com>
Closes#5064 from jonlee2/kip-219
Implementation of [KIP-174](https://cwiki.apache.org/confluence/display/KAFKA/KIP-174+-+Deprecate+and+remove+internal+converter+configs+in+WorkerConfig)
Configuration properties 'internal.key.converter' and 'internal.value.converter'
are deprecated, and default to org.apache.kafka.connect.json.JsonConverter.
Warnings are logged if values are specified for either, or if properties that
appear to configure instances of internal converters (i.e., ones prefixed with
either 'internal.key.converter.' or 'internal.value.converter.') are given.
The property 'schemas.enable' is also defaulted to false for internal
JsonConverter instances (both for keys and values) if it isn't specified.
Documentation and code have also been updated with deprecation notices and
annotations, respectively.
Unit tests have been updated in `PluginsTest` to account for the new defaults for `schemas.enable` for internal key/value converters, and to ensure that (for the time being), internal key/value converters are still configurable despite being deprecated.
Author: Chris Egerton <chrise@confluent.io>
Author: Ewen Cheslack-Postava <me@ewencp.org>
Reviewers: Randall Hauch <rhauch@gmail.com>, Ewen Cheslack-Postava <ewen@confluent.io>
Closes#4693 from C0urante/kafka-5540
* Removed Scala producers, request classes, kafka.tools.ProducerPerformance, encoders,
tests.
* Updated ConsoleProducer to remove Scala producer support (removed `BaseProducer`
and several options that are not used by the Java producer).
* Updated a few Scala consumer tests to use the new producer (including a minor
refactor of `produceMessages` methods in `TestUtils`).
* Updated `ClientUtils.fetchTopicMetadata` to use `SimpleConsumer` instead of
`SyncProducer`.
* Removed `TestKafkaAppender` as it looks useless and it defined an `Encoder`.
* Minor import clean-ups
No new tests added since behaviour should remain the same after these changes.
Author: Ismael Juma <ismael@juma.me.uk>
Reviewers: Manikumar Reddy O <manikumar.reddy@gmail.com>, Dong Lin <lindong28@gmail.com>
Closes#5045 from ijuma/kafka-6921-remove-old-producer
* Set --source, --target and --release to 1.8.
* Build Scala 2.12 by default.
* Remove some conditionals in the build file now that Java 8
is the minimum version.
* Bump the version of Jetty, Jersey and Checkstyle (the newer
versions require Java 8).
* Fixed issues uncovered by the new version if Checkstyle.
* A couple of minor updates to handle an incompatible source
change in the new version of Jetty.
* Add dependency to jersey-hk2 to fix failing tests caused
by Jersey upgrade.
* Update release script to use Java 8 and to take into account
that Scala 2.12 is now built by default.
* While we're at it, bump the version of Gradle, Gradle plugins,
ScalaLogging, JMH and apache directory api.
* Minor documentation updates including the readme and upgrade
notes. A number of Streams Java 7 examples can be removed
subsequently.
Implementation of KIP-279 as described here: https://cwiki.apache.org/confluence/display/KAFKA/KIP-279%3A+Fix+log+divergence+between+leader+and+follower+after+fast+leader+fail+over
In summary:
- Added leader_epoch to OFFSET_FOR_LEADER_EPOCH_RESPONSE
- Leader replies with the pair( largest epoch less than or equal to the requested epoch, the end offset of this epoch)
- If Follower does not know about the leader epoch that leader replies with, it truncates to the end offset of largest leader epoch less than leader epoch that leader replied with, and sends another OffsetForLeaderEpoch request. That request contains the largest leader epoch less than leader epoch that leader replied with.
Reviewers: Dong Lin <lindong28@gmail.com>, Jun Rao <junrao@gmail.com>
Updated the upgrade doc as well since we do not have an overloaded function without the deprecated parameter before. Also renamed the 1.2 release version to 2.0.
Reviewers: Bill Bejeck <bill@confluent.io>, Matthias J. Sax <matthias@confluent.io>
This patch fixes a bug in the validation of the inter-broker protocol and the message format version. We should allow the configured message format api version to be greater than the inter-broker protocol api version as long as the actual message format versions are equal. For example, if the message format version is set to 1.0, it is fine for the inter-broker protocol version to be 0.11.0 because they both use message format v2.
I have added a unit test which checks compatibility for all combinations of the message format version and the inter-broker protocol version.
Author: Jason Gustafson <jason@confluent.io>
Reviewers: Ismael Juma <ismael@juma.me.uk>
Closes#4583 from hachikuji/KAFKA-6328-REOPENED
Enable deep-iteration option when print-data-log is enabled in DumpLogSegments. Otherwise data is not printed.
Reviewers: Jason Gustafson <jason@confluent.io>, Ismael Juma <ismael@juma.me.uk>
**[KIP-145](https://cwiki.apache.org/confluence/display/KAFKA/KIP-145+-+Expose+Record+Headers+in+Kafka+Connect) has been accepted, and this PR implements KIP-145 except without the SMTs.**
Changed the Connect API and runtime to support message headers as described in [KIP-145](https://cwiki.apache.org/confluence/display/KAFKA/KIP-145+-+Expose+Record+Headers+in+Kafka+Connect).
The new `Header` interface defines an immutable representation of a Kafka header (key-value pair) with support for the Connect value types and schemas. This interface provides methods for easily converting between many of the built-in primitive, structured, and logical data types.
The new `Headers` interface defines an ordered collection of headers and is used to track all headers associated with a `ConnectRecord` (and thus `SourceRecord` and `SinkRecord`). This does allow multiple headers with the same key. The `Headers` contains methods for adding, removing, finding, and modifying headers. Convenience methods allow connectors and transforms to easily use and modify the headers for a record.
A new `HeaderConverter` interface is also defined to enable the Connect runtime framework to be able to serialize and deserialize headers between the in-memory representation and Kafka’s byte[] representation. A new `SimpleHeaderConverter` implementation has been added, and this serializes to strings and deserializes by inferring the schemas (`Struct` header values are serialized without the schemas, so they can only be deserialized as `Map` instances without a schema.) The `StringConverter`, `JsonConverter`, and `ByteArrayConverter` have all been extended to also be `HeaderConverter` implementations. Each connector can be configured with a different header converter, although by default the `SimpleHeaderConverter` is used to serialize header values as strings without schemas.
Unit and integration tests are added for `ConnectHeader` and `ConnectHeaders`, the two implementation classes for headers. Additional test methods are added for the methods added to the `Converter` implementations. Finally, the `ConnectRecord` object is already used heavily, so only limited tests need to be added while quite a few of the existing tests already cover the changes.
Author: Randall Hauch <rhauch@gmail.com>
Reviewers: Arjun Satish <arjun@confluent.io>, Ted Yu <yuzhihong@gmail.com>, Magesh Nandakumar <magesh.n.kumar@gmail.com>, Konstantine Karantasis <konstantine@confluent.io>, Ewen Cheslack-Postava <ewen@confluent.io>
Closes#4319 from rhauch/kafka-5142-b
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
It should only depend on slf4j-api (like kafka-clients). The
release tarball still includes log4j and slf4j-log4j12.
Manually verified that there are no duplicate dependencies
in the release tarball and `./gradlew core:dependencies`
looks good.
Author: Ismael Juma <ismael@juma.me.uk>
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>
Closes#4297 from ijuma/kafka-6317-kafka-slf4j-api-only
From 0.11 to 1.0, we moved `DescribeClusterOptions timeoutMs(Integer timeoutMs)` from
DescribeClusterOptions to AbstractOptions (similarly for other Options classes). This can
cause code compiled against 0.11.0.x to fail when it is executed with 1.0 kafka-clients jar.
This patch adds back these methods to restore binary compatibility with 0.11.
Author: Dong Lin <lindong28@gmail.com>
Reviewers: Ismael Juma <ismael@juma.me.uk>
Closes#4257 from lindong28/KAFKA-6174
The upgrade instructions concerning the message format versions did not account
for upgrades from versions prior to 0.11.0.x.
Author: Jason Gustafson <jason@confluent.io>
Reviewers: Ismael Juma <ismael@juma.me.uk>
Closes#4256 from hachikuji/KAFKA-6328
Since we removed the unused `TRACE` option from `SecurityProtocol`, it now seems safer to expose it from `AuthenticationContext`. Additionally this patch exposes javadocs under security.auth and relocates the `Login` and `AuthCallbackHandler` to a non-public package.
Author: Jason Gustafson <jason@confluent.io>
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>, Manikumar Reddy <manikumar.reddy@gmail.com>, Ismael Juma <ismael@juma.me.uk>
Closes#3863 from hachikuji/use-security-protocol-in-auth-context
This is less likely to break custom metric reporters and since the method
is deprecated, people will be warned about this potential issue.
Author: Ismael Juma <ismael@juma.me.uk>
Reviewers: Ewen Cheslack-Postava <ewen@confluent.io>, Xavier Léauté <xavier@confluent.io>
Closes#3996 from ijuma/avoid-exception-in-measurable-value
Adds new metrics to support health checks:
1. Error rates for each request type, per-error code
2. Request size and temporary memory size
3. Message conversion rate and time
4. Successful and failed authentication rates
5. ZooKeeper latency and status
6. Client version
Author: Rajini Sivaram <rajinisivaram@googlemail.com>
Reviewers: Ismael Juma <ismael@juma.me.uk>
Closes#3705 from rajinisivaram/KAFKA-5746-new-metrics
Make documentation consistent across methods and throw
IllegalStateException instead of IllegalArgumentException in
some cases.
Also include a couple of minor fixes in upgrade.html.
Author: Dong Lin <lindong28@gmail.com>
Reviewers: Ismael Juma <ismael@juma.me.uk>
Closes#3781 from lindong28/minor-admin-client-comment
Author: Mickael Maison <mickael.maison@gmail.com>
Reviewers: Ismael Juma <ismael@juma.me.uk>, Jason Gustafson <jason@confluent.io>
Closes#3036 from mimaison/KAFKA-3356
Developed with edoardocomar
Author: Mickael Maison <mickael.maison@gmail.com>
Reviewers: Edoardo Comar <ecomar@uk.ibm.com>, Rajini Sivaram <rajinisivaram@googlemail.com>
Closes#3906 from mimaison/KAFKA-5735
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
1. Add upgrade section for 1.0.0, including Streams API changes section.
2. Add metrics name changes section.
Author: Guozhang Wang <wangguoz@gmail.com>
Reviewers: Eno Thereska <eno.thereska@gmail.com>, Damian Guy <damian.guy@gmail.com>
Closes#3687 from guozhangwang/KMinor-metrics-upgrade-guide
These handlers were previously used on the broker to handle uncaught exceptions, but now the broker users the new Java request objects exclusively.
Author: Jason Gustafson <jason@confluent.io>
Reviewers: Ismael Juma <ismael@juma.me.uk>
Closes#3646 from hachikuji/remove-old-request-error-handlers
This is important because Bits.reserveMemory calls System.gc() hoping to free native
memory in order to avoid throwing an OutOfMemoryException. This call is currently
a no-op due to -XX:+DisableExplicitGC.
It's worth mentioning that -XX:MaxDirectMemorySize can be used to increase the
amount of native memory available for allocation of direct byte buffers.
Author: Ismael Juma <ismael@juma.me.uk>
Reviewers: Apurva Mehta <apurva@confluent.io>, Manikumar Reddy <manikumar.reddy@gmail.com>, Jason Gustafson <jason@confluent.io>
Closes#3371 from ijuma/kafka-5470-explicit-gc-invokes-concurrent