For metadata request version 6 and above, use a different error code to indicate missing listener on leader broker to enable diagnosis of listener configuration issues.
Reviewers: Ismael Juma <ismael@juma.me.uk>
Also:
- Remove exceptions in `kafka.common` that are no longer used.
- Keep `kafka.common.KafkaException` as it's still used by `ZkUtils`,
`kafka.admin.AdminClient` and `kafka.security.auth` classes and
we would like to maintain compatibility for now.
- Add deprecated annotation to `kafka.admin.AdminClient`. The scaladoc
stated that the class is deprecated, but the annotation was missing.
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>, Manikumar Reddy <manikumar.reddy@gmail.com>
A broker with multiple log dirs will die on startup if dir.getCanonicalPath() throws
IOException for one of the log dirs. We should mark such log directory as offline
instead and the broker should start if there is a healthy log dir.
Reviewers: Ismael Juma <ismael@juma.me.uk>
This patch fixes the following issues in the log splitting logic added to address KAFKA-6264:
1. We were not handling the case when all messages in the segment overflowed the index. In this case, there is only one resulting segment following the split.
2. There was an off-by-one error in the recovery logic when completing a swap operation which caused an unintended segment deletion.
Additionally, this patch factors out of `splitOverflowedSegment` a method to write to a segment using from with an instance of `FileRecords`. This allows for future reuse and isolated testing.
Reviewers: Dhruvil Shah <dhruvil@confluent.io>, Ismael Juma <ismael@juma.me.uk>, Jun Rao <junrao@gmail.com>
- 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>
Avoid unnecessary processing of SSL channels when there are some bytes buffered, but not enough to make progress.
Reviewers: Radai Rosenblatt <radai.rosenblatt@gmail.com>, Jun Rao <junrao@gmail.com>
This check was left over from the old consumer logic in which the join group was bound by the session timeout. Since we use a custom timeout for JoinGroup, this restriction no longer makes sense.
Reviewers: Ismael Juma <ismael@juma.me.uk>
This version is a WIP and intentionally leaves out some additional required changes to keep the reviewing effort more manageable. This version of the process includes
1. Cleaning up the graph objects to reduce the number of parameters and make the naming conventions more clear.
2. Intercepting all calls to the InternalToplogyBuilder and capturing all details required for possible optimizations and building the final topology.
This PR does not include writing out the current physical plan, so no tests included. The next PR will include additional changes to building the graph and writing the topology out without optimizations, using the current streams tests.
Reviewers: John Roesler <john@confluent.io>, Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <wangguoz@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>
https://issues.apache.org/jira/browse/KAFKA-7058
* Summary of testing strategy: Added new unit test
Author: Gunnar Morling <gunnar.morling@googlemail.com>
Reviewers: Randall Hauch <rhauch@gmail.com>, Ewen Cheslack-Postava <ewen@confluent.io>
Closes#5225 from gunnarmorling/KAFKA-7058
This was originally missed when headers were added as part of KIP-145 in AK 1.1. An additional unit test was added in line with the StringConverter.
This should be backported to the AK `1.1` branch so that it is included in the next bugfix release. The `SimpleHeaderConverter` class that we're referencing was first added in the 1.1.0 release, so there's no reason to backport earlier.
Author: Randall Hauch <rhauch@gmail.com>
Reviewers: Ewen Cheslack-Postava <ewen@confluent.io>
Closes#5204 from rhauch/kafka-7047
Create an instance of the plugin only it's a Versioned Plugin. Prior to KIP-285, this was done for only for Connector and this PR will continue to have the same behavior.
Author: Magesh Nandakumar <magesh.n.kumar@gmail.com>
Reviewers: Randall Hauch <rhauch@gmail.com>, Konstantine Karantasis <konstantine@confluent.io>, Ewen Cheslack-Postava <ewen@confluent.io>
Closes#5191 from mageshn/KAFKA-7039
KIP-305 added numeric converters to Connect, but these were added in Connect’s API module in the same package as the `StringConverter`. This commit moves them into the Runtime module and into the `converters` package where the `ByteArrayConverter` already lives. These numeric converters have not yet been included in a release, and so they can be moved without concern.
All of Connect’s converters must be referenced in worker / connector configurations and are therefore part of the API, but otherwise do not need to be in the “api” module as they do not need to be instantiated or directly used by extensions. This change makes them more similar to and aligned with the `ByteArrayConverter`.
It also gives us the opportunity to move them into the “api” module in the future (keeping the same package name), should we ever want or need to do so. However, if we were to start out with them in the “api” module, we would never be able to move them out into the “runtime” module, even if we kept the same package name. Therefore, moving them to “runtime” now gives us a bit more flexibility.
This PR moves the unit tests for the numeric converters accordingly, and updates the `PluginsUtil` and `PluginUtilsTest` as well.
Author: Randall Hauch <rhauch@gmail.com>
Reviewers: Konstantine Karantasis <konstantine@confluent.io>, Ewen Cheslack-Postava <ewen@confluent.io>
Closes#5222 from rhauch/kafka-7056
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.
The initial PR for KIP-290 #5117 added a new `ResourceNameType`, which was initially a field on `Resource` and `ResourceFilter`. However, follow on PRs have now moved the name type fields to new `ResourcePattern` and `ResourcePatternFilter` classes. This means the old name is no longer valid and may be confusing. The PR looks to rename the class to a more intuitive `resource.PatternType`.
@cmccabe also requested that the current `ANY` value for this class be renamed to avoid confusion. `PatternType.ANY` currently causes `ResourcePatternFilter` to bring back all ACLs that would affect the supplied resource, i.e. it brings back literal, wildcard ACLs, and also does pattern matching to work out which prefix acls would affect the resource. This is very different from the behaviour of `ResourceType.ANY`, which just means the filter ignores the type of resources.
`ANY` is to be renamed to `MATCH` to disambiguate it from other `ANY` filter types. A new `ANY` will be added that works in the same way as others, i.e. it will cause the filter to ignore the pattern type, (but won't do any pattern matching).
Reviewers: Colin Patrick McCabe <colin@cmccabe.xyz>, Jun Rao <junrao@gmail.com>
This does the minimal amount of work so that the tool
relies on public non-deprecated APIs (i.e. it no longer
relies on Scala clients code).
Additional improvements (not included here) have
been proposed via KIP-308.
There are a few other PRs that touch this class with
overlapping goals:
- https://github.com/apache/kafka/pull/2891
- https://github.com/apache/kafka/pull/3051
- https://github.com/apache/kafka/pull/3320
One of them remains relevant in the context of KIP-308, but
the others have been superseded. I included the authors of
the 3 PRs as co-authors.
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>, Vahid Hashemian <vahidhashemian@us.ibm.com>, Manikumar Reddy <manikumar.reddy@gmail.com>
Co-authored-by: Arseniy Tashoyan <tashoyan@gmail.com>
Co-authored-by: Vahid Hashemian <vahidhashemian@us.ibm.com>
Co-authored-by: Mohammed Amine GARMES
Co-authored-by: Ismael Juma <ismael@juma.me.uk>
It is possible that log start offset may fall in the middle of the batch after AdminClient#deleteRecords(). This will cause a follower starting from log start offset to fail fetching (all records). Use-cases when a follower will start fetching from log start offset includes: 1) new replica due to partition re-assignment; 2) new local replica created as a result of AdminClient#AlterReplicaLogDirs(); 3) broker that was down for some time while AdminClient#deleteRecords() move log start offset beyond its HW.
Added two integration tests:
1) Produce and then AdminClient#deleteRecords() while one of the followers is down, and then restart of the follower requires fetching from log start offset;
2) AdminClient#AlterReplicaLogDirs() after AdminClient#deleteRecords()
Reviewers: Ismael Juma <ismael@juma.me.uk>, Jun Rao <junrao@gmail.com>, Jason Gustafson <jason@confluent.io>
For tests that are not testing the old consumers functionality. As part of this,
consolidate `TopicMetadataTest` into `MetadataRequestTest`. Finally,
remove `ProducerBounceTest` which has no tests left in it.
Reviewers: Jason Gustafson <jason@confluent.io>
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>
The old timeout configs no longer take effect, as of
53ca52f855. They are replaced
by the new one.
Reviewers: Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
Upgrade strongly recommended due to security fixes for
jackson-databind (same as ones in 2.7.9.4 and 2.8.11.2).
Reviewers: Matthias J. Sax <matthias@confluent.io>
Reviewers: Matthias J. Sax <matthias@confluent.io>, Bill Bejeck <bill@confluent.io>, Guozhang Wang <guozhang@confluent.io>, Ted Yu <yuzhihong@gmail.com>
* KAFKA-6474: Rewrite tests to use new public TopologyTestDriver [part 2]
* Refactor:
-KTableFilterTest.java
-KTableImplTest.java
-KTableMapValuesTest.java
-KTableSourceTest.java
* Add access to task, processorTopology, and globalTopology in TopologyTestDriver via TopologyTestDriverWrapper
* Remove unnecessary constructor in TopologyTestDriver
* Change how TopologyTestDriverWrapper#getProcessorContext sets the current node
Reviewers: John Roesler <john@confluent.io>, Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
- Update some tests to use the Java consumer.
- Remove ignored `ProducerBounceTest.testBrokerFailure`. This test
is flaky and it has been superseded by `TransactionBounceTest`.
- Use non-blocking poll for consumption methods in `TestUtils`.
This is a step on the road to remove the old consumers.
This could be backported to older branches to reduce the extra log warning messages there, too.
Running Connect system tests in this branch builder job: https://jenkins.confluent.io/job/system-test-kafka-branch-builder/1773/
Author: Randall Hauch <rhauch@gmail.com>
Reviewers: Konstantine Karantasis <konstantine@confluent.io>, Ewen Cheslack-Postava <ewen@confluent.io>
Closes#5151 from rhauch/kafka-7009
Connect API currently depends on Jersey API as a side-effect of KIP-285. It should only depend on the JAX RS API.
Author: Magesh Nandakumar <magesh.n.kumar@gmail.com>
Reviewers: Randall Hauch <rhauch@gmail.com>, Ewen Cheslack-Postava <ewen@confluent.io>
Closes#5190 from mageshn/KAFKA-7031
(cherry picked from commit 51ac53d903)
Signed-off-by: Ewen Cheslack-Postava <me@ewencp.org>
Several recently-added converters are included in the plugin isolation whitelist, similarly to the `StringConverter`. This is a change in the implementation, and does not affect the approved KIP. Several unit tests were added to verify they are being loaded in isolation, again similarly to `StringConverter`.
These changes should be applied only to `trunk` and `2.0`, since these converters were added as part of KIP-305 for AK 2.0.
Author: Randall Hauch <rhauch@gmail.com>
Reviewers: Magesh Nandakumar <magesh.n.kumar@gmail.com>, Konstantine Karantasis <konstantine@confluent.io>, Ewen Cheslack-Postava <ewen@confluent.io>
Closes#5198 from rhauch/kafka-7043
After [3173](https://github.com/apache/kafka/commit/e0150a25e8), the field "activePaths" is not used anymore.
Author: Chia-Ping Tsai <chia7712@gmail.com>
Reviewers: Ewen Cheslack-Postava <ewen@confluent.io>
Closes#4984 from chia7712/remove_unused_field_from_DelegatingClassLoader
A bug in the original KIP-277 submission was caught during code review,
but it was not detected by the tests. Fix that gap.
Co-authored-by: Edoardo Comar <ecomar@uk.ibm.com>
Co-authored-by: Mickael Maison <mickael.maison@gmail.com>
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>
This moves FileConfigProvider to the org.apache.common.config.provider package to more easily isolate provider implementations going forward.
Reviewers: Konstantine Karantasis <konstantine@confluent.io>, Randall Hauch <rhauch@gmail.com>, Ismael Juma <ismael@juma.me.uk>, Jason Gustafson <jason@confluent.io>