This new policy enables active/passive, one-way replication without renaming topics, similar to MM1. This implementation is described in KIP-382 (adopted), originally as "LegacyReplicationPolicy".
This enables operators to migrate from MM1 to MM2 without re-architecting their replication flows, and enables some additional use-cases for MM2. For example, operators may wish to "upgrade" their Kafka clusters by mirroring everything to a completely new cluster. Such a migration would have been difficult with either MM1 or MM2 previously.
When using IdentityReplicationPolicy, operators should be aware that MM2 will not be able to detect cycles among replicated topics. A misconfigured topology may result in replicating the same records back-and-forth or in an infinite loop. However, we don't prevent this behavior, as some use-cases involve filtering records (via SMTs) to prevent cycles.
Reviewers: Mickael Maison <mickael.maison@gmail.com>
Co-authored-by: Ryanne Dolan <rdolan@twitter.com>
Co-authored-by: Matthew de Detrich <mdedetrich@gmail.com>
Co-authored-by: Ivan Yurchenko <ivanyu@aiven.io>
Update the ZooKeeper version to v3.6.3. This requires adding dropwizard
as a new dependency.
Also, add Kafka v2.8.0 to the ducktape system test image.
Reviewers: Luke Chen <showuon@gmail.com>, Colin P. McCabe <cmccabe@apache.org>, Ismael Juma <ismael@juma.me.uk>
Remove the `rest.host.name` and `rest.port` Connect worker configs that were deprecated in KIP-208 and AK 1.1.
Author: Kalpesh Patel <kalpeshpatel.india@gmail.com>
Reviewers: Randall Hauch <rhauch@gmail.com>, Konstantine Karantasis <konstantine@confluent.io>, wenbingshen <oliver.shen999@gmail.com>
They have been deprecated since 0.10.0. Full list of removes configs:
* port
* host.name
* advertised.port
* advertised.host.name
Also adjust tests to take the removals into account. Some tests were
no longer relevant and have been removed.
Finally, took the chance to:
* Clean up unnecessary usage of `KafkaConfig$.MODULE$` in
related files.
* Add missing `Test` annotations to `AdvertiseBrokerTest` and
make necessary changes for the tests to pass.
Reviewers: David Jacot <djacot@confluent.io>, Luke Chen <showuon@gmail.com>
They were both deprecated in Apache Kafka 2.4 and it's a straightforward change
to use the non deprecated variants.
Reviewers: David Jacot <djacot@confluent.io>
Deprecates the following
1. StreamsConfig.EXACTLY_ONCE
2. StreamsConfig.EXACTLY_ONCE_BETA
3. Producer#sendOffsetsToTransaction(Map offsets, String consumerGroupId)
And introduces a new StreamsConfig.EXACTLY_ONCE_V2 config. Additionally, this PR replaces usages of the term "eos-beta" throughout the code with the term "eos-v2"
Reviewers: Matthias J. Sax <mjsax@confluent.io>
The methods have been deprecated since 0.11 without replacement since
message format 2 moved the checksum to the record batch (instead of the
record).
Unfortunately, we did not deprecate the constructors that take a checksum
(even though we intended to) so we cannot remove them. I have deprecated
them for removal in 4.0 and added a single non deprecated constructor to
`ConsumerRecord` and `RecordMetadata` that take all remaining parameters.
`ConsumerRecord` could do with one additional convenience constructor, but
that requires a KIP and hence should be done separately.
Also:
* Removed `ChecksumMessageFormatter`, which is technically not public
API, but may have been used with the console consumer.
* Updated all usages of `ConsumerRecord`/`RecordMetadata` constructors
to use the non deprecated ones.
* Added tests for deprecated `ConsumerRecord/`RecordMetadata`
constructors.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, David Jacot <djacot@confluent.io>
`Sum` and `Total` classes were deprecated and replaced by `WindowedSum` and `CumulativeSum` in 2.4. This patch removes them for 3.0.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Ismael Juma <ismael@juma.me.uk>
`quota.producer.default` and `quota.consumer.default` were deprecated in AK 0.11.0.0. Dynamic default quotas must be used instead. This patch removes them for AK 3.0.
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>, Ismael Juma <ismael@juma.me.uk>
* Remove `ExtendedSerializer` and `ExtendedDeserializer`, deprecated since 2.1.
The extra functionality was also made available in `Serializer` and `Deserializer`.
* Remove `close(long, TimeUnit)` from the producer, consumer and admin client,
deprecated since 2.0 for the consumer and 2.2 for the rest. The replacement is `close(Duration)`.
* Remove `ConsumerConfig.addDeserializerToConfig` and `ProducerConfig.addSerializerToConfig`,
deprecated since 2.7 with no replacement. These methods were not intended to be public API
and are likely not used much (if at all).
* Remove `NoOffsetForPartitionException.partition()`, deprecated since 0.11. `partitions()`
should be used instead.
* Remove `MessageFormatter.init(Properties)`, deprecated since 2.7. The `configure(Map)`
method should be used instead.
* Remove `kafka.common.MessageFormatter`, deprecated since 2.7.
`org.apache.kafka.common.MessageFormatter` should be used instead.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, David Jacot <djacot@confluent.io>
These were deprecated in Apache Kafka 2.4 (released in December 2019) to be replaced
by `org.apache.kafka.server.authorizer.Authorizer` and `AclAuthorizer`.
As part of KIP-500, we will implement a new `Authorizer` implementation that relies
on a topic (potentially a KRaft topic) instead of `ZooKeeper`, so we should take the chance
to remove related tech debt in 3.0.
Details on the issues affecting the old Authorizer interface can be found in the KIP:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-504+-+Add+new+Java+Authorizer+Interface
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>, Ron Dagostino <rdagostino@confluent.io>
The config has been deprecated since Kafka 2.6 (released ~1 year before
3.0), but it was the default before it got deprecated. As such, it's
reasonably unlikely that people would have set it explicitly.
Given the confusing `default` name even though it's _not_ the default, I
think we should remove it in 3.0.
Also remove `ClientDnsLookup.DEFAULT` (not public API), which unlocks
a number of code simplications.
Reviewers: David Jacot <djacot@confluent.io>
The `kafka-preferred-replica-election` command was deprecated in 2.4. This path removes it for 3.0. `kafka-leader-election` can be used instead.
Reviewers: Ismael Juma <ismael@juma.me.uk>, Chia-Ping Tsai <chia7712@gmail.com>, Jason Gustafson <jason@confluent.io>
ConfigEntry's public constructor was deprecated in 1.1.0. This patch removes it in AK 3.0.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Ismael Juma <ismael@juma.me.uk>
`Admin.electLeaders` is the replacement since the deprecation in Apache Kafka 2.4.0.
The methods were originally introduced in Apache Kafka 2.2.0, so they were only
non deprecated for two releases.
Reviewers: David Jacot <djacot@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
More specifically, remove deprecated:
- Constants in SslConfigs
- Constants in SaslConfigs
- AclBinding constructor
- AclBindingFilter constructor
- PrincipalBuilder and DefaultPrincipalBuilder classes
- ResourceFilter
Also simplify tests and code that no longer have to handle the removed `PrincipalBuilder`.
These removals seem non controversial. There is a straightforward alternative. The
deprecations happened in 1.0.0 and 2.0.0.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
Gradle 7.0 is required for Java 16 compatibility and it removes a number of
deprecated APIs. Fix most issues preventing the upgrade to Gradle 7.0.
The remaining ones are more complicated and should be handled
in a separate PR. Details of the changes:
* Release tarball no longer includes includes test, sources, javadoc and test sources jars (these
are still published to the Maven Central repository).
* Replace `compile` with `api` or `implementation` - note that `implementation`
dependencies appear with `runtime` scope in the pom file so this is a (positive)
change in behavior
* Add missing dependencies that were uncovered by the usage of `implementation`
* Replace `testCompile` with `testImplementation`
* Replace `runtime` with `runtimeOnly` and `testRuntime` with `testRuntimeOnly`
* Replace `configurations.runtime` with `configurations.runtimeClasspath`
* Replace `configurations.testRuntime` with `configurations.testRuntimeClasspath` (except for
the usage in the `streams` project as that causes a cyclic dependency error)
* Use `java-library` plugin instead of `java`
* Use `maven-publish` plugin instead of deprecated `maven` plugin - this changes the
commands used to publish and to install locally, but task aliases for `install` and
`uploadArchives` were added for backwards compatibility
* Removed `-x signArchives` line from the readme since it was wrong (it was a
no-op before and it fails now, however)
* Replaces `artifacts` block with an approach that works with the `maven-publish` plugin
* Don't publish `jmh-benchmark` module - the shadow jar is pretty large and not
particularly useful (before this PR, we would publish the non shadow jars)
* Replace `version` with `archiveVersion`, `baseName` with `archiveBaseName` and
`classifier` with `archiveClassifier`
* Update Gradle and plugins to the latest stable version (7.0 is not stable yet)
* Use `plugin` DSL to configure plugins
* Updated notable changes for 3.0
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Randall Hauch <rhauch@gmail.com>
Implements KIP-418, that deprecated the `branch()` operator in favor of the newly added and type-safe `split()` operator.
Reviewers: Matthias J. Sax <matthias@confluent.io>, John Roesler <john@confluent.io>
The previous dynamic configuration validation did not actually compare new configs to original configs as intended, so the expected exception was not thrown.
Reviewers: Ismael Juma <ismael@juma.me.uk>, Rajini Sivaram <rajinisivaram@googlemail.com>, Jason Gustafson <jason@confluent.io>
- part of KIP-572
- removed the usage of `retries` in `GlobalStateManger`
- instead of retries the new `task.timeout.ms` config is used
Reviewers: John Roesler <john@confluent.io>, Boyang Chen <boyang@confluent.io>, Guozhang Wang <guozhang@confluent.io>
- part of KIP-572
- deprecates producer config `retries` (still in use)
- deprecates admin config `retries` (still in use)
- deprecates Kafka Streams config `retries` (will be ignored)
- adds new Kafka Streams config `task.timeout.ms` (follow up PRs will leverage this new config)
Reviewers: John Roesler <john@confluent.io>, Jason Gustafson <jason@confluent.io>, Randall Hauch <randall@confluent.io>
Brokers currently return NOT_LEADER_FOR_PARTITION to producers and REPLICA_NOT_AVAILABLE to consumers if a replica is not available on the broker during reassignments. Non-Java clients treat REPLICA_NOT_AVAILABLE as a non-retriable exception, Java consumers handle this error by explicitly matching the error code even though it is not an InvalidMetadataException. This PR renames NOT_LEADER_FOR_PARTITION to NOT_LEADER_OR_FOLLOWER and uses the same error for producers and consumers. This is compatible with both Java and non-Java clients since all clients handle this error code (6) as retriable exception. The PR also makes ReplicaNotAvailableException a subclass of InvalidMetadataException.
- ALTER_REPLICA_LOG_DIRS continues to return REPLICA_NOT_AVAILABLE. Retained this for compatibility since this request never returned NOT_LEADER_FOR_PARTITION earlier.
- MetadataRequest version 0 also returns REPLICA_NOT_AVAILABLE as topic-level error code for compatibility. Newer versions filter these out and return Errors.NONE, so didn't change this.
- Partition responses in MetadataRequest return REPLICA_NOT_AVAILABLE to indicate that one of the replicas is not available. Did not change this since NOT_LEADER_FOR_PARTITION is not suitable in this case.
Reviewers: Ismael Juma <ismael@juma.me.uk>, Jason Gustafson <jason@confluent.io>, Bob Barrett <bob.barrett@confluent.io>
This applies to the producer, consumer, admin client, connect worker
and inter broker communication.
`ClientDnsLookup.DEFAULT` has been deprecated and a warning
will be logged if it's explicitly set in a client config.
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Ismael Juma <ismael@juma.me.uk>
1. Enables `TLSv1.3` by default with Java 11 or newer.
2. Add unit tests that cover the various TLSv1.2 and TLSv1.3 combinations.
3. Extend `benchmark_test.py` and `replication_test.py` to run with 'TLSv1.2'
or 'TLSv1.3'.
Reviewers: Ismael Juma <ismael@juma.me.uk>
*More detailed description of your change,
if necessary. The PR title and PR message become
the squashed commit message, so use a separate
comment to ping reviewers.*
*Summary of testing strategy (including rationale)
for the feature or bug fix. Unit and/or integration
tests are expected for any behaviour change and
system tests should be considered for larger changes.*
Author: Ron Dagostino <rdagostino@confluent.io>
Reviewers: Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
Closes#8139 from rondagostino/KAFKA-9575
Removes references to the old scala Acl classes from kafka.security.auth (Acl, Operation, ResourceType, Resource etc.) and replaces these with the Java API. Only the old SimpleAclAuthorizer, AuthorizerWrapper used to wrap legacy authorizer instances and tests using SimpleAclAuthorizer continue to use the old API. Deprecates the old scala API.
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
* Adjust build and documentation.
* Use lambda syntax for SAM types in `core`, `streams-scala` and
`connect-runtime` modules.
* Remove `runnable` and `newThread` from `CoreUtils` as lambda
syntax for SAM types make them unnecessary.
* Remove stale comment in `FunctionsCompatConversions`,
`KGroupedStream`, `KGroupedTable' and `KStream` about Scala 2.11,
the conversions are needed for Scala 2.12 too.
* Deprecate `org.apache.kafka.streams.scala.kstream.Suppressed`
and use `org.apache.kafka.streams.kstream.Suppressed` instead.
* Use `Admin.create` instead of `AdminClient.create`. Static methods
in Java interfaces can be invoked since Scala 2.12. I noticed that
MirrorMaker 2 uses `AdminClient.create`, but I did not change them
as Connectors have restrictions on newer client APIs.
* Improve efficiency in a few `Gauge` implementations by avoiding
unnecessary intermediate collections.
* Remove pointless `Option.apply` in `ZookeeperClient`
`SessionState` metric.
* Fix unused import/variable and other compiler warnings.
* Reduce visibility of some vals/defs.
Reviewers: Manikumar Reddy <manikumar@confluent.io>, Guozhang Wang <wangguoz@gmail.com>, Gwen Shapira <gwen@confluent.io>
Author: Manikumar Reddy <manikumar.reddy@gmail.com>
Reviewers: Ismael Juma <ismael@juma.me.uk>, Magnus Edenhill <magnus@edenhill.se>, Colin Patrick McCabe <cmccabe@apache.org>
Closes#7818 from omkreddy/zk-note
ZK upgrade from 3.4.X to 3.5.6 fails with "java.io.IOException: No snapshot found" if there are no snapshot files. This was discussed in https://issues.apache.org/jira/browse/ZOOKEEPER-3056
Author: Manikumar Reddy <manikumar.reddy@gmail.com>
Reviewers: Ismael Juma <ismael@juma.me.uk>
Closes#7625 from omkreddy/zk-upgrade
Add a paragraph explaining the producer caller's expected behavior change on record validation failure scenarios that are improved by KIP-467.
Reviewers: Tu V. Tran <tu@confluent.io>, Jason Gustafson <jason@confluent.io>
Document the upgrade path for the consumer and for Streams (note that they differ significantly).
Needs to be cherry-picked to 2.4
Reviewers: Guozhang Wang <wangguoz@gmail.com>
Implement the revisions to the controller state machine and reassignment logic needed for KIP-455.
Add the addingReplicas and removingReplicas field to the topics ZNode.
Deprecate the methods initiating a reassignment via direct ZK access in KafkaZkClient.
Add ControllerContextTest, and add some test cases to ReassignPartitionsClusterTest.
Add a note to upgrade.html recommending not initiating reassignments during an upgrade.
Reviewers: Colin P. McCabe <cmccabe@apache.org>, Viktor Somogyi <viktorsomogyi@gmail.com>
Splits the existing StickyAssignor logic into an AbstractStickyAssignor class, which is extended by the existing (eager) StickyAssignor and by the new CooperativeStickyAssignor which supports incremental cooperative rebalancing.
There is no actual change to the logic -- most methods from StickyAssignor were moved to AbstractStickyAssignor to be shared with CooperativeStickyAssignor, and the abstract MemberData memberData(Subscription) method converts the Subscription to the embedded list of owned partitions for each assignor.
The "generation" logic is left in, however this is always Optional.empty() for the CooperativeStickyAssignor as onPartitionsLost should always be called when a generation is missed.
Reviewers: Jason Gustafson <jason@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
Follow up to new PartitionAssignor interface merged in 7108 is merged
Adds a PartitionAssignorAdapter class to maintain backwards compatibility
Reviewers: Boyang Chen <boyang@confluent.io>, Jason Gustafson <jason@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
Api exception types usually have a single argument constructor which accepts the exception message. However, some types actually use this constructor to initialize a field. This inconsistency has led to some cases where exception messages were being incorrectly passed to these constructors and interpreted incorrectly. For example, this leads to confusing messages like the following in the log when we hit a GROUP_MAX_SIZE_REACHED error:
```
Attempt to join group failed due to fatal error: Consumer group The consumer group has reached its max size. already has the configured ...
```
This patch fixes the problem by changing these constructors so that the exception message is provided consistently. This affected `GroupAuthorizationException`, `TopicAuthorizationException`, and `GroupMaxSizeReachedException`.
Reviewers: Stanislav Kozlovski <stanislav_kozlovski@outlook.com>, Kamal Chandraprakash <kamal.chandraprakash@gmail.com>, Ismael Juma <ismael@juma.me.uk>
This commit makes three changes:
- Adds a constructor for NewTopic(String, Optional<Integer>, Optional<Short>)
which allows users to specify Optional.empty() for numPartitions or
replicationFactor in order to use the broker default.
- Changes AdminManager to accept -1 as valid options for replication
factor and numPartitions (resolving to broker defaults).
- Makes --partitions and --replication-factor optional arguments when creating
topics using kafka-topics.sh.
- Adds a dependency on scalaJava8Compat library to make it simpler to
convert Scala Option to Java Optional
Reviewers: Ismael Juma <ismael@juma.me.uk>, Ryanne Dolan <ryannedolan@gmail.com>, Jason Gustafson <jason@confluent.io>
*More detailed description of your change,
if necessary. The PR title and PR message become
the squashed commit message, so use a separate
comment to ping reviewers.*
*Summary of testing strategy (including rationale)
for the feature or bug fix. Unit and/or integration
tests are expected for any behaviour change and
system tests should be considered for larger changes.*
Author: Matthias J. Sax <matthias@confluent.io>
Reviewers: Bill Bejeck
Closes#6490 from mjsax/minor-streams-docs-rocksdb
This PR adds a upgrade notes and changes examples to use the bootstrap-server.
Author: Viktor Somogyi-Vass <viktorsomogyi@gmail.com>
Reviewers: Srinivas <srinivas96alluri@gmail.com>, Manikumar Reddy <manikumar.reddy@gmail.com>
Closes#6118 from viktorsomogyi/topiccommand-adminclient-doc
Improve the default group id behavior by:
* changing the default consumer group to null, where no offset commit or fetch, or group management operations are allowed
* deprecating the use of empty (`""`) consumer group on the client
Reviewers: Jason Gustafson <jason@confluent.io>
We seemed to be missing the usual rolling upgrade instructions so I've added them and emphasized the impact for downgrades after bumping the inter-broker protocol version.
Author: Jason Gustafson <jason@confluent.io>
Reviewers: Dong Lin <lindong28@gmail.com>, Ismael Juma <ismael@juma.me.uk>
Closes#5857 from hachikuji/KAFKA-7481
Author: Viktor Somogyi <viktorsomogyi@gmail.com>
Reviewers: Ismael Juma <ismael@juma.me.uk>, Andras Katona <41361962+akatona84@users.noreply.github.com>, Dong Lin <lindong28@gmail.com>
Closes#5685 from viktorsomogyi/upgrade-notes-for-serializer-consolidation
"Jetty 9.4.12 includes compatibility for JDK 11. Additionally, TLS 1.3 support has been implemented. While full functionality for new JDK features is not yet supported, this release has been built and tested for compatibility with the latest releases from Oracle."
http://dev.eclipse.org/mhonarc/lists/jetty-announce/msg00124.html
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>
Use delivery timeout instead of retries when possible and remove various TODOs associated with completion of KIP-91.
Reviewers: Ismael Juma <ismael@juma.me.uk>, Guozhang Wang <wangguoz@gmail.com>
ZooKeeper client from version 3.4.13 doesn't handle connections to localhost very well. If ZooKeeper is started on 127.0.0.1 on a machine that has both ipv4 and ipv6 and a client is created using localhost rather than the IP address in the connection string, ZooKeeper client attempts to connect to ipv4 or ipv6 randomly with a fixed one second backoff if connection fails. Use 127.0.0.1 instead of localhost in streams tests to avoid intermittent test failures due to ZK client connection timeouts if ipv6 is chosen in consecutive address selections. Also add note to upgrade docs for 2.0.0.
Reviewers: Ismael Juma <github@juma.me.uk>, Matthias J. Sax <matthias@confluent.io>
This patch contains the improved offset expiration semantics proposed in KIP-211. Committed offsets will not be expired as long as a group is active. Once all members have left the group, then offsets will be expired after the timeout configured by `offsets.retention.minutes`. Note that the optimization for early expiration of unsubscribed topics will be implemented in a separate patch.
- Removed Scala consumers (`SimpleConsumer` and `ZooKeeperConsumerConnector`)
and their tests.
- Removed Scala request/response/message classes.
- Removed any mention of new consumer or new producer in the code
with the exception of MirrorMaker where the new.consumer option was
never deprecated so we have to keep it for now. The non-code
documentation has not been updated either, that will be done
separately.
- Removed a number of tools that only made sense in the context
of the Scala consumers (see upgrade notes).
- Updated some tools that worked with both Scala and Java consumers
so that they only support the latter (see upgrade notes).
- Removed `BaseConsumer` and related classes apart from `BaseRecord`
which is used in `MirrorMakerMessageHandler`. The latter is a pluggable
interface so effectively public API.
- Removed `ZkUtils` methods that were only used by the old consumers.
- Removed `ZkUtils.registerBroker` and `ZKCheckedEphemeral` since
the broker now uses the methods in `KafkaZkClient` and no-one else
should be using that method.
- Updated system tests so that they don't use the Scala consumers except
for multi-version tests.
- Updated LogDirFailureTest so that the consumer offsets topic would
continue to be available after all the failures. This was necessary for it
to work with the Java consumer.
- Some multi-version system tests had not been updated to include
recently released Kafka versions, fixed it.
- Updated findBugs and checkstyle configs not to refer to deleted
classes and packages.
Reviewers: Dong Lin <lindong28@gmail.com>, Manikumar Reddy <manikumar.reddy@gmail.com>
Support configuration of dynamic broker configs in ZooKeeper before starting brokers using ConfigCommand. This will allow password configs to be encrypted and stored in ZooKeeper, without requiring clear passwords in server.properties to bootstrap the broker first.
Reviewers: Jason Gustafson <jason@confluent.io>
We might decide to drop certain message batches during down-conversion because older clients might not be able to interpret them. One such example is control batches which are typically removed by the broker if down-conversion to V0 or V1 is required. This patch makes sure the chunked down-conversion implementation is able to handle such cases.
This patch changes the default `request.timeout.ms` of the consumer to 30 seconds. Additionally, it adds logic to `NetworkClient` and related to components to support timeouts at the request level. We use this to handle the special case of the JoinGroup request, which may block for as long as the value configured by `max.poll.interval.ms`.
Reviewers: Ismael Juma <ismael@juma.me.uk>, Guozhang Wang <guozhang@confluent.io>
Adds a configuration that specifies the default timeout for KafkaConsumer APIs that could block. This was introduced in KIP-266.
Reviewers: Satish Duggana <satish.duggana@gmail.com>, Jason Gustafson <jason@confluent.io>
Keep Literal ACLs on the old paths, using the old formats, to maintain backwards compatibility.
Have Prefixed, and any latter types, go on new paths, using JSON, (old brokers are not aware of them)
Add checks to reject any adminClient requests to add prefixed acls before the cluster is fully upgraded.
Colin Patrick McCabe <colin@cmccabe.xyz>, Jun Rao <junrao@gmail.com>
It takes O(n^2) time to instantiate a mbean with n attributes which can be very slow if the number of attributes of this mbean is large. This PR removes metrics whose number of attributes can grow with the number of partitions in the cluster to fix the performance issue. These metrics have already been marked for removal in 2.0 by KIP-225.
Author: Dong Lin <lindong28@gmail.com>
Reviewers: Ismael Juma <ismael@juma.me.uk>
Closes#5172 from lindong28/remove-deprecated-metrics
Reviewers: Colin Patrick McCabe <colin@cmccabe.xyz>, Jun Rao <junrao@gmail.com>
Co-authored-by: Piyush Vijay <pvijay@apple.com>
Co-authored-by: Andy Coates <big-andy-coates@users.noreply.github.com>
- CreateTopicsRequest now requires Create auth on Topic resource
or Create on Cluster resource.
- AclCommand --producer option adjusted
- Existing unit and Integration tests adjusted accordingly and
new tests added.
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>, Ismael Juma <ismael@juma.me.uk>
Co-authored-by: Edoardo Comar <ecomar@uk.ibm.com>
Co-authored-by: Mickael Maison <mickael.maison@gmail.com>
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
Author: Dong Lin <lindong28@gmail.com>
Reviewers: Jiangjie Qin <becket.qin@gmail.com>, Jun Rao <junrao@gmail.com>, Ismael Juma <ismael@juma.me.uk>, Onur Karaman <okaraman@linkedin.com>
Closes#2929 from lindong28/KAFKA-4763
For topics that support timestamp search, if no offset is found for a partition, the partition should still be included in the result with a `null` offset value. This `KafkaConsumer` method currently excludes such partitions from the result.
Author: Vahid Hashemian <vahidhashemian@us.ibm.com>
Reviewers: Ismael Juma <ismael@juma.me.uk>, Jason Gustafson <jason@confluent.io>
Closes#3460 from vahidhashemian/KAFKA-5534
We had originally increased Snappy’s block size as part of KAFKA-3704. However,
we had some issues with excessive memory usage in the producer and we reverted
it in 7c6ee8d5e.
After more investigation, we fixed the underlying reason why memory usage seemed
to grow much more than expected via KAFKA-3747 (included in 0.10.0.1).
In 0.10.2, we changed the broker to use the same classes as the producer and the
broker’s block size for Snappy was changed from 32 KB to 1KB. As reported in
KAFKA-5236, the on disk size is, in some cases, 50% larger when the data is compressed
with 1 KB instead of 32 KB as the block size.
As discussed in KAFKA-3704, it may be worth making this configurable and/or allocate
the compression buffers from the producer pool. However, for 0.11.0.0, I think the
simplest thing to do is to default to 32 KB for Snappy (the default if no block size
is provided).
I also increased the Gzip buffer size. 1 KB is too small and the default is smaller
still (512 bytes). 8 KB (which is the default buffer size for BufferedOutputStream)
seemed like a reasonable default.
Author: Ismael Juma <ismael@juma.me.uk>
Reviewers: Jason Gustafson <jason@confluent.io>
Closes#3205 from ijuma/kafka-5236-snappy-block-size
Author: Matthias J. Sax <matthias@confluent.io>
Reviewers: Ismael Juma, Ewen Chesklack-Postava, Bill Bejeck, Guozhang Wang
Closes#3194 from mjsax/minor-update-docs-for-kip-123
Add a new entry in upgrade.html for `group.initial.rebalance.delay.ms`
Author: Damian Guy <damian.guy@gmail.com>
Reviewers: Ismael Juma <ismael@juma.me.uk>, Guozhang Wang <wangguoz@gmail.com>
Closes#3189 from dguy/relabance-delay
Author: Jason Gustafson <jason@confluent.io>
Reviewers: Apurva Mehta <apurva@confluent.io>, Ismael Juma <ismael@juma.me.uk>
Closes#3123 from hachikuji/KAFKA-4935
Update upgrade.html
Raising this now, as KIP-118 is pulled from release as such submitting this without java 8 changes.
As per remaining review comment from https://github.com/apache/kafka/pull/2772, updating the upgrade notes.
Author: Michael André Pearce <michael.andre.pearce@me.com>
Author: Michael Andre Pearce <Michael.Andre.Pearce@me.com>
Reviewers: Jiangjie Qin <becket.qin@gmail.com>, Ismael Juma <ismael@juma.me.uk>
Closes#2991 from michaelandrepearce/KIP-82
This only removes deprecated methods,
fields and constructors in a small number of classes.
Deprecated producer configs is tracked via KAFKA-3353
and the old clients and related (tools, etc.) won't
be removed in 0.11.0.0.
Author: Ismael Juma <ismael@juma.me.uk>
Reviewers: Jason Gustafson <jason@confluent.io>
Closes#2995 from ijuma/kafka-3763-remove-deprecated-0.11
These configs have been deprecated since 0.9.0.0:
block.on.buffer.full, metadata.fetch.timeout.ms and timeout.ms
Author: Ismael Juma <ismael@juma.me.uk>
Reviewers: Jason Gustafson <jason@confluent.io>
Closes#2987 from ijuma/kafka-3353-remove-deprecated-producer-configs
Add a default log retention policy to keep GC logs from growing too large
Author: Ryan P <ryan.n.pridgeon@gmail.com>
Reviewers: Ismael Juma <ismael@juma.me.uk>
Closes#1431 from rnpridgeon/KAFKA-3754
Introduced in PR #2824. Already fixed in the
website github.
Author: Gwen Shapira <cshapi@gmail.com>
Reviewers: Ismael Juma <ismael@juma.me.uk>
Closes#2844 from gwenshap/docs-hotfix
(cherry picked from commit 5f728532ac)
Signed-off-by: Ismael Juma <ismael@juma.me.uk>
Author: Dong Lin <lindong28@gmail.com>
Reviewers: Jun Rao <junrao@gmail.com>, Ismael Juma <ismael@juma.me.uk>, Jiangjie Qin <becket.qin@gmail.com>
Closes#2476 from lindong28/KAFKA-4586
Author: Jason Gustafson <jason@confluent.io>
Reviewers: Jun Rao <junrao@gmail.com>, Apurva Mehta <apurva@confluent.io>, Guozhang Wang <wangguoz@gmail.com>, Ismael Juma <ismael@juma.me.uk>
Closes#2614 from hachikuji/exactly-once-message-format
Author: Jiangjie Qin <becket.qin@gmail.com>
Reviewers: Jun Rao <junrao@gmail.com>, Ismael Juma <ismael@juma.me.uk>
Closes#2544 from becketqin/KAFKA-4340_follow_up
1. Added an architecture section.
2. Added a configuration / execution sub-section to developer guide.
Minor tweaks and a bunch of missing fixes from `kafka-site` repo.
Author: Guozhang Wang <wangguoz@gmail.com>
Reviewers: Matthias J. Sax <matthias@confluent.io>, Derrick Or <derrickor@gmail.com>, Ewen Cheslack-Postava <ewen@confluent.io>
Closes#2488 from guozhangwang/KMinor-streams-docs-second-pass
Author: Matthias J. Sax <matthias@confluent.io>
Reviewers: Ismael Juma, Will Marshall, Damian Guy, Guozhang Wang, Michael G. Noll
Closes#2461 from mjsax/addStreamsUpdateSecton
Kafka brokers have a config called "offsets.topic.replication.factor" that specify the replication factor for the "__consumer_offsets" topic. The problem is that this config isn't being enforced. If an attempt to create the internal topic is made when there are fewer brokers than "offsets.topic.replication.factor", the topic ends up getting created anyway with the current number of live brokers. The current behavior is pretty surprising when you have clients or tooling running as the cluster is getting setup. Even if your cluster ends up being huge, you'll find out much later that __consumer_offsets was setup with no replication.
The cluster not meeting the "offsets.topic.replication.factor" requirement on the internal topic is another way of saying the cluster isn't fully setup yet.
The right behavior should be for "offsets.topic.replication.factor" to be enforced. Topic creation of the internal topic should fail with GROUP_COORDINATOR_NOT_AVAILABLE until the "offsets.topic.replication.factor" requirement is met. This closely resembles the behavior of regular topic creation when the requested replication factor exceeds the current size of the cluster, as the request fails with error INVALID_REPLICATION_FACTOR.
Author: Onur Karaman <okaraman@linkedin.com>
Reviewers: Jason Gustafson <jason@confluent.io>, Ismael Juma <ismael@juma.me.uk>, Ewen Cheslack-Postava <ewen@confluent.io>
Closes#2177 from onurkaraman/KAFKA-3959
Author: Ismael Juma <ismael@juma.me.uk>
Reviewers: Ewen Cheslack-Postava <ewen@confluent.io>, Michael G. Noll
Closes#2445 from ijuma/kafka-4578-upgrade-notes-0.10.2
Updated the docs with changes in KAFKA-4060.
Author: Hojjat Jafarpour <hojjat@Hojjat-Jafarpours-MBP.local>
Author: Hojjat Jafarpour <hojjat@HojjatJpoursMBP.attlocal.net>
Reviewers: Ismael Juma, Matthias J. Sax, Guozhang Wang
Closes#2377 from hjafarpour/KAFKA-4060-docs-update
Author: Rajini Sivaram <rajinisivaram@googlemail.com>
Reviewers: Ismael Juma <ismael@juma.me.uk>, Jason Gustafson <jason@confluent.io>
Closes#2366 from rajinisivaram/KAFKA-4626
This makes it consistent with MirrorMaker with the old consumer.
Author: huxi <huxi@zhenrongbao.com>
Author: amethystic <huxi_2b@hotmail.com>
Reviewers: Vahid Hashemian <vahidhashemian@us.ibm.com>, Ismael Juma <ismael@juma.me.uk>
Closes#2072 from amethystic/kafka-4351_Regex_behavior_change_for_new_consumer
Author: Matthias J. Sax <matthias@confluent.io>
Reviewers: Michael G. Noll, Eno Thereska, Damian Guy, Guozhang Wang
Closes#2117 from mjsax/kafka-4393-improveInvalidTsHandling
And improve readability by adding proper punctuations.
Author: Vahid Hashemian <vahidhashemian@us.ibm.com>
Reviewers: Jason Gustafson <jason@confluent.io>
Closes#2002 from vahidhashemian/doc/fix_typos
This is a minor change to fix the most glaring issues. We have another JIRA to revamp the upgrade docs.
Author: Ismael Juma <ismael@juma.me.uk>
Reviewers: Jeff Klukas <jeff@klukas.net>, Jiangjie Qin <becket.qin@gmail.com>, Jason Gustafson <jason@confluent.io>
Closes#1971 from ijuma/kafka-4257-upgrade-docs-inconsitencies
Reopening of https://github.com/apache/kafka/pull/1428
Author: Edoardo Comar <ecomar@uk.ibm.com>
Author: Mickael Maison <mickael.maison@gmail.com>
Reviewers: Grant Henke <granthenke@gmail.com>, Ismael Juma <ismael@juma.me.uk>, Jason Gustafson <jason@confluent.io>
Closes#1908 from edoardocomar/KAFKA-3396
This is to imply that the Java consumer/producer are the recommended consumer/producer now.
Author: Vahid Hashemian <vahidhashemian@us.ibm.com>
Reviewers: Jason Gustafson <jason@confluent.io>
Closes#1921 from vahidhashemian/KAFKA-3697
Author: Jiangjie Qin <becket.qin@gmail.com>
Reviewers: Ismael Juma <ismael@juma.me.uk>, Jason Gustafson <jason@confluent.io>
Closes#1897 from becketqin/KAFKA-4194
- Updated implementation docs with details on Cluster Id generation.
- Mention cluster id in "noteworthy changes for 0.10.1.0" in upgrade docs.
Author: Sumit Arrawatia <sumit.arrawatia@gmail.com>
Author: arrawatia <sumit.arrawatia@gmail.com>
Reviewers: Ismael Juma <ismael@juma.me.uk>
Closes#1895 from arrawatia/kip-78-docs
Author: Jiangjie Qin <becket.qin@gmail.com>
Reviewers: Jun Rao <junrao@gmail.com>, Ismael Juma <ismael@juma.me.uk>, Jason Gustafson <jason@confluent.io>
Closes#1852 from becketqin/KAFKA-4148
Include a few clean-ups (also in producer section), mention deprecation plans and reorder so that the new consumer documentation is before the old consumers.
Author: Ismael Juma <ismael@juma.me.uk>
Reviewers: Jason Gustafson <jason@confluent.io>
Closes#1880 from ijuma/remove-beta-from-new-consumer-documentation
Author: Jiangjie Qin <becket.qin@gmail.com>
Reviewers: Ismael Juma <ismael@juma.me.uk>, Jun Rao <junrao@gmail.com>
Closes#1809 from becketqin/KAFKA-4099
The reasons to remove it are:
1. It's currently broken. The purpose of the [JIRA](https://issues.apache.org/jira/browse/KAFKA-3761) was to report that the RunningAsController state gets overwritten back to "RunningAsBroker".
2. It's not a useful state.
a. If clients want to use this metric to know whether a broker is ready to receive requests or not, they do not care whether or not the broker is the controller
b. there is already a separate boolean property, KafkaController.isActive which contains this information.
Author: Roger Hoover <roger.hoover@gmail.com>
Reviewers: Grant Henke <granthenke@gmail.com>, Ismael Juma <ismael@juma.me.uk>
Closes#1437 from theduderog/KAFKA-3761-broker-state
We had mentioned this step in the performance impact section in the middle of a long paragraph, which made it easy to miss. I also tweaked the reason for setting `log.message.format.version` as it could be misinterpreted previously.
Author: Ismael Juma <ismael@juma.me.uk>
Reviewers: Ewen Cheslack-Postava <ewen@confluent.io>
Closes#1514 from ijuma/tweak-upgrade-notes
Author: MayureshGharat <gharatmayuresh15@gmail.com>
Reviewers: Grant Henke <granthenke@gmail.com>, Ismael Juma <ismael@juma.me.uk>
Closes#1060 from MayureshGharat/KAFKA-3393
This is not an exact revert as the code changed a bit since the
original commit. We also include a note in `upgrade.html`.
The original commit is 1182d61deb.
Author: Ismael Juma <ismael@juma.me.uk>
Reviewers: Gwen Shapira, Guozhang Wang
Closes#1391 from ijuma/kafka-3704-revert and squashes the following commits:
7891b67 [Ismael Juma] Tweak upgrade note based on Gwen's feedback
1673cd0 [Ismael Juma] Revert "KAFKA-3704: Remove hard-coded block size in KafkaProducer"
If you're pushing close to the network capacity, 0.10's additional 8 bytes per message can lead to overload of your network. We (Heroku Kafka) ran into this issue whilst benchmarking 0.10 RC and the ijuma suggested it belonged in the update note.
Comments/suggestions welcome.
Author: Tom Crayford <tcrayford@googlemail.com>
Reviewers: Jun Rao <junrao@gmail.com>, Ismael Juma <ismael@juma.me.uk>
Closes#1389 from tcrayford/upgrade_note_about_increased_network_bandwidth
…e throughput drop due to the addition of timestamp field.
Author: Jiangjie Qin <becket.qin@gmail.com>
Reviewers: Ismael Juma <ismael@juma.me.uk>, Jun Rao <junrao@gmail.com>
Closes#1372 from becketqin/KAFKA-3565
This fixes test_producer_throughput with compression_type=snappy.
Also: added heap dump on out of memory error to `producer_performance.py` and corrected the upgrade note related to the change in buffer size for compression streams.
Author: Ismael Juma <ismael@juma.me.uk>
Reviewers: Gwen Shapira
Closes#1385 from ijuma/kafka-3713-test_producer_throughput-snappy-fail and squashes the following commits:
54c7962 [Ismael Juma] Correct upgrade note about buffer size for compression stream
515040b [Ismael Juma] Call `compressor.close()` to fix memory leak
5311e5b [Ismael Juma] Dump heap on out of memory error when running `producer_performance.py`
Author: Ismael Juma <ismael@juma.me.uk>
Reviewers: Guozhang Wang <wangguoz@gmail.com>, Jun Rao <junrao@gmail.com>
Closes#1373 from ijuma/fix-producer-buffer-size-upgrade-note
Author: Guozhang Wang <wangguoz@gmail.com>
Reviewers: Michael G. Noll <michael@confluent.io>, Ismael Juma <ismael@juma.me.uk>
Closes#1321 from guozhangwang/KStreamsJavaDoc
This contribution is my original work and I license the work under Apache 2.0.
Author: Dana Powers <dana.powers@gmail.com>
Author: Ismael Juma <ismael@juma.me.uk>
Reviewers: Jun Rao <junrao@gmail.com>, Ismael Juma <ismael@juma.me.uk>
Closes#1212 from dpkp/KAFKA-3160
Please see https://cwiki.apache.org/confluence/display/KAFKA/KIP-36+Rack+aware+replica+assignment for the overall design.
The update to TopicMetadataRequest/TopicMetadataResponse will be done in a different PR.
Author: Allen Wang <awang@netflix.com>
Author: Ismael Juma <ismael@juma.me.uk>
Reviewers: Ismael Juma <ismael@juma.me.uk>, Jason Gustafson <jason@confluent.io>, Grant Henke <granthenke@gmail.com>, Jun Rao <junrao@gmail.com>
Closes#132 from allenxwang/KAFKA-1215
* Change `MessageFormat.writeTo` to take a `ConsumerRecord`
* Change `MessageReader.readMessage()` to use `ProducerRecord`
Author: Ismael Juma <ismael@juma.me.uk>
Reviewers: Jun Rao <junrao@gmail.com>
Closes#972 from ijuma/kafka-3273-message-formatter-and-reader-resilient
This PR includes a number of clean-ups:
* Code style
* Documentation wording improvements
* Efficiency improvements
Author: Ismael Juma <ismael@juma.me.uk>
Reviewers: Jun Rao <junrao@gmail.com>
Closes#943 from ijuma/kafka-3259-kip-31-32-clean-ups
See KIP-31 and KIP-32 for details.
A few notes on the patch:
1. This patch implements KIP-31 and KIP-32. The patch includes features in both KAFKA-3025, KAFKA-3026 and KAFKA-3036
2. All unit tests passed.
3. The unit tests were run with new and old message format.
4. When message format conversion occurs during consumption, the consumer will not be able to detect the message size too large situation. I did not try to fix this because the situation seems rare and only happen during migration phase.
Author: Jiangjie Qin <becket.qin@gmail.com>
Author: Ismael Juma <ismael@juma.me.uk>
Author: Jiangjie (Becket) Qin <becket.qin@gmail.com>
Reviewers: Jason Gustafson <jason@confluent.io>, Anna Povzner <anna@confluent.io>, Ismael Juma <ismael@juma.me.uk>, Guozhang Wang <wangguoz@gmail.com>, Jun Rao <junrao@gmail.com>
Closes#764 from becketqin/KAFKA-3025
Set default to 1 instead of 1024, this matches the existing doc and feels like a better default value. Have run the unit tests with the change.
Author: Rajini Sivaram <rajinisivaram@googlemail.com>
Reviewers: Jason Gustafson, Ismael Juma, Grant Henke, Guozhang Wang
Closes#832 from rajinisivaram/KAFKA-3170
Provides a configuration to opt out of broker id generation.
Author: Grant Henke <granthenke@gmail.com>
Reviewers: Gwen Shapira
Closes#762 from granthenke/id-generation
Fix two minor typos in Kafka official website page.
Author: Gabriel Zhang <smalldirector@yahoo.com>
Reviewers: Ismael Juma, Guozhang Wang
Closes#742 from smalldirector/kafka-document-typos-fix
Author: Grant Henke <granthenke@gmail.com>
Reviewers: Ismael Juma <ismael@juma.me.uk>, Jun Rao <junrao@gmail.com>
Closes#686 from granthenke/compaction
Credit to Gwen for some of the text.
Author: Ismael Juma <ismael@juma.me.uk>
Reviewers: Guozhang Wang <wangguoz@gmail.com>, Ewen Cheslack-Postava <ewen@confluent.io>
Closes#678 from ijuma/mirror-maker-compatibility-note
People are facing problems upgrading their clusters with configured broker IDs above 1000 due to `reserved.broker.max.id` which wasn't very well announced.
This PR attempts to improve that somewhat by fixing the broker config docs and adding a note to the upgrade documentation.
Author: Magnus Edenhill <magnus@edenhill.se>
Reviewers: Grant Henke <granthenke@gmail.com>, Ewen Cheslack-Postava <ewen@confluent.io>
Closes#670 from edenhill/docs-reserved.broker.max.id
Often it is useful to link to a specific header within the documentation. Especially when referencing docs in the mailing lists.
This adds anchors and links for all headers in the docs.
Author: Grant Henke <granthenke@gmail.com>
Reviewers: Jun Rao <junrao@gmail.com>
Closes#498 from granthenke/doc-links
This PR copies the latest kafka docs to kafka repo docs directory. Here I have copied 0.8.3/ directory contents from svn website repo to kafka/docs repository.
Some questions: This PR contains generated javadocs also. Do we need to copy javadocs here?
Author: Manikumar reddy O <manikumar.reddy@gmail.com>
Reviewers: Gwen Shapira, Ismael Juma
Closes#171 from omkreddy/KAFKA-2425-MOVE-DOCS-TO-KAFKA-REPO