Commit Graph

8725 Commits

Author SHA1 Message Date
Bruno Cadonna b519117b22
KAFKA-10357: Add missing repartition topic validation (#10305)
Reviewers: Rohan Desai <rohan@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
2021-03-12 08:59:41 -08:00
Luke Chen 4e60ad72fb
MINOR: Improve error message in MirrorConnectorsIntegrationBaseTest (#10268)
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2021-03-12 12:14:53 +00:00
bmaidics 0dbafce8dd
MINOR: Add missing unit tests for Mirror Connect (#10192)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Viktor Somogyi-Vass <viktorsomogyi@gmail.com>
2021-03-12 11:32:17 +00:00
Dongjoon Hyun 1f9c9f8bb5
KAFKA-12442: Upgrade ZSTD JNI from 1.4.8-4 to 1.4.9-1 (#10285)
Since the new features are not used, you may right. However, I believe the benefits are three-fold.
- There exists ZSTD side bug-fixes at ZSTD 1.4.9.
- There exists ZSTD JNI side memory optimization improvements at ZSTD JNI 1.4.8-5 ~ 1.4.8-7. (This includes some incompatible changes and recovery. So, 1.4.9 is more human-readable stable version number.).
- I hope this will reduce the chance of future potential version conflict issues across Apache projects. It's important when some downstream project starts to use new feature.
   - Apache Spark 3.2.0 will use ZSTD 1.4.9. (ba7e525a11)
   - Apache Parquet 1.12.0 will use ZSTD 1.4.9 (66ac28ce23)
   - Apache Avro 1.10.3 will use ZSTD 1.4.9 (806667cb2b)

Reviewers: Lee Dongjin <dongjin@apache.org>, Ismael Juma <ismael@juma.me.uk>, Chia-Ping Tsai <chia7712@gmail.com>
2021-03-12 13:37:33 +08:00
Colin Patrick McCabe 15620ae54f
MINOR: socket setup max should be 30 seconds #10306 (#10306)
socket.connection.setup.timeout.max.ms should be 30 seconds.
The current value of 127 seconds is longer than the default API timeout
for AdminClient, and longer than the default request timeouts for the
producer and consumer.  We should bring these configs into line with
each other.

Reviewers: Jason Gustafson <jason@confluent.io>
2021-03-11 14:50:43 -08:00
dengziming 0e5591beda
KAFKA-12205; Delete snapshots less than the snapshot at the log start (#10021)
This patch adds logic to delete old snapshots. There are three cases we handle:

1. Remove old snapshots after a follower completes fetching a snapshot and truncates the log to the latest snapshot
2. Remove old snapshots after a new snapshot is created.
3. Remove old snapshots during recovery after the node is restarted.

Reviewers: Cao Manh Dat<caomanhdat317@gmail.com>, José Armando García Sancio <jsancio@users.noreply.github.com>, Jason Gustafson <jason@confluent.io>
2021-03-11 10:10:27 -08:00
Bruno Cadonna 800d9b5abc
KAFKA-10357: Add validation method for internal topics (#10266)
For KIP-698, we need a way to validate internal topics before we create them. This PR adds a validation method to the InternalTopicManager for that purpose.

Reviewers: Rohan Desai <rohan@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
2021-03-11 09:55:30 -08:00
Jorge Esteban Quilcate Otoya c534bf45ce
KAFKA-12287: Add WARN logging on consumer-groups when reset-offsets by timestamp or duration can't find an offset and defaults to latest (#10092)
Reviewers: Matthias J. Sax <matthias@confluent.io>
2021-03-10 17:25:59 -08:00
Matthias J. Sax 50518fa8e0
KAFKA-12441: remove deprecated method StreamsBuilder#addGlobalStore (#10284)
The method StreamsBuilder#addGlobalStore was simplified via KIP-233 in 1.1.0 release. This PR removes the old and deprecated overload.

Reviewers: A. Sophie Blee-Goldman <sophie@confluent.io>, Guozhang Wang <guozhang@confluent.io>
2021-03-10 16:14:44 -08:00
Rohit Deshpande 029f5a136a
KAFKA-10062: Add a methods to retrieve the current timestamps as known by the Streams app (#9744)
Implements KIP-622.

Reviewers: Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <guozhang@confluent.io>
2021-03-10 14:26:24 -08:00
dengziming 8c9bc9c640
MINOR: Add entityType for metadata record definitions (#10116)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2021-03-10 13:27:06 +08:00
Lee Dongjin e6f8f5d0ae
MINOR: Remove unused variables, methods, parameters, unthrown exceptions, and fix typos (#9457)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com
2021-03-10 13:21:30 +08:00
Ron Dagostino b92d606379
MINOR: disable round_trip_fault_test system tests for Raft quorums (#10249)
The KIP-500 early access release will not support creating a partition with a manual
partition assignment that includes a broker that is not currently online. This patch disables
system tests for Raft-based metadata quorums where the test depends on this functionality
to pass.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2021-03-09 13:57:15 -08:00
Ron Dagostino 0fc53652e1
MINOR: fix failing system test delegation_token_test (#10237)
Reviewers: Colin P. McCabe <cmccabe@apache.org>, David Arthur <mumrah@gmail.com>
2021-03-09 13:55:29 -08:00
Luke Chen 8d6ae33ac5
KAFKA-10192: Increase max time to wait for worker to start in some integration tests (#10118)
Author: Luke Chen <showuon@gmail.com>
Reviewers: Chris Egerton <chrise@confluent.io>, Randall Hauch <rhauch@gmail.com>
2021-03-09 11:03:41 -06:00
Ismael Juma 0e8a84e5d7
MINOR: Various javadoc fixes (#10272)
- Use consistent options for `javadoc` and `aggregatedJavadoc`
- `aggregatedJavadoc` depends on `compileJava`
- `connect-api` inherits `options.links`
- `streams` and `streams-test-utils` javadoc exclusions should be more
specific to avoid unexpected behavior in `aggregatedJavadoc` when the
javadoc for multiple modules is generated together

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2021-03-09 06:00:16 -08:00
Ismael Juma 1c3a139831
MINOR: Include number of members in group coordinator messages (#10273)
Reviewers: Jason Gustafson <jason@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
2021-03-09 05:49:20 -08:00
Tom Bentley a1fb80ffe2
MINOR: Add missing log argument (#10262)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2021-03-09 12:46:43 +08:00
Jason Gustafson 6b432e48ba
MINOR: Refer users to `kafka-storage.sh` if `meta.properties` is missing (#10279)
The KIP-500 server requires users to run `kafka-storage.sh` to format log directories before the server will start. If the directory is not formatted, the error message complains about a missing `meta.properties` file. It is useful for the message to refer users to `kafka-storage.sh` directly since formatting is a new requirement.

This patch also reduces the log level of a very spammy log message in `BrokerLifecycleManager`. 

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2021-03-08 13:04:46 -08:00
Jason Gustafson 31a121c3b7
KAFKA-12403; Ensure local state deleted on `RemoveTopicRecord` received (#10252)
This patch implements additional handling logic for `RemoveTopic` records:

- Update `MetadataPartitions` to ensure addition of deleted partitions to `localRemoved` set
- Ensure topic configs are removed from `ConfigRepository`
- Propagate deleted partitions to `GroupCoordinator` so that corresponding offset commits can be removed

This patch also changes the controller topic id generation logic to use `Uuid.randomUuid` rather than `Random`.

Reviewers: Ismael Juma <ismael@juma.me.uk>, Chia-Ping Tsai <chia7712@gmail.com>
2021-03-08 11:21:42 -08:00
Chia-Ping Tsai cf329cbfb7
MINOR: add missing space to errro message when setting uint16 (#10274)
Reviewers: David Arthur <mumrah@gmail.com>
2021-03-08 14:21:56 +08:00
Boyang Chen 17851da667
KAFKA-12381: remove live broker checks for forwarding topic creation (#10240)
Removed broker number checks for invalid replication factor when doing the forwarding, in order to reduce false alarms for clients.

Reviewers: Jason Gustafson <jason@confluent.io>
2021-03-05 15:55:14 -08:00
Walker Carlson 207bb0826b
KAFKA-12347: updating TaskMetadata (#10211)
added committed offset, high watermark and idle duration to the taskMetadata.

Reviewers: Boyang Chen <boyang@confluent.io>
2021-03-05 11:27:25 -08:00
Manikumar Reddy 728be21bde
MINOR: Update log level in SaslServerAuthenticator (#10270)
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>
2021-03-05 18:00:50 +05:30
A. Sophie Blee-Goldman 5c22e03e9e
MINOR: add missing docs for record-e2e-latency metrics (#10251)
Need to add missing docs for the record-e2e-latency metrics, and new TRACE recording level

Reviewers: Walker Carlson <wcarlson@confluent.io>
2021-03-04 14:42:56 -08:00
Jason Gustafson 0685b9dcd5
MINOR: Raft max batch size needs to propagate to log config (#10256)
This patch ensures that the constant max batch size defined in `KafkaRaftClient` is propagated to the constructed log configuration in `KafkaMetadataLog`. We also ensure that the fetch max size is set consistently with appropriate testing. 

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, David Arthur <mumrah@gmail.com>
2021-03-04 14:40:31 -08:00
Lee Dongjin 83c68399cd
Remove ReassigningPartitions metric when shutting down ReplicaManager (#10260)
One gauge is missing in ReplicaManager#removeMetrics

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Anna Sophie Blee-Goldman < ableegoldman@apache.org>
2021-03-04 14:25:51 -08:00
Jason Gustafson 60a097ae40
HOTFIX: Controller topic deletion should be atomic (#10264)
Topic deletions should be atomic. This fixes a build error caused by merging of both https://github.com/apache/kafka/pull/10253 and https://github.com/apache/kafka/pull/10184 at about the same time. 

Reviewers: David Arthur <mumrah@gmail.com>
2021-03-04 12:19:34 -08:00
Colin Patrick McCabe eebc6f279e
MINOR: Enable topic deletion in the KIP-500 controller (#10184)
This patch enables delete topic support for the new KIP-500 controller. Also fixes the following:
- Fix a bug where feature level records were not correctly replayed.
- Fix a bug in TimelineHashMap#remove where the wrong type was being returned.

Reviewers: Jason Gustafson <jason@confluent.io>, Justine Olshan <jolshan@confluent.io>, Ron Dagostino <rdagostino@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>, Jun Rao <junrao@gmail.com>

Co-authored-by: Jason Gustafson <jason@confluent.io>
2021-03-04 11:28:20 -08:00
Ismael Juma 7a3ebbebbc
KAFKA-12415 Prepare for Gradle 7.0 and restrict transitive scope for non api dependencies (#10203)
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>
2021-03-04 11:22:22 -08:00
José Armando García Sancio 96a2b7aac4
KAFKA-12376: Apply atomic append to the log (#10253) 2021-03-04 13:55:43 -05:00
Chia-Ping Tsai be1476869f
MINOR: make sure all generated data tests cover all versions (#10078)
Reviewers: David Jacot <djacot@confluent.io>
2021-03-05 00:22:57 +08:00
Michael G. Noll 1fd75bf1ed
KAFKA-12393: Document multi-tenancy considerations (#334) (#10263)
KAFKA-12393: Document multi-tenancy considerations
Addressed review feedback by @dajac and @rajinisivaram
Ported from apache/kafka-site#334

Reviewers: Bill Bejeck <bbejeck@apache.org>
2021-03-04 10:47:48 -05:00
Lee Dongjin ea005cc700
KAFKA-12407: Document Controller Health Metrics (#10257)
Reviewers: Luke Chen <showuon@gmail.com>, Dong Lin <lindong28@gmail.com>
2021-03-04 22:24:50 +08:00
Chia-Ping Tsai 8205051e90
MINOR: remove FetchResponse.AbortedTransaction and redundant construc… (#9758)
1. rename INVALID_HIGHWATERMARK to INVALID_HIGH_WATERMARK
2. replace FetchResponse.AbortedTransaction by FetchResponseData.AbortedTransaction
3. remove redundant constructors from FetchResponse.PartitionData
4. rename recordSet to records
5. add helpers "recordsOrFail" and "recordsSize" to FetchResponse to process record casting

Reviewers: Ismael Juma <ismael@juma.me.uk>
2021-03-04 18:06:50 +08:00
David Jacot 3ef39e1365
MINOR; Clean up LeaderAndIsrResponse construction in `ReplicaManager#becomeLeaderOrFollower` (#10234)
This patch refactors the code, which constructs the `LeaderAndIsrResponse` in `ReplicaManager#becomeLeaderOrFollower`, to improve the readability and to remove unnecessary operations.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2021-03-04 10:31:35 +01:00
Sven Erik Knop f06a47a7bb
KAFKA-12170: Fix for Connect Cast SMT to correctly transform a Byte array into a string (#9950)
Cast SMT transformation for bytes -> string.
Without this fix, the conversion becomes ByteBuffer.toString(), which always gives this useless result:
    "java.nio.HeapByteBuffer[pos=0 lim=4 cap=4]"

With this change, the byte array is converted into a base64 string of the byte buffer content.

Reviewers: Mickael Maison <mickael.maison@gmail.com>, Randall Hauch <rhauch@gmail.com>, Konstantine Karantasis <k.karantasis@gmail.com>
2021-03-03 22:17:49 +00:00
David Arthur f40a82e054
KAFKA-10759 Add ARM build stage (#9992)
Only validation and unit test stages are enabled

Co-authored-by: Peng.Lei <73098678+xiao-penglei@users.noreply.github.com>
2021-03-03 15:57:27 -05:00
Luke Chen f6929637b9
KAFKA-12284: Wait for mirrored topics to be created (#10185)
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2021-03-03 19:20:08 +00:00
Satish Duggana 0d9a95a7d0
KAFKA-9548 Added SPIs and public classes/interfaces introduced in KIP-405 for tiered storage feature in Kafka. (#10173)
KIP-405 introduces tiered storage feature in Kafka. With this feature, Kafka cluster is configured with two tiers of storage - local and remote. The local tier is the same as the current Kafka that uses the local disks on the Kafka brokers to store the log segments. The new remote tier uses systems, such as HDFS or S3 or other cloud storages to store the completed log segments. Consumers fetch the records stored in remote storage through the brokers with the existing protocol.

We introduced a few SPIs for plugging in log/index store and remote log metadata store.

This involves two parts
1. Storing the actual data in remote storage like HDFS, S3, or other cloud storages.
2. Storing the metadata about where the remote segments are stored. The default implementation uses an internal Kafka topic.

You can read KIP for more details at https://cwiki.apache.org/confluence/display/KAFKA/KIP-405%3A+Kafka+Tiered+Storage

Reviewers: Jun Rao <junrao@gmail.com>
2021-03-03 08:55:13 -08:00
Lee Dongjin b77deece1d KAFKA-12400: Upgrade jetty to fix CVE-2020-27223
Here is the fix. The reason of [CVE-2020-27223](https://nvd.nist.gov/vuln/detail/CVE-2020-27223) was DOS vulnerability for Quoted Quality CSV headers and [patched in 9.4.37.v20210219](https://github.com/eclipse/jetty.project/security/advisories/GHSA-m394-8rww-3jr7).

This PR updates Jetty dependency into the following version, 9.4.38.v20210224.

Author: Lee Dongjin <dongjin@apache.org>

Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>

Closes #10245 from dongjinleekr/feature/KAFKA-12400
2021-03-03 10:13:40 +05:30
dengziming cfb60064ec
MINOR: Fix null exception in coordinator log (#10250)
Reviewers: A. Sophie Blee-Goldman <sophie@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
2021-03-03 12:38:06 +08:00
Lee Dongjin 4b3e3a9e86 y
This security vulnerability was found in netty-codec-http, but [caused by netty itself](c735357bf2) and [fixed in 4.1.59.Final](https://github.com/netty/netty/security/advisories/GHSA-5mcr-gq6c-3hq2). So, upgrade the netty version from 4.1.51.Final to 4.1.59.Final.

Author: Lee Dongjin <dongjin@apache.org>

Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>

Closes #10235 from dongjinleekr/feature/KAFKA-12389
2021-03-03 09:45:24 +05:30
A. Sophie Blee-Goldman 23b61ba383
KAFKA-12375: don't reuse thread.id until a thread has fully shut down (#10215)
Always grab a new thread.id and verify that a thread has fully shut down to DEAD before removing from the `threads` list and making that id available again

Reviewers: Walker Carlson <wcarlson@confluent.io>, Bruno Cadonna <cadonna@confluent.io>
2021-03-02 16:28:15 -08:00
Lucas Bradstreet 36d61650f4
KAFKA-12177: apply log start offset retention before time and size based retention (#10216)
Log start offset retention is the cheapest retention to evaluate and does not require access to maxTimestamp fields for segments, nor segment sizes. In addition, it may unblock other types of retention such as time based retention. Without this change retention is not idempotent. It's possible for one deleteOldSegments call to delete segments due to log start offset retention, and a follow up call to delete due to time based retention, even if the time has not changed.

Reviewers: Jun Rao <junrao@gmail.com>
2021-03-02 13:11:16 -08:00
Jason Gustafson 3708a7c6c1
KAFKA-12369; Implement `ListTransactions` API (#10206)
This patch implements the `ListTransactions` API as documented in KIP-664: https://cwiki.apache.org/confluence/display/KAFKA/KIP-664%3A+Provide+tooling+to+detect+and+abort+hanging+transactions.

Reviewers: Tom Bentley <tbentley@redhat.com>, Chia-Ping Tsai <chia7712@gmail.com>
2021-03-02 13:01:35 -08:00
Bruno Cadonna a848e0c420
KAFKA-10357: Extract setup of changelog from Streams partition assignor (#10163)
To implement the explicit user initialization of Kafka Streams as
described in KIP-698, we first need to extract the code for the
setup of the changelog topics from the Streams partition assignor
so that it can also be called outside of a rebalance.

Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>, Guozhang Wang <guozhang@confluent.io>
2021-03-02 12:00:00 -08:00
Luke Chen 420162d190
KAFKA-10251: increase timeout for consuming records (#10228)
Bump the `pollUntilAtLeastNumRecords` timeout from 15s to 30s

Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>
2021-03-02 11:46:32 -08:00
Jason Gustafson a4ba73270c
KAFKA-12394; Return `TOPIC_AUTHORIZATION_FAILED` in delete topic response if no describe permission (#10223)
We now accept topicIds in the `DeleteTopic` request. If the client principal does not have `Describe` permission, then we return `TOPIC_AUTHORIZATION_FAILED`. This is justified because the topicId is not considered sensitive. However, in this case, we should not return the name of the topic in the response since we do consider it sensitive.

Reviewers: David Jacot <djacot@confluent.io>, dengziming <dengziming1993@gmail.com>, Justine Olshan <jolshan@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
2021-03-02 10:20:07 -08:00
Ron Dagostino 29b4a3d1fe
MINOR: Disable transactional/idempotent system tests for Raft quorums (#10224) 2021-03-02 12:57:12 -05:00