Commit Graph

11672 Commits

Author SHA1 Message Date
David Arthur 1de84590c4 KAFKA-15605: Fix topic deletion handling during ZK migration (#14545)
This patch adds reconciliation logic to migrating ZK brokers to deal with pending topic deletions as well as missed StopReplicas.

During the hybrid mode of the ZK migration, the KRaft controller is asynchronously sending UMR and LISR to the ZK brokers to propagate metadata. Since this process is essentially "best effort" it is possible for a broker to miss a StopReplicas. The new logic lets the ZK broker examine its local logs compared with the full set of replicas in a "Full" LISR. Any local logs which are not present in the set of replicas in the request are removed from ReplicaManager and marked as "stray".

To avoid inadvertent data loss with this new behavior, the brokers do not delete the "stray" partitions. They will rename the directories and log warning messages during log recovery. It will be up to the operator to manually delete the stray partitions. We can possibly enhance this in the future to clean up old stray logs.

This patch makes use of the previously unused Type field on LeaderAndIsrRequest. This was added as part of KIP-516 but never implemented. Since its introduction, an implicit 0 was sent in all LISR. The KRaft controller will now send a value of 2 to indicate a full LISR (as specified by the KIP). The presence of this value acts as a trigger for the ZK broker to perform the log reconciliation.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
Conflicts:
- ReplicaManagerTest.scala: fix imports

- ZkMigrationIntegrationTest.scala: handle absence of KIP-919 changes that added a different way to
  fetch the quorum voters config.

- KRaftMigrationDriverTest.java: handle absence of KIP-919 changes that added
  setupDeltaForMigration.
2023-11-13 15:16:45 -08:00
Justine Olshan a1d1834942
KAFKA-15780: Wait for consistent KRaft metadata when creating or deleting topics (#14695) (#14713)
TestUtils.createTopicWithAdmin calls waitForAllPartitionsMetadata which waits for partition(s) to be present in each brokers' metadata cache. This is a sufficient check in ZK mode because the controller sends an LISR request before sending an UpdateMetadataRequest which means that the partition in the ReplicaManager will be updated before the metadata cache.

In KRaft mode, the metadata cache is updated first, so the check may return before partitions and other metadata listeners are fully initialized.

Testing:
Insert a Thread.sleep(100) in BrokerMetadataPublisher.onMetadataUpdate after

      // Publish the new metadata image to the metadata cache.
      metadataCache.setImage(newImage)
and run EdgeCaseRequestTest.testProduceRequestWithNullClientId and the test will fail locally nearly deterministically. After the change(s), the test no longer fails.

Conflicts:
	core/src/test/scala/unit/kafka/server/GroupCoordinatorBaseRequestTest.scala
	core/src/test/scala/integration/kafka/admin/TopicCommandIntegrationTest.scala

Reviewers: Justine Olshan <jolshan@confluent.io>, Divij Vaidya <diviv@amazon.com>, David Mao <dmao@confluent.io>
2023-11-13 09:26:23 -08:00
Greg Harris e4b817fe6e KAFKA-15800: Prevent DataExceptions from corrupting KafkaOffsetBackingStore (#14718)
Signed-off-by: Greg Harris <greg.harris@aiven.io>

Reviewers: Yash Mayya <yash.mayya@gmail.com>
2023-11-10 08:41:59 -08:00
Justine Olshan 814de813ea
cherrypick KAFKA-15653: Pass requestLocal as argument to callback so we use the correct one for the thread (#14712)
With the new callback mechanism we were accidentally passing context with the wrong request local. Now include a RequestLocal as an explicit argument to the callback.

Also make the arguments passed through the callback clearer by separating the method out.

Added a test to ensure we use the request handler's request local and not the one passed in when the callback is executed via the request handler.

Reviewers: Ismael Juma ismael@juma.me.uk, Divij Vaidya diviv@amazon.com, David Jacot djacot@confluent.io, Jason Gustafson jason@confluent.io, Artem Livshits alivshits@confluent.io, Jun Rao junrao@gmail.com,

Conflicts:
core/src/main/scala/kafka/server/KafkaRequestHandler.scala
core/src/main/scala/kafka/server/ReplicaManager.scala
core/src/test/scala/kafka/server/KafkaRequestHandlerTest.scala

Conflicts around verification guard, running the callback on the same thread, and checking the coordinator node before AddPartitionsToTxnManager. Remove test that is not applicable since we don't have 08aa33127a
2023-11-09 21:14:25 -08:00
Chris Egerton 6b4ba0eb62
KAFKA-15693: Immediately reassign lost connectors and tasks when scheduled rebalance delay is disabled (#14647)
Reviewers: Sagar Rao <sagarmeansocean@gmail.com>, Yash Mayya <yash.mayya@gmail.com>
2023-11-09 10:48:59 -05:00
Luke Chen 1072525edc Merge branch '3.6' of https://github.com/apache/kafka into 3.6 2023-11-09 18:02:24 +08:00
David Arthur 2214e8bb69 KAFKA-15552 Fix Producer ID ZK migration (#14506)
This patch fixes a problem where we migrate the current producer ID batch to KRaft instead of the next producer ID batch. Since KRaft stores the next batch in the log, we end up serving up a duplicate batch to the first caller of AllocateProducerIds once the KRaft controller has taken over.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2023-11-09 17:59:31 +08:00
Qichao Chu 77e9c18797 MINOR: Fix flaky ProducerIdManagerTest.testUnrecoverableErrors (#14688)
We add a sleep until RetryBackoffMs to ensure that next call to generateProducerId() is triggered.

Reviewers: Divij Vaidya <diviv@amazon.com>
2023-11-07 08:48:10 +00:00
Xiaobing Fang 0834b2efb7 KAFKA-15771: fix concurrency bug in ProduceRequest#partitionSizes() (#14674)
A commit fixes a bug in ProduceRequest#partitionSizes() that may cause this method to incorrectly returning an empty or incomplete response for a thread when another thread is in the process of initialising it. 

Reviewers: Divij Vaidya <diviv@amazon.com>, hudeqi <1217150961@qq.com>, vamossagar12 <sagarmeansocean@gmail.com>

--------------------------------
Co-authored-by: fangxiaobing <fangxiaobing@kuaishou.com>
2023-11-07 08:47:35 +00:00
kumarpritam863 264e983f21
KAFKA-15680: Fix sink task partition-count metric when cooperative consumer protocol is used (#14630)
Reviewers: Chris Egerton <chrise@aiven.io>
2023-11-06 11:48:02 -05:00
Justine Olshan ed9c2c4e95 Add note about KAFKA-15653 (#14689)
Adding the note from the kafka-site repo to the main repo. I also included the fixed link.
apache/kafka-site@9fa596c
apache/kafka-site@4eb2409

Reviewers:  Divij Vaidya <diviv@amazon.com>, Ismael Juma <ismael@juma.me.uk>
2023-11-02 09:43:22 -07:00
Matthias J. Sax dfc16d858a HOTFIX: remove unused import to fix checkstyle error 2023-10-30 13:23:02 -07:00
Matthias J. Sax 1d6d157d80 KAFKA-15602: revert KAFKA-4852 (#14617)
This PR reverts
 - 51dbd175b0
 - 496ae054c2

Reviewers:  Philip Nee <pnee@confluent.io>, Guozhang Wang <guozhang@confluent.io>
2023-10-30 13:14:56 -07:00
Matthias J. Sax 3055cd7c18 HOTFIX: close iterator to avoid resource leak (#14624)
Reviewers: Hao Li <hli@confluent.io>, Bill Bejeck <bill@confluent.io>
2023-10-26 10:34:19 -07:00
atu-sharm df601e333f KAFKA-15644: Fix CVE-2023-4586 in netty:handler (#14584)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Luke Chen <showuon@gmail.com>
2023-10-26 18:39:34 +02:00
Lucas Brutschy a593d8ac30 MINOR: Fix misleading log-line (#14643)
After finishing restoration, we should only log the active tasks. Standby tasks are not part of restoration and it can be confusing to see them show up on this log message.

Reviewers: Matthias J. Sax <matthias@confluent.io>
2023-10-26 08:33:00 -07:00
Greg Harris af228ca84f KAFKA-14767: Fix missing commitId build error after git gc (#13315)
git gc moves commit hashes from individual .git/refs/heads/ to .git/packed-refs which is not read
by the determineCommitId function.

Replace the existing lookup within the .git directory with a GrGit lookup that handles packed and
unpacked refs transparently.

Reviewers: Ismael Juma <ismael@juma.me.uk>
2023-10-23 09:43:50 -07:00
Mickael Maison e569b5d2c8 KAFKA-15093: Add 3.5 Streams upgrade system tests (#14602)
Reviewers: Matthias J. Sax <mjsax@apache.org>
2023-10-23 13:27:22 +02:00
Mickael Maison 526d0f63b5 KAFKA-15664: Add 3.4 Streams upgrade system tests (#14601)
Reviewers: Luke Chen <showuon@gmail.com>,  Matthias J. Sax <mjsax@apache.org>
2023-10-23 11:28:15 +02:00
Matthias J. Sax 0b83eecddb HOTFIX: revert fix to broken system
This reverted fix applied only to `trunk` (3.7.0-SNAPSHOT)
2023-10-20 16:35:02 -07:00
Matthias J. Sax 680deffe7a KAFKA-15378: fix streams upgrade system test (#14539)
Fixing bad test setup. We tried to fix an upgrade bug for FK-joins in 3.1 release, but it later turned out that the PR was not sufficient to fix it. We finally fixed in 3.4 release.

This PR updates the system test matrix to only test working versions with FK-joins, limited to available test versions.

Reviewers: Guozhang Wang <wangguoz@gmail.com>, Hao Li <hli@confluent.io>, Mickael Maison <mickael.maison@gmail.com>
2023-10-20 16:34:02 -07:00
hudeqi 9fca00871a
KAFKA-15607: Fix NPE in MirrorCheckpointTask::syncGroupOffset (#14587)
Reviewers: Chris Egerton <chrise@aiven.io>
2023-10-20 12:18:47 -04:00
Luke Chen 33b8ee3aae remove unused import 2023-10-19 11:37:59 +08:00
Kamal Chandraprakash b83a71e053 KAFKA-15479: Remote log segments should be considered once for retention breach (#14407)
When a remote log segment contains multiple epoch, then it gets considered for multiple times during breach by retention size/time/start-offset. This will affect the deletion by remote log retention size as it deletes the number of segments less than expected. This is a follow-up of KAFKA-15352

Reviewers: Divij Vaidya <diviv@amazon.com>, Christo Lolov <lolovc@amazon.com>, Satish Duggana <satishd@apache.org>
2023-10-19 11:26:14 +08:00
Nick Telford 4f1fdfe986 KAFKA-13973: Fix inflated block cache metrics (#14317)
All block cache metrics are being multiplied by the total number of
column families. In a `RocksDBTimestampedStore`, we have 2 column
families (the default, and the timestamped values), which causes all
block cache metrics in these stores to become doubled.

The cause is that our metrics recorder uses `getAggregatedLongProperty`
to fetch block cache metrics. `getAggregatedLongProperty` queries the
property on each column family in the database, and sums the results.

Since we always configure all column families to share the same block
cache, that causes the same block cache to be queried multiple times for
its metrics, with the results added togehter, effectively multiplying
the real value by the total number of column families.

To fix this, we should simply use `getLongProperty`, which queries a
single column family (the default one). Since all column families share
the same block cache, querying just one of them will give us the correct
metrics for that shared block cache.

Note: the same block cache is shared among all column families of a store
irrespective of whether the user has configured a shared block cache
across multiple stores.

Reviewers: Matthias J. Sax <matthias@confluent.io>, Bruno Cadonna <cadonna@apache.org>
2023-10-18 18:05:19 -07:00
Matthias J. Sax 6b72b68894 MINOR: add KIP-941 to Kafka Streams upgrade docs (#14577)
Reviewers: Hao Li <hli@confluent.io>, Walker Carlson <wcarlson@confluent.io>, Bill Bejeck <bill@confluent.io>
2023-10-18 17:21:20 -07:00
Federico Valeri f51a1a8a7c MINOR: Add upgrade documentation for 3.6.0 (#14534)
This change adds the upgrade documentation for 3.6.0 and fixes the position of the notable changes in 3.5.0.
In previous releases, notable changes always come after the upgrade instructions.

Reviewers:  Luke Chen <showuon@gmail.com>, Satish Duggana <satishd@apache.org>
2023-10-13 17:29:18 +05:30
Mickael Maison 7a92d487a2 KAFKA-15596: Upgrade ZooKeeper to 3.8.3 (#14535)
Reviewers: Luke Chen <showuon@gmail.com>, Divij Vaidya <diviv@amazon.com>
2023-10-12 17:32:34 +02:00
Levani Kokhreidze a55f1624e4 KAFKA-15571: `StateRestoreListener#onRestoreSuspended` is never called because `DelegatingStateRestoreListener` doesn't implement `onRestoreSuspended` (#14519)
With https://issues.apache.org/jira/browse/KAFKA-10575 StateRestoreListener#onRestoreSuspended was added. But local tests show that it is never called because DelegatingStateRestoreListener was not updated to call a new method

Reviewers: Anna Sophie Blee-Goldman <sophie@responsive.dev>, Bruno Cadonna <cadonna@confluent.io>
2023-10-11 16:05:06 -07:00
Luke Chen eade5b8777 KAFKA-15498: upgrade to snappy 1.1.10.5 (#14458)
Release notes - https://github.com/xerial/snappy-java/releases/tag/v1.1.10.5

This release contains adds support for Windows ARM and fixes some dependencies associated with Linux ppc64. 

Reviewers: Josep Prat <josep.prat@aiven.io>
2023-10-11 18:05:26 +02:00
Mayank Shekhar Narula 69a5f14b52
KAFKA-15415: On producer-batch retry, skip-backoff on a new leader (#14384) (#14522)
This PR backports https://github.com/apache/kafka/pull/14384

Reviewers: Walker Carlson <wcarlson@apache.org>
2023-10-11 10:18:12 -05:00
Manikumar Reddy 728666f3ad KAFKA-15502: Update SslEngineValidator to handle large stores (#14445)
We have observed an issue where inter broker SSL listener is not coming up when running with TLSv3/JDK 17 .
SSL debug logs shows that TLSv3 post handshake messages >16K are not getting read and causing SslEngineValidator process to stuck while validating the provided trust/key store.

- Right now, WRAP returns if there is already data in the buffer. But if we need more data to be wrapped for UNWRAP to succeed, we end up looping forever. To fix this, now we always attempt WRAP and only return early on BUFFER_OVERFLOW.
- Update SslEngineValidator to unwrap post-handshake messages from peer when local handshake status is FINISHED.

Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>
2023-10-08 12:28:40 +05:30
Matthias J. Sax c9ae44e811
MINOR: update Kafka versions for system tests (#14501)
Reviewers: Bill Bejeck <bill@confluent.io>
2023-10-05 11:00:44 -07:00
Justine Olshan 9d7a821273 KAFKA-15330: Add missing documentation of metrics introduced as part of KAFKA-15028 (#14480)
I've added details for VerificationFailureRate and VerificationTimeMs.

I considered adding the documentation for the AddPartitionsToTxnVerification metrics, but I noticed that all the request metrics simply listed Produce|FetchConsumer|FetchFollower. If we don't already report the AddPartitionsToTxn request metrics in this file, it doesn't make sense to add the verification variant. (As well as all the other APIs we report)

Filed a followup jira if we want to redo that whole section.

Reviewers: Reviewers: Divij Vaidya <diviv@amazon.com>
2023-10-04 13:30:50 -07:00
Satish Duggana 2edd22bcab MINOR Update 3.6 branch version to 3.6.1-SNAPSHOT 2023-10-03 14:04:42 -07:00
Satish Duggana 2097c8fa4c Merge tag '3.6.0-rc2' into 3.6
3.6.0-rc2
2023-10-03 13:41:20 -07:00
David Arthur 0022949281
KAFKA-15483: Add KIP-938 and KIP-866 metrics to bundled docs (#14421)
Reviewers: Divij Vaidya <diviv@amazon.com>, Ron Dagostino <rdagostino@confluent.io>
2023-10-03 13:41:41 +02:00
Lucas Brutschy 72e275f6ea MINOR: Logging fix in StreamsPartitionAssignor (#14435)
Fix broken log message

Reviewer: A. Sophie Blee-Goldman <ableegoldman@apache.org>
2023-10-02 12:33:09 +02:00
Hao Li 3a793b094c MINOR: only log error when rack aware assignment is enabled (#14415)
Reviewers:  Lucas Brutschy <lbrutschy@confluent.io>, Matthias J. Sax <matthias@confluent.io>
2023-09-29 10:17:37 -07:00
iit2009060 1897af3ef9 KAFKA-15511: Handle CorruptIndexException in RemoteIndexCache (#14459)
A bug in the RemoteIndexCache leads to a situation where the cache does not replace the corrupted index with a new index instance fetched from remote storage. This commit fixes the bug by adding correct handling for `CorruptIndexException`.

Reviewers: Divij Vaidya <diviv@amazon.com>, Satish Duggana <satishd@apache.org>, Kamal Chandraprakash <kamal.chandraprakash@gmail.com>, Alexandre Dupriez <duprie@amazon.com>
2023-09-29 10:28:37 +00:00
Satish Duggana 60e845626d Bump version to 3.6.0 2023-09-28 21:56:28 -07:00
Kamal Chandraprakash 0d553cc9c6 KAFKA-15499: Fix the flaky DeleteSegmentsDueToLogStartOffsetBreach test (#14439)
DeleteSegmentsDueToLogStartOffsetBreach configures the segment such that it can hold at-most 2 record-batches. And, it asserts that the local-log-start-offset based on the assumption that each segment will contain exactly two messages.

During leader switch, the segment can get rotated and may not always contain two records. Previously, we were checking whether the expected local-log-start-offset is equal to the base-offset-of-the-first-local-log-segment. With this patch, we will scan the first local-log-segment for the expected offset.

Reviewers: Divij Vaidya <diviv@amazon.com>
2023-09-28 13:06:40 +00:00
Luke Chen 4fdac6136b KAFKA-15498: bump snappy-java version to 1.1.10.4 (#14434)
bump snappy-java version to 1.1.10.4, and add more tests to verify the compressed data can be correctly decompressed and read.

For LogCleanerParameterizedIntegrationTest, we increased the message size for snappy decompression since in the new version of snappy, the decompressed size is increasing compared with the previous version. But since the compression algorithm is not kafka's scope, all we need to do is to make sure the compressed data can be successfully decompressed and parsed/read.

Reviewers: Divij Vaidya <diviv@amazon.com>, Ismael Juma <ismael@juma.me.uk>, Josep Prat <josep.prat@aiven.io>, Kamal Chandraprakash <kamal.chandraprakash@gmail.com>
2023-09-27 19:02:04 +08:00
Divij Vaidya a6dd6c58e2 Upgrade Jetty to 9.4.52.v20230823 (#14438)
Reviewers: Viktor Somogyi-Vass <viktorsomogyi@gmail.com>, Luke Chen <showuon@gmail.com>, Satish Duggana <satishd@apache.org>
2023-09-25 10:26:08 -07:00
Luke Chen be527ea36c MINOR: fix kraft upgrade system test (#14424)
We should use DEV_BRANCH instead of DEV_VERSION in this case, otherwise, error will be thrown:

RunnerClient: kafkatest.tests.core.kraft_upgrade_test.TestKRaftUpgrade.test_isolated_mode_upgrade.from_kafka_version=3.6.0-SNAPSHOT.metadata_quorum=ISOLATED_KRAFT: FAIL: RemoteCommandError({'ssh_config': {'host': 'ducker10', 'hostname': 'ducker10', 'user': 'ducker', 'port': 22, 'password': '', 'identityfile': '/home/ducker/.ssh/id_rsa', 'connecttimeout': None}, 'hostname': 'ducker10', 'ssh_hostname': 'ducker10', 'user': 'ducker', 'externally_routable_ip': 'ducker10', '_logger': <Logger kafkatest.tests.core.kraft_upgrade_test.TestKRaftUpgrade.test_isolated_mode_upgrade.from_kafka_version=3.6.0-SNAPSHOT.metadata_quorum=ISOLATED_KRAFT-2 (DEBUG)>, 'os': 'linux', '_ssh_client': <paramiko.client.SSHClient object at 0xffffb35d5820>, '_sftp_client': <paramiko.sftp_client.SFTPClient object at 0xffffb35f8ca0>, '_custom_ssh_exception_checks': None}, '/opt/kafka-3.6.0-SNAPSHOT/bin/kafka-storage.sh format --ignore-formatted --config /mnt/kafka/kafka.properties --cluster-id I2eXt9rvSnyhct8BYmW6-w', 127, b'bash: line 1: /opt/kafka-3.6.0-SNAPSHOT/bin/kafka-storage.sh: No such file or directory\n')

Reviewers: Satish Duggana <satishd@apache.org>
2023-09-25 16:15:51 +08:00
Divij Vaidya e8dffea9ab MINOR: Fix kafka-site formatting (#14419)
Reviewers: Satish Duggana <satishd@apache.org>, Josep Prat <jlprat@apache.org>
2023-09-21 09:31:04 +00:00
David Arthur 01fa95c216
MINOR: Fix the ZK migration system tests (#14409)
As part of validating 3.6.0 RC0, I ran the ZK migration system tests at the RC tag. Pretty much all of them failed due to recent changes (particularly, disallowing migrations with JBOD). All of the changes here are test fixes, so not a release blocker.

================================================================================
SESSION REPORT (ALL TESTS)
ducktape version: 0.11.3
session_id:       2023-09-19--007
run time:         8 minutes 51.147 seconds
tests run:        5
passed:           5
flaky:            0
failed:           0
ignored:          0

Reviewers:  Luke Chen <showuon@gmail.com>
2023-09-20 14:36:50 +08:00
Greg Harris ae352b6397 KAFKA-15473: Hide duplicate plugins in /connector-plugins (#14398)
Reviewers: Yash Mayya <yash.mayya@gmail.com>, Sagar Rao <sagarmeansocean@gmail.com>, Hector Geraldino <hgeraldino@gmail.com>, Chris Egerton <chrise@aiven.io>
2023-09-19 22:30:18 +05:30
Satish Duggana 193d8c5be8
Added missing licenses for libraries (#14393)
Reviewers: Luke Chen <showuon@gmail.com>
2023-09-15 23:23:28 +05:30
Luke Chen 8319163062 KAFKA-15442: add a section in doc for tiered storage (#14382)
Added 6.11: Tiered Storage section and notable changes ini v3.6.0

Reviewers: Satish Duggana <satishd@apache.org>, Gantigmaa Selenge <gselenge@redhat.com>
2023-09-14 21:13:26 +05:30