Commit Graph

474 Commits

Author SHA1 Message Date
PoAn Yang f80a5c9c2a
MINOR: migrate DescribeConsumerGroupTest to use ClusterTestExtensions (#15908)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-06-16 05:16:01 +08:00
dujian0068 133f2b0f31
KAFKA-16879 SystemTime should use singleton mode (#16266)
Reviewers: Greg Harris <gharris1727@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-06-14 08:49:19 +08:00
gongxuanzhang 596b945072
KAFKA-16643 Add ModifierOrder checkstyle rule (#15890)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-06-13 15:39:32 +08:00
Ivan Yurchenko dd755b7ea9
KAFKA-8206: KIP-899: Allow client to rebootstrap (#13277)
This commit implements KIP-899: Allow producer and consumer clients to rebootstrap. It introduces the new setting `metadata.recovery.strategy`, applicable to all the types of clients.

Reviewers: Greg Harris <gharris1727@gmail.com>, Rajini Sivaram <rajinisivaram@googlemail.com>
2024-06-12 20:48:32 +01:00
gongxuanzhang af86e56fcd
KAFKA-10787 Apply spotless to tools and tools-api module (#16262)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-06-11 14:48:14 +08:00
ShivsundarR 68070c94a6
KAFKA-16724: Added support for fractional throughput and monotonic payload in kafka-producer-perf-test.sh
Added support for fractional throughput and monotonic payload in kafka-producer-perf-test.sh.
https://cwiki.apache.org/confluence/display/KAFKA/KIP-932%3A+Queues+for+Kafka#KIP932:QueuesforKafka-kafka-producer-perf-test.sh

Reviewers: Andrew Schofield <aschofield@confluent.io>, Manikumar Reddy <manikumar.reddy@gmail.com>
2024-06-11 11:19:31 +05:30
PoAn Yang dfd3c8e04b
MINOR migrate ReassignPartitionsIntegrationTest to use ClusterTestExtensions (#15675)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-06-04 16:26:53 +08:00
Ken Huang 355d5da79a
MINOR: reduce the test suits of consumer group tools (#16155)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-06-02 01:20:59 +08:00
David Jacot ba61ff0cd9
KAFKA-16860; [1/2] Introduce group.version feature flag (#16120)
This patch introduces the `group.version` feature flag with one version:
1) Version 1 enables the new consumer group rebalance protocol (KIP-848).

Reviewers: Justine Olshan <jolshan@confluent.io>
2024-05-31 12:48:55 -07:00
Ken Huang 3327435c8d
KAFKA-16598 Mirgrate `ResetConsumerGroupOffsetTest` to new test infra (#15779)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-05-30 21:51:16 +08:00
PoAn Yang 3b92046c08
MINOR: migrate ListConsumerGroupTest to use ClusterTestExtensions (#15821)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-05-30 21:30:19 +08:00
PoAn Yang 4d04eb83ea
KAFKA-16796 Introduce new org.apache.kafka.tools.api.Decoder to replace kafka.serializer.Decoder (#16064)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-05-29 03:13:33 +08:00
Kuan-Po (Cooper) Tseng 4020307ac2
KAFKA-16795 Fix broken compatibility in kafka.tools.NoOpMessageFormatter, kafka.tools.DefaultMessageFormatter, and kafka.tools.LoggingMessageFormatter (#16020)
This commit allows users to apply the scala version Formatters, but users will receive the warning messages about deprecation. 

This compatibility support will be removed from 4.0.0
 
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-05-24 02:51:26 +08:00
Mickael Maison affe8da54c
KAFKA-7632: Support Compression Levels (KIP-390) (#15516)
Reviewers: Jun Rao <jun@confluent.io>,  Luke Chen <showuon@gmail.com>
Co-authored-by: Lee Dongjin <dongjin@apache.org>
2024-05-21 17:58:49 +02:00
TaiJuWu 89083520ef
KAFKA-16654 Refactor kafka.test.annotation.Type and ClusterTestExtensions (#15916)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-05-21 22:29:06 +08:00
Johnny Hsu dac569b967
KAFKA-16668 Add tags support in ClusterTestExtension (#15861)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-05-16 18:02:13 +08:00
PoAn Yang 334d5d58bb
KAFKA-16677 Replace ClusterType#ALL and ClusterType#DEFAULT by Array (#15897)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-05-13 14:24:59 +08:00
Cheng-Kai, Zhang 643db430a7
KAFKA-16679 merge unit test down to the class of integration test (#15884)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-05-11 23:38:23 +08:00
PoAn Yang 397d58b14c
MINOR: use classic consumer with ZK mode for DeleteOffsetsConsumerGroupCommandIntegrationTest (#15872)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-05-09 20:12:22 +08:00
TingIāu "Ting" Kì 0de3b7c40b
KAFKA-16593 Rewrite DeleteConsumerGroupsTest by ClusterTestExtensions (#15766)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-05-07 14:04:32 +08:00
Chia Chuan Yu 55a00be4e9
MINOR: Replaced Utils.join() with JDK API. (#15823)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-05-06 15:13:01 +08:00
Ken Huang da5f4424dc
MINOR: Clean up TestUtils.scala (#15808)
This PR do the following cleanup for TestUtils.scala

1) remove unused methods
2) move methods used by single test class out of

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-05-01 04:13:29 +08:00
PoAn Yang c287ad5dbe
MINOR: refine DeleteOffsetsConsumerGroupCommandIntegrationTest#produceRecord (#15802)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-27 07:11:10 +08:00
TaiJuWu ce9026f597
MINOR: Modified System.getProperty("line.separator") to java.lang.System.lineSeparator() (#15782)
Reviewers: Igor Soarez  <soarez@apple.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-04-26 02:32:11 +08:00
Chia-Ping Tsai 4e23378aa8
HOTFIX: make sure all ConsumerGroupServices get closed (#15801)
Some services are not closed, so our CI print following error.

org.opentest4j.AssertionFailedError: Found 16 unexpected threads during @BeforeAll: `kafka-admin-client-thread | adminclient-287,kafka-admin-client-thread | adminclient-276,kafka-admin-client-thread | adminclient-271,kafka-admin-client-thread | adminclient-293,kafka-admin-client-thread | adminclient-281,kafka-admin-client-thread | adminclient-302,kafka-admin-client-thread | adminclient-334,kafka-admin-client-thread | adminclient-323,kafka-admin-client-thread | adminclient-257,kafka-admin-client-thread | adminclient-336,kafka-admin-client-thread | adminclient-308,kafka-admin-client-thread | adminclient-263,kafka-admin-client-thread | adminclient-273,kafka-admin-client-thread | adminclient-278,kafka-admin-client-thread | adminclient-283,kafka-admin-client-thread | adminclient-317` ==> expected: <true> but was: <false>

#15679 use AfterEach to release service. However, the test cases having multi consumerConfigs will create a lot of services in testing. Hence, the intermediate servers are not closed.

Reviewers: Luke Chen <showuon@gmail.com>
2024-04-25 14:23:34 +08:00
TingIāu "Ting" Kì 864744ffd4
KAFKA-16610 Replace "Map#entrySet#forEach" by "Map#forEach" (#15795)
Reviewers: Apoorv Mittal <amittal@confluent.io>, Igor Soarez <soarez@apple.com>
2024-04-25 01:52:24 +01:00
PoAn Yang 960794a677
KAFKA-16483 migrate DeleteOffsetsConsumerGroupCommandIntegrationTest to use ClusterTestExtensions (#15679)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-24 19:07:58 +08:00
Omnia Ibrahim cfe5ab5cf2
KAFKA-15853 Move quota configs into server-common package (#15774)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-04-24 13:05:18 +08:00
Kirk True 53f9675b33
KAFKA-16461: New consumer fails to consume records in security_test.py system test (#15746)
The system test was failing because the VerifiableConsumer failed with a NullPointerException during startup. The reason for the NPE was an attempt to put a null as the value of --group-remote-assignor in the Consumer's configuration.

Reviewers: Lianet Magrans <lianetmr@gmail.com>, Lucas Brutschy <lbrutschy@confluent.io>
2024-04-23 10:28:07 +02:00
Calvin Liu 53ff1a5a58
KAFKA-15585: DescribeTopicPartitions client side change. (#15470)
Add the support for DescribeTopicPartitions API to AdminClient. For this initial implementation, we are simply loading all of the results into memory on the client side. 

Reviewers: Andrew Schofield <aschofield@confluent.io>, Kirk True <ktrue@confluent.io>, David Jacot <djacot@confluent.io>, Artem Livshits <alivshits@confluent.io>, David Arthur <mumrah@gmail.com>
2024-04-18 12:09:14 -04:00
Omnia Ibrahim 363f4d2847
KAFKA-15853 Move consumer group and group coordinator configs out of core (#15684)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-17 20:41:22 +08:00
Omnia Ibrahim 8c0458861c
KAFKA-15853 Move KafkaConfig Replication properties and docs out of … (#15575)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-16 15:28:35 +08:00
Mickael Maison 395fdae5f2
MINOR: Various cleanups in tools (#15709)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-15 17:25:17 +02:00
Owen Leung 437ebb941e
KAFKA-15729 Add KRaft support in GetOffsetShellTest (#15489)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-14 21:13:49 +08:00
Alok Thatikunta c034cf2953
MINOR: Fix incorrect Java equals comparison of Uuid by reference (#15707)
Reviewers: Justine Olshan <jolshan@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
2024-04-13 20:55:48 +08:00
Cheng-Kai, Zhang 72b823e9bd
KAFKA-16482 Eliminate the IDE warnings of accepting ClusterConfig in BeforeEach (LeaderElectionCommandTest and ProducerIdsIntegrationTest) (#15676)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-11 18:42:14 +08:00
Kuan-Po (Cooper) Tseng f895ab5145
KAFKA-16455: Check partition exists before send reassignments to server in ReassignPartitionsCommand (#15659)
Currently, when executing kafka-reassign-partitions.sh with the --execute option, if a partition number specified in the JSON file does not exist, this check occurs only when submitting the reassignments to alterPartitionReassignments on the server-side.

We can perform this check in advance before submitting the reassignments to the server side.

Reviewers: Luke Chen <showuon@gmail.com>
2024-04-09 07:56:31 +08:00
Nikolay 31e8a7fb04
MINOR: Default test name added to tools (#15666)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-07 21:35:00 +08:00
PoAn Yang 5c1b819b02
KAFKA-16472 Fix integration tests in Java with parameter name (#15663)
Following test cases don't really run kraft case. The reason is that the test info doesn't contain parameter name, so it always returns false in TestInfoUtils#isKRaft.

1) TopicCommandIntegrationTest
2) DeleteConsumerGroupsTest
3) AuthorizerIntegrationTest
4) DeleteOffsetsConsumerGroupCommandIntegrationTest

We can fix it by adding options.compilerArgs << '-parameters' after

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-06 10:43:22 +08:00
Cheng-Kai, Zhang 7a10f4a17e
MINOR: enhance kafka-reassign-partitions command output (#15610)
Currently, when we using kafka-reassign-partitions to move the log directory, the output only indicates which replica's movement has successfully started.

This PR propose to show more detailed information, helping end users understand that the operation is proceeding as expected.

Reviewers: Luke Chen <showuon@gmail.com>, Andrew Schofield <aschofield@confluent.io>
2024-04-02 14:54:01 +08:00
Nikolay d8673b26bf
KAFKA-15899 [1/2] Move kafka.security package from core to server module (#15572)
1) This PR moves kafka.security classes from core to server module.
2) AclAuthorizer not moved, because it has heavy dependencies on core classes that not rewrited from scala at the moment.
3) AclAuthorizer will be deleted as part of ZK removal

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-03-30 11:54:22 +08:00
PoAn Yang 6f8d4fe26b
KAFKA-15949: Unify metadata.version format in log and error message (#15505)
There were different words for metadata.version like metadata version or metadataVersion. Unify format as metadata.version.

Reviewers: Luke Chen <showuon@gmail.com>
2024-03-26 20:09:29 +08:00
Dmitry Werner 2d4abb85bf
KAFKA-16415 Fix handling of "--version" option in ConsumerGroupCommand (#15592)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-03-26 15:44:23 +08:00
PoAn Yang fa1cf7975e
KAFKA-16409: DeleteRecordsCommand should use standard exception handling (#15586)
DeleteRecordsCommand should use standard exception handling

Reviewers: Luke Chen <showuon@gmail.com>
2024-03-26 08:44:59 +08:00
Kuan-Po (Cooper) Tseng 7b2fc469ad
KAFKA-16410 kafka-leader-election / LeaderElectionCommand doesn't set exit code on error (#15591)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-03-25 12:31:37 +08:00
Dmitry Werner 0434c29e58
KAFKA-16408 kafka-get-offsets / GetOffsetShell doesn't handle --version or --help (#15583)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-03-25 12:12:23 +08:00
Nikolay 0f216b6448
MINOR: Tuple2 replaced with Map.Entry (#15560)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-03-23 11:44:05 +08:00
Nikolay b6183a4134
KAFKA-14589 ConsumerGroupCommand rewritten in java (#14471)
This PR contains changes to rewrite ConsumerGroupCommand in java and transfer it to tools module

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-03-20 15:34:45 +08:00
Kuan-Po (Cooper) Tseng 12a1d85362
KAFKA-12187 replace assertTrue(obj instanceof X) with assertInstanceOf (#15512)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-03-20 10:36:25 +08:00
Luke Chen 834efa6606
KAFKA-16342 fix getOffsetByMaxTimestamp for compressed records (#15474)
Fix getOffsetByMaxTimestamp for compressed records.

This PR adds:

1) For inPlaceAssignment case, compute the correct offset for maxTimestamp when traversing the batch records, and set to ValidationResult in the end, instead of setting to last offset always.

2) For not inPlaceAssignment, set the offsetOfMaxTimestamp for the log create time, like non-compressed, and inPlaceAssignment cases, instead of setting to last offset always.

3) Add tests to verify the fix.

Reviewers: Jun Rao <junrao@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>
2024-03-15 06:09:45 +08:00
Nikolay 414365979e
KAFKA-14589 [4/4] Tests of ConsoleGroupCommand rewritten in java (#15465)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-03-09 03:54:39 +08:00
PoAn Yang 5dd382ccbd
MINOR: Use INFO logging for tools tests (#15487)
Reviewers: Luke Chen <showuon@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-03-08 03:02:22 +08:00
Dmitry Werner ba0db81e53
KAFKA-16246: Cleanups in ConsoleConsumer (#15457)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Omnia Ibrahim <o.g.h.ibrahim@gmail.com>
2024-03-07 09:39:16 +01:00
Nikolay 5f4806fd1c
KAFKA-14589 [2/4] Tests of ConsoleGroupCommand rewritten in java (#15363)
This PR is part of #14471
It contains some of ConsoleGroupCommand tests rewritten in java.
Intention of separate PR is to reduce changes and simplify review.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-03-07 07:44:17 +08:00
Nikolay f6198bc075
KAFKA-14589 [3/4] Tests of ConsoleGroupCommand rewritten in java (#15365)
Is contains some of ConsoleGroupCommand tests rewritten in java.
Intention of separate PR is to reduce changes and simplify review.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-03-06 17:13:39 +08:00
PoAn Yang c254b22a48
MINOR: simplify ensure topic exists condition (#15458)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-03-04 19:50:56 +08:00
Ritika Reddy 96c68096a2
KAFKA-15462: Add Group Type Filter for List Group to the Admin Client (#15150)
In KIP-848, we introduce the notion of Group Types based on the protocol type that the members in the consumer group use. As of now we support two types of groups:
* Classic : Members use the classic consumer group protocol ( existing one )
* Consumer : Members use the consumer group protocol introduced in KIP-848.
Currently List Groups allows users to list all the consumer groups available. KIP-518 introduced filtering the consumer groups by the state that they are in. We now want to allow users to filter consumer groups by type.

This patch includes the changes to the admin client and related files. It also includes changes to parameterize the tests to include permutations of the old GC and the new GC with the different protocol types.

Reviewers: David Jacot <djacot@confluent.io>
2024-02-29 00:38:42 -08:00
Yang Yu b4e96913cc
KAFKA-16265: KIP-994 (Part 1) Minor Enhancements to ListTransactionsRequest (#15384)
Introduces a new filter in ListTransactionsRequest API. This enables caller to filter on transactions that have been running for longer than a certain duration of time.

This PR includes the following changes:

bumps version for ListTransactionsRequest API to 1. Set the durationFilter to -1L when communicating with an older broker that does not support version 1.
bumps version for ListTransactionsResponse to 1 without changing the response structure.
adds durationFilter option to kafka-transactions.sh --list
Tests:

Client side test to build request with correct combination of duration filter and API version: testBuildRequestWithDurationFilter
Server side test to filter transactions based on duration: testListTransactionsFiltering
Added test case for kafka-transactions.sh change in TransactionsCommandTest

Reviewers: Justine Olshan <jolshan@confluent.io>, Raman Verma <rverma@confluent.io>
2024-02-24 06:09:23 -08:00
Owen Leung 71a4e6fc0c
KAFKA-15140: improve TopicCommandIntegrationTest to be less flaky (#14891)
This PR improves TopicCommandIntegrationTest by :
    - using TestUtils.createTopicWithAdmin
    - replacing \n with lineSeperator
    - using waitForAllReassignmentsToComplete
    - adding more log when assertion fails

Reviewers: Luke Chen <showuon@gmail.com>, Justine Olshan <jolshan@confluent.io>
2024-02-19 19:37:31 +08:00
David Jacot e247bd03af
MINOR: Improve ListConsumerGroupTest.testListGroupCommand (#15382)
While reviewing https://github.com/apache/kafka/pull/15150, I found that our tests verifying the console output are really hard to read. Here is my proposal to make it better.

Reviewers: Ritika Reddy <rreddy@confluent.io>, Justine Olshan <jolshan@confluent.io>
2024-02-17 00:07:50 -08:00
Kirk True 7a07aefd86
KAFKA-16230: Update verifiable_consumer.py to support KIP-848’s group protocol config (#15328)
The Python VerifiableConsumer now passes in the --group-protocol and --group-remote-assignor command line arguments to VerifiableConsumer if the node is running 3.7.0+ and using the new consumer group.protocol.

Reviewers: Andrew Schofield <aschofield@confluent.io>, Lucas Brutschy <lbrutschy@confluent.io>
2024-02-16 11:53:53 +01:00
Mickael Maison 0bf830fc9c
KAFKA-14576: Move ConsoleConsumer to tools (#15274)
Reviewers: Josep Prat <josep.prat@aiven.io>, Omnia Ibrahim <o.g.h.ibrahim@gmail.com>
2024-02-13 19:24:07 +01:00
Nikolay 88c5543ccf
KAFKA-14589: [1/3] Tests of ConsoleGroupCommand rewritten in java (#15256)
This PR is part of #14471
Is contains some of ConsoleGroupCommand tests rewritten in java.
Intention of separate PR is to reduce changes and simplify review.

Reviewers: Luke Chen <showuon@gmail.com>
2024-02-13 11:02:36 +08:00
Kirk True 08b68583fa
KAFKA-16037: Update VerifiableConsumer to support KIP-848’s group protocol config (#15325)
Add the optional --group-protocol command line option that can be set in the system tests

Reviewers: Andrew Schofield <aschofield@confluent.io>, Lucas Brutschy <lbrutschy@confluent.io>
2024-02-07 10:06:13 +01:00
Philip Nee f0087ac6a8
KAFKA-16115: Adding missing heartbeat metrics (#15216)
Add a HeartbeatMetrics module to measure various metrics related to the heartbeat. Here is the highlight of the changes:

HeartMetrics encapsulate a heartbeat sensor
Add metrics suffix types to the AbstractConsumerMetrics so that it can be extended to different metric groups
Non-related refactor:

Rename the metric classes to "MetricManager" the reason being "Metrics" seems to relate to the Metric class however these managers are merely containers holding sensor references for recording purposes.
Created OffsetCommitMetricsManager so that we are more consistent with the metrics management
Extended KafkaConsumerMetrics to the AbstractConsumerMetrics so that the groupName is consistently created (without random concatenation)
Some follow ups:

Refactor commit sensor by introducing a CommitMetrics module so that we can keep most of the metrics in one place and let it be a bit more consistent stylistically
Possibly refactor fetch manager metrics

Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
2024-02-02 15:06:41 +01:00
Nikolay 13c0c5ee97
KAFKA-14589 ConsumerGroupServiceTest rewritten in java (#15248)
This PR is part of #14471
Is contains single test rewritten in java.
Intention of separate PR is to reduce changes and simplify review.

Reviewers: Justine Olshan <jolshan@confluent.io>
2024-01-26 10:32:48 -08:00
Nikolay ff25c350a7
KAFKA-14589 ConsumerGroupCommand options and case classes rewritten (#14856)
This PR is part of #14471
It contains ConsumerGroupCommandOptions and case classes used by ConsumerGroupCommand rewritten in java.

The goal of PR is to reduce main PR size.

Co-authored-by: Taras Ledkov <tledkov@apache.org>

Reviewers: Mickael Maison <mickael.maison@gmail.com>, Taras Ledkov <tledkov@apache.org>, Justine Olshan <jolshan@confluent.io>
2024-01-22 16:30:34 -08:00
David Arthur 7bf7fd99a5
KAFKA-16078: Be more consistent about getting the latest MetadataVersion
This PR creates MetadataVersion.latestTesting to represent the highest metadata version (which may be unstable) and MetadataVersion.latestProduction to represent the latest version that should be used in production. It fixes a few cases where the broker was advertising that it supported the testing versions even when unstable metadata versions had not been configured.

Reviewers: Colin P. McCabe <cmccabe@apache.org>, Ismael Juma <ismael@juma.me.uk>
2024-01-17 14:59:22 -08:00
Divij Vaidya 65424ab484
MINOR: New year code cleanup - include final keyword (#15072)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Ismael Juma <ismael@juma.me.uk>, Sagar Rao <sagarmeansocean@gmail.com>
2024-01-11 17:53:35 +01:00
Sergio Troiano ae4d308f68
KAFKA-16015: Fix custom timeouts overwritten by defaults in LeaderElectionCommand (#15030)
This commit fixes a bug in LeaderElectionCommand due to which custom timeout configuration was not being respected.

Reviewers: Divij Vaidya <diviv@amazon.com>, Proven Provenzano <pprovenzano@confluent.io>
2023-12-29 10:50:26 +01:00
Proven Provenzano b0e99b5593
KAFKA-15922: Bump MetadataVersion to support JBOD with KRaft (#14984)
Moves ELR from MetadataVersion IBP_3_7_IV3 into the new IBP_3_8_IV0 because the ELR feature was not completed before 3.7 reached feature freeze.  Leaves IBP_3_7_IV3 empty -- it is a no-op and is not reused for anything.  Adds the new MetadataVersion IBP_3_7_IV4 for the FETCH request changes from KIP-951, which were mistakenly never associated with a MetadataVersion.  Updates the LATEST_PRODUCTION MetadataVersion to IBP_3_7_IV4 to declare both KRaft JBOD and the KIP-951 changes ready for production use.

Reviewers: Omnia G H Ibrahim <o.g.h.ibrahim@gmail.com>, Ron Dagostino <rdagostino@confluent.io>, Ismael Juma <ismael@juma.me.uk>, José Armando García Sancio <jsancio@apache.org>, Justine Olshan <jolshan@confluent.io>
2023-12-14 10:08:54 -05:00
Andrew Schofield 46852eea1c
KAFKA-15871: kafka-client-metrics.sh (#14926)
Initial implementation of kafka-client-metrics.sh tools for KIP-714 and KIP-1000.

Reviewers: Igor Soarez <soarez@apple.com>, Jun Rao <junrao@gmail.com>
2023-12-06 10:10:10 -08:00
Nikolay 783698c525
KAFKA-15645: Move ReplicationQuotasTestRig to tools module (#14588)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Justine Olshan <jolshan@confluent.io>, Taras Ledkov <tledkov@apache.org>
2023-12-05 10:03:33 +01:00
David Jacot 26274afd05
MINOR: Ensure that DisplayName is set in all parameterized tests (#14850)
This is a follow-up to https://github.com/apache/kafka/pull/14687 as we found out that some parameterized tests do not include the test method name in their name. For the context, the JUnit XML report does not include the name of the method by default but only rely on the display name provided.

Reviewers: David Arthur <mumrah@gmail.com>
2023-12-04 23:58:48 -08:00
Colin Patrick McCabe a94bc8d6d5
KAFKA-15922: Add a MetadataVersion for JBOD (#14860)
Assign MetadataVersion.IBP_3_7_IV2 to JBOD.

Move KIP-966 support to MetadataVersion.IBP_3_7_IV3.

Create MetadataVersion.LATEST_PRODUCTION as the latest metadata version that can be used when formatting a
new cluster, or upgrading a cluster using kafka-features.sh. This will allow us to clearly distinguish between stable
and unstable metadata versions for the first time.

Reviewers: Igor Soarez <soarez@apple.com>, Ron Dagostino <rndgstn@gmail.com>, Calvin Liu <caliu@confluent.io>, Proven Provenzano <pprovenzano@confluent.io>
2023-11-30 10:35:13 -08:00
runom 066635819a
MINOR: Improve printing topic name when created topic in TopicCommand (#14661)
The topic name was displayed as `Optional<String>` when the topic was created.
```
% bin/kafka-topics.sh --create --topic test --bootstrap-server localhost:9092
Created topic Optional[test].
```
This PR fixed to print the topic name as `String` instead of `Optional<String>`.
```
% bin/kafka-topics.sh --create --topic test --bootstrap-server localhost:9092
Created topic test.
```

Reviewers: Ismael Juma <ismael@juma.me.uk>
2023-11-19 16:03:07 -08:00
Nikolay 76b1b50b64
KAFKA-14595 Move ReassignPartitionsCommand to java (#13247)
This PR contains changes required to move PartitionReassignmentState class to java code.

Reviewers: Mickael Maison <mickael.maison@gmail.com>, Justine Olshan <jolshan@confluent.io>, Federico Valeri <fedevaleri@gmail.com>, Taras Ledkov Taras Ledkov <tledkov@apache.org>, Kamal Chandraprakash<kamal.chandraprakash@gmail.com>,
2023-10-31 17:29:05 -07:00
Calvin Liu af747fbfed
KAFKA-15581: Introduce ELR (#14312)
This patch introduces preliminary changes for Eligible Leader Replicas (KIP-966)

* New MetadataVersion 16 (3.7-IV1)
* New record versions for PartitionRecord and PartitionChangeRecord
* New tagged fields on PartitionRecord and PartitionChangeRecord
* New static config "eligible.leader.replicas.enable" to gate the whole feature

Reviewers: Artem Livshits <alivshits@confluent.io>, David Arthur <mumrah@gmail.com>, Colin P. McCabe <cmccabe@apache.org>
2023-10-19 14:05:15 -04:00
Matthias J. Sax 9b468fb278
MINOR: Do not end Javadoc comments with `**/` (#14540)
Reviewers: Bruno Cadonna <bruno@confluent.io>, Bill Bejeck <bill@confluent.io>, Hao Li <hli@confluent.io>, Josep Prat <josep.prat@aiven.io>
2023-10-17 21:11:04 -07:00
Omnia G.H Ibrahim 9af1e74b5e
KAFKA-14596: Move TopicCommand to tools (#13201)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Federico Valeri <fedevaleri@gmail.com>
2023-10-17 11:40:15 +02:00
Federico Valeri aec07f76d7
KAFKA-15537: Fix metadata downgrade documentation (#14484)
In KIP-778 we introduced the "unsafe" (lossy) downgrade in case metadata has changes in one of the versions between target and current, as defined in MetadataVersion.

The documentation says it is possible:

"Note that the cluster metadata version cannot be downgraded to a pre-production 3.0.x, 3.1.x, or 3.2.x version once it has been upgraded. However, it is possible to downgrade to production versions such as 3.3-IV0, 3.3-IV1, etc."

The command line tool shows that this doesn't work:

bin/kafka-features.sh --bootstrap-server :9092 downgrade --metadata 3.4 --unsafe
Could not downgrade metadata.version to 8. Invalid metadata.version 8. Unsafe metadata downgrade is not supported in this version.
1 out of 1 operation(s) failed.

In addition to unsafe, also safe metadata downgrades are not supported in practice. For example, when you upgrade to 3.5, you land on 3.5-IV2 as metadata version, which has metadata changes and won't let you to downgrade. This is true for every other release at the moment.

This change fixes the documentation to reflect that, and improves the error messages.

Signed-off-by: Federico Valeri <fedevaleri@gmail.com>

Reviewers: Luke Chen <showuon@gmail.com>, Jakub Scholz <github@scholzj.com>
2023-10-12 11:12:44 +08:00
Omnia G.H Ibrahim 7553d3f562
KAFKA-14593: Move LeaderElectionCommand to tools (#13204)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Federico Valeri <fedevaleri@gmail.com>
2023-10-03 11:59:56 +02:00
Nikolay 8f8dbad564
KAFKA-14595 ReassignPartitionsIntegrationTest rewritten in java (#14456)
This PR is part of #13247
It contains ReassignPartitionsIntegrationTest rewritten in java.
Goal of PR is reduce changes size in main PR.

Reviewers: Taras Ledkov  <tledkov@apache.org>, Justine Olshan <jolshan@confluent.io>
2023-10-02 13:22:17 -07:00
Colin Patrick McCabe fcac880fd5
KAFKA-15466: Add KIP-919 support for some admin APIs (#14399)
Add support for --bootstrap-controller in the following command-line tools:
    - kafka-cluster.sh
    - kafka-configs.sh
    - kafka-features.sh
    - kafka-metadata-quorum.sh

To implement this, the following AdminClient APIs now support the new bootstrap.controllers
configuration:
    - Admin.alterConfigs
    - Admin.describeCluster
    - Admin.describeConfigs
    - Admin.describeFeatures
    - Admin.describeMetadataQuorum
    - Admin.incrementalAlterConfigs
    - Admin.updateFeatures

Command-line tool changes:
    - Add CommandLineUtils.initializeBootstrapProperties to handle parsing --bootstrap-controller
      in addition to --bootstrap-server.
    - Add --bootstrap-controller to ConfigCommand.scala, ClusterTool.java, FeatureCommand.java, and
      MetadataQuorumCommand.java.

KafkaAdminClient changes:
    - Add the AdminBootstrapAddresses class to handle extracting bootstrap.servers or
      bootstrap.controllers from the config map for KafkaAdminClient.
    - In AdminMetadataManager, store the new usingBootstrapControllers boolean. Generalize
      authException to encompass the concept of fatal exceptions in general. (For example, the
      fatal exception where we talked to the wrong node type.) Treat
      MismatchedEndpointTypeException and UnsupportedEndpointTypeException as fatal exceptions.
    - Extend NodeProvider to include information about whether bootstrap.controllers is supported.
    - Modify the APIs described above to support bootstrap.controllers.

Server-side changes:
    - Support DescribeConfigsRequest on kcontrollers.
    - Add KRaftMetadataCache to the kcontroller to simplify implemeting describeConfigs (and
      probably more APIs in the future). It's mainly a wrapper around MetadataImage, so there is
      essentially no extra resource consumption.
    - Split RuntimeLoggerManager out of ConfigAdminManager to handle the incrementalAlterConfigs
      support for BROKER_LOGGER. This is now supported on kcontrollers as well as brokers.
    - Fix bug in AuthHelper.computeDescribeClusterResponse that resulted in us always sending back
      BROKER as the endpoint type, even on the kcontroller.

Miscellaneous:
    - Fix a few places in exceptions and log messages where we wrote "broker" instead of "node".
      For example, an exception in NodeApiVersions.java, and a log message in NetworkClient.java.
    - Fix the slf4j log prefix used by KafkaRequestHandler logging so that request handlers on a
      controller don't look like they're on a broker.
    - Make the FinalizedVersionRange constructor public for the sake of a junit test.
    - Add unit and integration tests for the above.

Reviewers: David Arthur <mumrah@gmail.com>, Doguscan Namal <namal.doguscan@gmail.com>
2023-09-26 14:43:42 -07:00
Nikolay daf8a0deda
KAFKA-14595 ReassignPartitionsUnitTest rewritten in java (#14355)
This PR is part of #13247
It contains changes to rewrite single test in java.
Intention is reduce changes in parent PR.

Reviewers: Luke Chen <showuon@gmail.com>, Taras Ledkov <tledkov@apache.org>
2023-09-23 09:45:14 +08:00
Ruslan Krivoshein b72d92919f
KAFKA-14581: Moving GetOffsetShell to tools (#13562)
This PR moves GetOffsetShell from core module to tools module with rewriting from Scala to Java.

Reviewers: Federico Valeri fedevaleri@gmail.com, Ziming Deng dengziming1993@gmail.com, Mickael Maison mimaison@apache.org.
2023-09-11 10:30:22 +08:00
Colin Patrick McCabe 41b695b6e3
KAFKA-15369: Implement KIP-919: Allow AC to Talk Directly with Controllers (#14306)
Implement KIP-919: Allow AdminClient to Talk Directly with the KRaft Controller Quorum and add
Controller Registration. This KIP adds a new version of DescribeClusterRequest which is supported
by KRaft controllers. It also teaches AdminClient how to use this new DESCRIBE_CLUSTER request to
talk directly with the controller quorum. This is all gated behind a new MetadataVersion,
IBP_3_7_IV0.

In order to share the DESCRIBE_CLUSTER logic between broker and controller, this PR factors it out
into AuthHelper.computeDescribeClusterResponse.

The KIP adds three new errors codes: MISMATCHED_ENDPOINT_TYPE, UNSUPPORTED_ENDPOINT_TYPE, and
UNKNOWN_CONTROLLER_ID. The endpoint type errors can be returned from DescribeClusterRequest

On the controller side, the controllers now try to register themselves with the current active
controller, by sending a CONTROLLER_REGISTRATION request. This, in turn, is converted into a
RegisterControllerRecord by the active controller. ClusterImage, ClusterDelta, and all other
associated classes have been upgraded to propagate the new metadata. In the metadata shell, the
cluster directory now contains both broker and controller subdirectories.

QuorumFeatures previously had a reference to the ApiVersions structure used by the controller's
NetworkClient. Because this PR removes that reference, QuorumFeatures now contains only immutable
data. Specifically, it contains the current node ID, the locally supported features, and the list
of quorum node IDs in the cluster.

Reviewers: David Arthur <mumrah@gmail.com>, Ziming Deng <dengziming1993@gmail.com>, Luke Chen <showuon@gmail.com>
2023-09-07 15:21:52 -07:00
Nikolay 0029bc4897
KAFKA-14595: ReassignPartitionsCommandArgsTest rewritten in java (#14217)
Reviewers: Taras Ledkov <tledkov@apache.org>, Greg Harris <greg.harris@aiven.io>
2023-09-07 10:12:07 -07:00
Ron Dagostino 8394ddc0d2
MINOR: Move delegation token support to Metadata Version 3.6-IV2 (#14270)
#14083 added support for delegation tokens in KRaft and attached that support to the existing
MetadataVersion 3.6-IV1. This patch moves that support into a separate MetadataVersion 3.6-IV2.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2023-08-22 16:04:53 -07:00
Greg Harris 6bd17419b7
KAFKA-15228: Add sync-manifests command to connect-plugin-path (KIP-898) (#14195)
Reviewers: Chris Egerton <chrise@aiven.io>
2023-08-16 11:37:33 -07:00
Colin Patrick McCabe adc16d0f31
KAFKA-14538: Implement KRaft metadata transactions in QuorumController
Implement the QuorumController side of KRaft metadata transactions.

As specified in KIP-868, this PR creates a new metadata version, IBP_3_6_IV1, which contains the
three new records: AbortTransactionRecord, BeginTransactionRecord, EndTransactionRecord.

In order to make offset management unit-testable, this PR moves it out of QuorumController.java and
into OffsetControlManager.java. The general approach here is to track the "last stable offset," which is
calculated by looking at the latest committed offset and the in-progress transaction (if any). When
a transaction is aborted, we revert back to this last stable offset. We also revert back to it when
the controller is transitioning from active to inactive.

In a follow-up PR, we will add support for the transaction records in MetadataLoader. We will also
add support for automatically aborting pending transactions after a controller failover.

Reviewers: David Arthur <mumrah@gmail.com>
2023-08-14 16:58:56 -07:00
Greg Harris f5655d31d3
KAFKA-15030: Add connect-plugin-path command-line tool (#14064)
Reviewers: Chris Egerton <chrise@aiven.io>
2023-08-11 12:05:51 -07:00
Nikolay 1fd58e30cf
KAFKA-14595: Move classes from ReassignPartitionsCommand to tools (#14172)
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2023-08-11 14:52:14 +02:00
Federico Valeri 8de3e0436a
KAFKA-15239: Fix system tests using producer performance service (#14092)
Reviewers: Greg Harris <greg.harris@aiven.io>
2023-08-10 14:23:43 -07:00
Federico Valeri bb677c4959
KAFKA-14583: Move ReplicaVerificationTool to tools (#14059)
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2023-07-26 12:04:34 +02:00
Federico Valeri 1bf73d89d0
KAFKA-15232: Move ToolsUtils to tools (#14066)
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2023-07-21 20:27:44 +02:00
Nikolay 4bba2c8a32
KAFKA-14591: Move DeleteRecordsCommand to tools (#13278)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Federico Valeri <fedevaleri@gmail.com>
2023-07-21 17:30:28 +02:00
Greg Harris 125dbb9286
KAFKA-14760: Move ThroughputThrottler from tools to clients, remove tools dependency from connect-runtime (#13313)
Reviewers: Ismael Juma <ismael@juma.me.uk>
2023-07-20 12:58:48 -07:00
Federico Valeri 334c41d604
KAFKA-14734: Use CommandDefaultOptions in StreamsResetter (#13983)
This PR adds CommandDefaultOptions usage like in the other joptsimple based tools. It also moves the associated unit test class from streams to tools module as discussed in #13127 (comment)

Reviewers:  Luke Chen <showuon@gmail.com>, Bruno Cadonna <cadonna@apache.org>, Sagar Rao <sagarmeansocean@gmail.com>
2023-07-20 18:45:05 +08:00
Manikumar Reddy 4e85bc9f80
MINOR: Fix Jmxtool to honour wait option when MBean is not yet avaibale in MBean server (#13995)
In JmxTool.scala, we will wait till all the object names are available from MBean server. But in the newer version, we only wait for subset of object names. Due to this, we may not enforce wait option and prematurely return the result if the objects are not yet registered in MBean sever.

Reviewers: Luke Chen <showuon@gmail.com>, Federico Valeri <fvaleri@redhat.com>
2023-07-12 17:01:10 +05:30
prasanthV 58fc264410
MINOR: Fix ToolsTestUtils by removing incorrect closure of Std Stream (#13922)
Reviewers: Lucas Bradstreet <lucas@confluent.io>, Divij Vaidya <diviv@amazon.com>
2023-06-28 17:46:22 +02:00
José Armando García Sancio 8ad0ed3e61
KAFKA-15021; Skip leader epoch bump on ISR shrink (#13765)
When the KRaft controller removes a replica from the ISR because of the controlled shutdown there is no need for the leader epoch to be increased by the KRaft controller. This is accurate as long as the topic partition leader doesn't add the removed replica back to the ISR.

This change also fixes a bug when computing the HWM. When computing the HWM, replicas that are not eligible to join the ISR but are caught up should not be included in the computation. Otherwise, the HWM will never increase for replica.lag.time.max.ms because the shutting down replica is not sending FETCH request. Without this additional fix PRODUCE requests would timeout if the request timeout is greater than replica.lag.time.max.ms.

Because of the bug above the KRaft controller needs to check the MV to guarantee that all brokers support this bug fix before skipping the leader epoch bump.

Reviewers: David Mao <47232755+splett2@users.noreply.github.com>, Divij Vaidya <diviv@amazon.com>, David Jacot <djacot@confluent.io>
2023-06-07 07:20:40 -07:00
Federico Valeri 7e9a82c732
MINOR: Fix for MetadataQuorumCommandErrorTest.testRelativeTimeMs (#13784)
Reviewers: Divij Vaidya <diviv@amazon.com>, David Jacot <djacot@confluent.io>
2023-05-31 18:48:26 +02:00
Federico Valeri 45520c1342
KAFKA-14982: Improve the kafka-metadata-quorum output (#13738)
When running kafka-metadata-quorum script to get the quorum replication status, the LastFetchTimestamp and LastCaughtUpTimestamp output is not human-readable.

I will be convenient to add an optional flag (-hr, --human-readable) to enable a human-readable format showing the delay in ms (i.e. 366 ms ago).

This dealy is computed as (now - timestamp), where they are both represented as Unix time (UTC based).

$ bin/kafka-metadata-quorum.sh --bootstrap-server :9092 describe --replication --human-readable
NodeId	LogEndOffset	Lag	LastFetchTimestamp	LastCaughtUpTimestamp	Status  	
2     	61          	0  	5 ms ago          	5 ms ago             	Leader  	
3     	61          	0  	56 ms ago         	56 ms ago            	Follower	
4     	61          	0  	56 ms ago         	56 ms ago            	Follower

Reviewers: Luke Chen <showuon@gmail.com>
2023-05-29 10:04:46 +08:00
Federico Valeri ac9d11b426
KAFKA-14997: Fix JmxToolTest failing on CI server (#13720)
This test was reported as flaky on CI server.

When connecting to a multi-homed machine using RMI, the wrong address may be returned by the RMI registry to the client, causing the connection to the RMI server to timeout.

This change explicitly set the hostname returned to the the clients in the remote stub object.

Reviewers: Luke Chen <showuon@gmail.com>, vamossagar12 <sagarmeansocean@gmail.com>, Kamal Chandraprakash <kamal.chandraprakash@gmail.com>, hudeqi <16120374@bjtu.edu.cn>, Christo Lolov <christololov@gmail.com>
2023-05-16 10:31:32 +08:00
Kamal Chandraprakash 54a4067f81
KAFKA-14559: Fix JMX tool to handle the object names with wildcard and optional attributes (#13060)
Reviewers: Federico Valeri <fedevaleri@gmail.com>, Satish Duggana <satishd@apache.org>
2023-05-11 21:49:21 +05:30
Christo Lolov dc7819d7f1
KAFKA-14594: Move LogDirsCommand to tools module (#13122)
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2023-05-04 12:00:33 +02:00
Gantigmaa Selenge ea540fa400
KAFKA-14592: Move FeatureCommand to tools (#13459)
KAFKA-14592: Move FeatureCommand to tools

Reviewers: Luke Chen <showuon@gmail.com>
2023-04-25 20:28:37 +08:00
Chia-Ping Tsai 637bc92ba1
MINOR: move RecordReader from org.apache.kafka.tools (client module) to org.apache.kafka.tools.api (tools-api module) (#13454)
Reviewers: Jun Rao <junrao@gmail.com>
2023-04-07 00:20:56 +08:00
Robert Young 2b26db0d38
Switch to SplittableRandom in ProducerPerformance utility (#13482)
Why:
Using java.util.Random to generate every byte sent from the ProducerPerformance
appears to be a limiting factor. Throughput of the ProducerPerformance script is
higher with a file of records as compared to randomly generated records.

On my machine a single thread can generate ~100MB/second of uppercase letters using
java.util.Random and ~300MB/sec using java.util.SplittableRandom. This is a limit on
throughput.

Note: you can optimise further by expanding it from 26 letters to 32 letter generated
as it is more efficient to generate a nicely distributed int when the bound is a
power of two.

Reviewers: Luke Chen <showuon@gmail.com>
2023-03-31 14:52:10 +08:00
vamossagar12 c14f56b484
KAFKA-14586: Moving StreamResetter to tools (#13127)
Moves StreamResetter to tools project.

Reviewers: Federico Valeri <fedevaleri@gmail.com>, Christo Lolov <lolovc@amazon.com>, Bruno Cadonna <cadonna@apache.org>
2023-03-28 14:43:22 +02:00
hudeqi aef004edee
KAFKA-14812:ProducerPerformance still counting successful sending in console when sending failed (#13404)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2023-03-21 16:59:18 +08:00
Chia-Ping Tsai 279c237632
Revert "MINOR: Fixed ProducerPerformance still counting successful sending when sending failed (#13348)" (#13401)
This reverts commit 8e4c0d0b04.

Reviewers: Luke Chen <showuon@gmail.com>
2023-03-16 21:26:01 +08:00
hudeqi 8e4c0d0b04
MINOR: Fixed ProducerPerformance still counting successful sending when sending failed (#13348)
Reviewers: Luke Chen <showuon@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2023-03-15 21:30:51 +08:00
Federico Valeri 07e2f6cd4d
KAFKA-14578: Move ConsumerPerformance to tools (#13215)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Alexandre Dupriez <alexandre.dupriez@gmail.com>
2023-03-06 18:16:55 +01:00
vamossagar12 bb3111f472
KAFKA-14580: Moving EndToEndLatency from core to tools module (#13095)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Federico Valeri <fedevaleri@gmail.com>, Ismael Juma <mlists@juma.me.uk>
2023-03-02 12:05:22 +01:00
Gantigmaa Selenge ea30ec4b56
KAFKA-14590: Move DelegationTokenCommand to tools (#13172)
KAFKA-14590: Move DelegationTokenCommand to tools

Reviewers: Luke Chen <showuon@gmail.com>, Christo Lolov <christo_lolov@yahoo.com>, Federico Valeri <fvaleri@redhat.com>
2023-03-02 14:30:07 +08:00
Ron Dagostino 631e6be3a0
KAFKA-14711: kafaka-metadata-quorum.sh does not honor --command-confi… (#13241)
…g option

https://github.com/apache/kafka/pull/12951 accidentally changed the behavior of the `kafaka-metadata-quorum.sh` CLI by making it silently ignore a `--command-config <filename>` properties file that exists. This was an undetected regression in the 3.4.0 release.  This patch fixes the issue such that any such specified file will be honored.

Reviewers: José Armando García Sancio <jsancio@apache.org>, Ismael Juma <ismael@juma.me.uk>
2023-02-13 18:33:20 -05:00
Federico Valeri 50e0e3c257
KAFKA-14582: Move JmxTool to tools (#13136)
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2023-02-02 11:23:26 +01:00
Mickael Maison 8b44237655
KAFKA-14575: Move ClusterTool to tools module (#13080)
Reviewers: dengziming <dengziming1993@gmail.com>, Federico Valeri  <fedevaleri@gmail.com>
2023-01-22 12:50:43 +01:00
Luke Chen 2575362639
KAFKA-14498: reduce the startup nodes to avoid timeout error (#13016)
In MetadataQuorumCommandTest, we sometimes got the error:

java.util.concurrent.ExecutionException: java.lang.RuntimeException: Received a fatal error while waiting for the broker to catch up with the current cluster metadata.

Since we tried to bring up 3 broker + 3 controllers at the same time, and the config initial.broker.registration.timeout.ms (default 1 min) is sometimes not enough for them to start up. Checking the tests, it doesn't require so many nodes. Reducing the nodes number to make these tests reliable.

Reviewers: dengziming <dengziming1993@gmail.com>, Ismael Juma <ismael@juma.me.uk>
2022-12-21 11:19:22 +08:00
Ismael Juma c0b28fde66
MINOR: Use INFO logging for tools and trogdor tests (#13006)
`TRACE` is too noisy and makes the build slower.

Reviewers: David Jacot <djacot@confluent.io>
2022-12-17 10:22:40 -08:00
Ismael Juma 88725669e7
MINOR: Move MetadataQuorumCommand from `core` to `tools` (#12951)
`core` should only be  used for legacy cli tools and tools that require
access to `core` classes instead of communicating via the kafka protocol
(typically by using the client classes).

Summary of changes:
1. Convert the command implementation and tests to Java and move it to
    the `tools` module.
2. Introduce mechanism to capture stdout and stderr from tests.
3. Change `kafka-metadata-quorum.sh` to point to the new command class.
4. Adjusted the test classpath of the `tools` module so that it supports tests
    that rely on the `@ClusterTests` annotation.
5. Improved error handling when an exception different from `TerseFailure` is
    thrown.
6. Changed `ToolsUtils` to avoid usage of arrays in favor of `List`.

Reviewers: dengziming <dengziming1993@gmail.com>
2022-12-09 09:22:58 -08:00
runom b8754c074a
KAFKA-14355: Fix integer overflow in ProducerPerformance (#12822)
Change types from int to long to avoid overflow

Reviewers: Luke Chen <showuon@gmail.com>,  Igor Soarez <soarez@apple.com>
2022-11-05 20:19:08 +08:00
Kirk True 8e43548175
KAFKA-13725: KIP-768 OAuth code mixes public and internal classes in same package (#12039)
* KAFKA-13725: KIP-768 OAuth code mixes public and internal classes in same package

Move classes into a sub-package of "internal" named "secured" that
matches the layout more closely of the "unsecured" package.

Replaces the concrete implementations in the former packages with
sub-classes of the new package layout and marks them as deprecated. If
anyone is already using the newer OAuth code, this should still work.

* Fix checkstyle and spotbugs violations

Co-authored-by: Kirk True <kirk@mustardgrain.com>

Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
2022-09-23 13:15:15 +05:30
dengziming 150fd5b0b1
KAFKA-13914: Add command line tool kafka-metadata-quorum.sh (#12469)
Add `MetadataQuorumCommand` to describe quorum status, I'm trying to use arg4j style command format, currently, we only support one sub-command which is "describe" and we can specify 2 arguments which are --status and --replication.

```
# describe quorum status
kafka-metadata-quorum.sh --bootstrap-server localhost:9092 describe --replication

ReplicaId	LogEndOffset	Lag	LastFetchTimeMs	LastCaughtUpTimeMs	Status  	
0        	10          	        0  	-1             	        -1                	                 Leader  	
1        	10          	        0  	-1             	        -1                	                 Follower	
2        	10          	        0  	-1             	        -1                	                 Follower	

kafka-metadata-quorum.sh --bootstrap-server localhost:9092 describe --status
ClusterId:                             fMCL8kv1SWm87L_Md-I2hg
LeaderId:                             3002
LeaderEpoch:                      2
HighWatermark:                  10
MaxFollowerLag:                 0
MaxFollowerLagTimeMs:   -1
CurrentVoters:                    [3000,3001,3002]
CurrentObservers:              [0,1,2]

# specify AdminClient properties
kafka-metadata-quorum.sh --bootstrap-server localhost:9092 --command-config config.properties describe --status
```

Reviewers: Jason Gustafson <jason@confluent.io>
2022-08-20 08:37:26 -07:00
Jason Gustafson f0a09ea003
MINOR: Fix event output inconsistencies in TransactionalMessageCopier (#12098)
This patch fixes some strangeness and inconsistency in the messages written by `TransactionalMessageCopier` to stdout. Here is a sample of two messages.

Progress message:
```
{"consumed":33000,"stage":"ProcessLoop","totalProcessed":33000,"progress":"copier-0","time":"2022/04/24 05:40:31:649","remaining":333}
```
The `transactionalId` is set to the value of the `progress` key.

And a shutdown message:
```
{"consumed":33333,"shutdown_complete":"copier-0","totalProcessed":33333,"time":"2022/04/24 05:40:31:937","remaining":0}
```
The `transactionalId` this time is set to the `shutdown_complete` key and there is no `stage` key.

In this patch, we change the following:

1. Use a separate key for the `transactionalId`.
2. Drop the `progress` and `shutdown_complete` keys.
3. Use `stage=ShutdownComplete` in the shutdown message.
4. Modify `transactional_message_copier.py` system test service accordingly.

Reviewers: David Arthur <mumrah@gmail.com>
2022-04-29 10:02:25 -07:00
彭小漪 6145974fef
KAFKA-13728: fix PushHttpMetricsReporter no longer pushes metrics when network failure is recovered. (#11879)
The class PushHttpMetricsReporter no longer pushes metrics when network failure is recovered.

I debugged the code and found the problem here: when we submit a task to the ScheduledThreadPoolExecutor that needs to be executed periodically, if the task throws an exception and is not swallowed, the task will no longer be scheduled to execute.

So when an IO exception occasionally occurs on the network, we should swallow it rather than throw it in task HttpReporter.

Reviewers: Guozhang Wang <wangguoz@gmail.com>
2022-03-19 21:09:28 -07:00
Justine Olshan 7afdb069bf
KAFKA-13750; Client Compatability KafkaTest uses invalid idempotency configs (#11909)
Reviewers: Luke Chen <showuon@gmail.com>, David Jacot <djacot@confluent.io>
2022-03-17 18:00:27 +01:00
Kirk True ec29b62e92
KAFKA-13444: Fix OAuthCompatibilityTool help and add SSL options (#11486)
Reviewers: Jun Rao <junrao@gmail.com>
2021-11-15 15:45:18 -08:00
Jorge Esteban Quilcate Otoya 214b59b3ec
KAFKA-13429: ignore bin on new modules (#11415)
Reviewers: John Roesler <vvcephei@apache.org>
2021-11-10 14:36:24 -06:00
Kirk True 7b379539a5
KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC (#11284)
This task is to provide a concrete implementation of the interfaces defined in KIP-255 to allow Kafka to connect to an OAuth/OIDC identity provider for authentication and token retrieval. While KIP-255 provides an unsecured JWT example for development, this will fill in the gap and provide a production-grade implementation.

The OAuth/OIDC work will allow out-of-the-box configuration by any Apache Kafka users to connect to an external identity provider service (e.g. Okta, Auth0, Azure, etc.). The code will implement the standard OAuth client credentials grant type.

The proposed change is largely composed of a pair of AuthenticateCallbackHandler implementations: one to login on the client and one to validate on the broker.

See the following for more detail:

KIP-768
KAFKA-13202

Reviewers: Yi Ding <dingyi.zj@gmail.com>, Ismael Juma <ismael@juma.me.uk>, Jun Rao <junrao@gmail.com>
2021-10-28 11:36:53 -07:00
Jason Gustafson c1c639db77
KAFKA-13288; Include internal topics when searching hanging transactions (#11319)
This patch ensures that internal topics are included when searching for hanging transactions with the `--broker-id` argument in `kafka-transactions.sh`.

Reviewers: David Jacot <djacot@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
2021-09-10 14:33:37 -07:00
Yanwen(Jason) Lin 66a27af2f1
KAFKA-10038: Supports default client.id for ConsoleConsumer, ProducerPerformance, ConsumerPerformance (#11297)
Reviewers: Guozhang Wang <wangguoz@gmail.com>
2021-09-07 13:49:50 -07:00
dengziming 1d22b0d706
KAFKA-10774; Admin API for Describe topic using topic IDs (#9769)
Reviewers: Justine Olshan <jolshan@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>, Satish Duggana <satishd@apache.org>, Rajini Sivaram <rajinisivaram@googlemail.com>
2021-08-28 09:00:36 +01:00
Jason Gustafson ba47beec01
MINOR: Ensure transactional message copier failures are logged (#11268)
This patch has a couple small improvements to `TransactionalMessageCopier` logging:

- Log all fatal exceptions which cause the copier to shutdown unexpectedly
- Log all non-fatal exceptions which cause the copier to abort a transaction

Reviewers: David Jacot <djacot@confluent.io>
2021-08-27 11:02:47 -07:00
David Jacot 4e2f2b0674
MINOR: Update `TransactionalMessageCopier` to use the latest transaction pattern (#11265)
Reviewers: Jason Gustafson <jason@confluent.io>
2021-08-27 11:11:57 +02:00
Jason Gustafson a5daae20b5
KAFKA-13155; Fix concurrent modification in consumer shutdown (#11164)
The `TransactionalMessageCopier` tool, which is used in system tests attempts to close the consumer as part of a shutdown hook. Although the access is synchronized, there is no guarantee that the consumer has finished polling when shutdown is invoked. The patch fixes the problem by call `wakeup()` from the shutdown hook and pushing the call to `close()` to the main thread.

Reviewers: David Jacot <djacot@confluent.io>
2021-08-10 09:39:59 -07:00
Jason Gustafson f29c43bdbb
KAFKA-12979; Implement command to find hanging transactions (#10974)
This patch implements the `find-hanging` command described in KIP-664: https://cwiki.apache.org/confluence/display/KAFKA/KIP-664%3A+Provide+tooling+to+detect+and+abort+hanging+transactions#KIP664:Providetoolingtodetectandaborthangingtransactions-FindingHangingTransactions.

Reviewers: Luke Chen <showuon@gmail.com>, David Jacot <djacot@confluent.io>
2021-07-06 10:39:59 -07:00
Jason Gustafson fce771579c
KAFKA-12888; Add transaction tool from KIP-664 (#10814)
This patch adds the transaction tool specified in KIP-664: https://cwiki.apache.org/confluence/display/KAFKA/KIP-664%3A+Provide+tooling+to+detect+and+abort+hanging+transactions. This includes all of the logic for describing transactional state and for aborting transactions. The only thing that is left out is the `--find-hanging` implementation, which will be left for a subsequent patch.

Reviewers: Boyang Chen <boyang@apache.org>, David Jacot <djacot@confluent.io>
2021-06-22 09:47:30 -07:00
CHUN-HAO TANG 580c111258
KAFKA-12662: add unit test for ProducerPerformance (#10588)
Reviewers: Luke Chen <showuon@gmail.com>, wenbingshen <oliver.shen999@gmail.com>, dengziming <dengziming1993@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2021-06-17 20:07:12 +08:00
Luke Chen efb7cda178
MINOR: update java doc for deprecated methods (#10722)
Reviewers: Matthias J. Sax <matthias@confluent.io>
2021-05-23 18:33:01 -07:00
A. Sophie Blee-Goldman 3805f3706f
KAFKA-12574: KIP-732, Deprecate eos-alpha and replace eos-beta with eos-v2 (#10573)
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>
2021-04-28 13:22:15 -07:00
Shay Elkin fc405d792d
Minor: Move trogdor out of tools and into its own gradle module (#10539)
Move Trogdor out of tools and into its own gradle module.  This allows us to minimize
the dependencies of the tools module.  We still keep Trogdor in the CLASSPATH
created by kafka-run-class.sh.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2021-04-15 11:37:15 -07:00
lamberken 80de188727
KAFKA-12611: Fix using random payload in ProducerPerformance incorrectly (#10469)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2021-04-13 14:28:31 +08:00
Ismael Juma 2342ec1d1c
KAFKA-12600: Remove deprecated config value `default` for client config `client.dns.lookup` (#10458)
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>
2021-04-01 07:59:59 -07:00
dengziming e6f8ca80cd
MINOR: Fix log statement whose placeholders are inconsistent with arguments (#10312)
1. When the 2nd argument is an exception we don't need a placeholder
2. Placeholders should equal to arguments.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2021-03-22 13:39:05 +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
dengziming 3769bc21b5
MINOR: replace hard-coding utf-8 with StandardCharsets.UTF_8 (#10079)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2021-02-09 10:06:01 +08:00
APaMio 1670362236
MINOR: Replace Collection.toArray(new T[size]) by Collection.toArray(new T[0]) (#9750)
This PR is based on the research of https://shipilev.net/blog/2016/arrays-wisdom-ancients

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2020-12-21 17:38:33 +08:00
Scott Hendricks baef516789
Add ConfigurableProducerSpec to Trogdor for improved E2E latency tracking. (#9736)
Reviewer: Colin P. McCabe <cmccabe@apache.org>
2020-12-18 13:03:59 -08:00
Geordie cc0247bf53
MINOR: Leaves lock() outside the try block (#9687)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2020-12-04 23:56:01 +08:00
Hamza Slama 43fd630d80
MINOR: remove unnecessary semicolon from Agent.java and AgentClient.java (#9625)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2020-11-20 15:34:27 +08:00
Jason Gustafson 927edfece3
KAFKA-10601; Add support for append linger to Raft implementation (#9418)
The patch adds `quorum.append.linger.ms` behavior to the raft implementation. This gives users a powerful knob to tune the impact of fsync.  When an append is accepted from the state machine, it is held in an accumulator (similar to the producer) until the configured linger time is exceeded. This allows the implementation to amortize fsync overhead at the expense of some write latency.

The patch also improves our methodology for testing performance. Up to now, we have relied on the producer performance test, but it is difficult to simulate expected controller loads because producer performance is limited by other factors such as the number of producer clients and head-of-line blocking. Instead, this patch adds a workload generator which runs on the leader after election.

Finally, this patch brings us nearer to the write semantics expected by the KIP-500 controller. It makes the following changes:

- Introduce `RecordSerde<T>` interface which abstracts the underlying log implementation from `RaftClient`. The generic type is carried over to `RaftClient<T>` and is exposed through the read/write APIs.
- `RaftClient.append` is changed to `RaftClient.scheduleAppend` and returns the last offset of the expected log append.
- `RaftClient.scheduleAppend` accepts a list of records and ensures that the full set are included in a single batch.
- Introduce `RaftClient.Listener` with a single `handleCommit` API which will eventually replace `RaftClient.read` in order to surface committed data to the controller state machine. Currently `handleCommit` is only used for records appended by the leader.

Reviewers: José Armando García Sancio <jsancio@users.noreply.github.com>, Guozhang Wang <wangguoz@gmail.com>
2020-10-27 12:10:13 -07:00
Lee Dongjin 8d4bbf22ad
MINOR: trivial cleanups, javadoc errors, omitted StateStore tests, etc. (#8130)
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>, Guozhang Wang <guozhang@confluent.io>, Matthias J. Sax <matthias@confluent.io>
2020-10-07 19:08:31 -07:00
Matthias J. Sax a15387f34d
KAFKA-9274: Revert deprecation of `retries` for producer and admin clients (#9333)
Reviewer: John Roesler <john@confluent.io>
2020-09-30 12:13:34 -07:00
Ismael Juma 7d0086e0c3
KAFKA-10447: Migrate tools module to JUnit 5 (#9231)
This change sets the groundwork for migrating other modules incrementally.

Main changes:
- Replace `junit` 4.13 with `junit-jupiter` and `junit-vintage` 5.7.0-RC1.
- All modules except for `tools` depend on `junit-vintage`.
- `tools` depends on `junit-jupiter`.
- Convert `tools` tests to JUnit 5.
- Update `PushHttpMetricsReporterTest` to use `mockito` instead of `powermock` and `easymock`
(powermock doesn't seem to work well with JUnit 5 and we don't need it since mockito can mock
static methods).
- Update `mockito` to 3.5.7.
- Update `TestUtils` to use JUnit 5 assertions since `tools` depends on it.

Unrelated clean-ups:
- Remove `unit` from package names in a few `core` tests.
- Replace `try/catch/fail` with `assertThrows` in a number of places.
- Tag `CoordinatorTest` as integration test.
- Remove unnecessary type parameters when invoking methods and constructors.

Tested with IntelliJ and gradle. Verified that the following commands work as expected:
* ./gradlew tools:unitTest
* ./gradlew tools:integrationTest
* ./gradlew tools:test
* ./gradlew core:unitTest
* ./gradlew core:integrationTest
* ./gradlew clients:test

Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
2020-09-10 16:14:38 -07:00
John Roesler 26a217c8e7
MINOR: Streams integration tests should not call exit (#9067)
- replace System.exit with Exit.exit in all relevant classes
- forbid use of System.exit in all relevant classes and add exceptions for others

Co-authored-by: John Roesler <vvcephei@apache.org>
Co-authored-by: Matthias J. Sax <matthias@confluent.io>

Reviewers: Lucas Bradstreet <lucas@confluent.io>, Ismael Juma <ismael@confluent.io>
2020-08-05 13:52:50 -07:00
Chia-Ping Tsai dccc82ea9d
KAFKA-10308: Fix flaky core/round_trip_fault_test.py (#9079)
Creating a topic may fail (due to timeout) in running system tests. However, `RoundTripWorker` does not ignore `TopicExistsException` which makes `round_trip_fault_test.py` be a flaky one.

More specifically, a network exception can cause the `CreateTopics` request to reach Kafka but Trogdor retry it
and hit a `TopicAlreadyExists` exception on the retry, failing the test.

Reviewers: Ismael Juma <ismael@juma.me.uk>
2020-08-04 06:53:10 -07:00
Matthias J. Sax 194c56fce2
KAFKA-9274: Mark `retries` config as deprecated and add new `task.timeout.ms` config (#8864)
- 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>
2020-07-21 12:19:13 -07:00
Manikumar Reddy c38825ab97 KAFKA-9432:(follow-up) Set `configKeys` to null in `describeConfigs()` to make it backward compatible with older Kafka versions.
- After #8312, older brokers are returning empty configs,  with latest `adminClient.describeConfigs`.  Old brokers  are receiving empty configNames in `AdminManageer.describeConfigs()` method. Older brokers does not handle empty configKeys. Due to this old brokers are filtering all the configs.
- Update ClientCompatibilityTest to verify describe configs
- Add test case to test describe configs with empty configuration Keys

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

Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>

Closes #9046 from omkreddy/KAFKA-9432
2020-07-21 17:32:11 +05:30
Manikumar Reddy 17256abb23 MINOR: Create ChannelBuilder for each connection in ConnectionStressWorker workload
- Currently we create single channel builder and reuse it in ConnectStressor workload.  This will fail when testing with secure connections, as we close channel builder after first connection.  This PR creates  ChannelBuilder for each test connection.
- Also increase to connect ready wait timeout to 500ms.

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

Reviewers: Ismael Juma <ismael@juma.me.uk>, Rajini Sivaram <rajinisivaram@googlemail.com>

Closes #8937 from omkreddy/Connect
2020-07-10 00:51:50 +05:30
Cheng Tan 55b5b248cd
KAFKA-9893: Configurable TCP connection timeout and improve the initial metadata fetch (KIP-601) (#8683)
Reviewers: David Jacot <djacot@confluent.io>, Rajini Sivaram <rajinisivaram@googlemail.com>
2020-06-30 12:15:17 +01:00
Scott Hendricks 5203ab0c5d
MINOR - Increase the number of Trogdor Histogram buckets to 10000 (#8627)
Reviewers: Colin P. McCabe <cmccabe@apache.org>
2020-05-06 21:10:19 -07:00
Nikolay c07db1c7d9
KAFKA-9573: Fix VerifiableProducer and VerifiableConsumer to work with older Kafka versions (#8197)
These classes are used by `upgrade_test.py` with old Kafka versions so they can
only use functionality that exists in all Kafka versions. This change fixes the test
for Kafka versions older than 0.11.0.

Reviewers: Ismael Juma <ismael@juma.me.uk>
2020-03-01 16:31:15 -08:00
Mitch 96c69da8c1
KAFKA-8507; Unify connection name flag for command line tool [KIP-499] (#8023)
This change updates ConsoleProducer, ConsumerPerformance, VerifiableProducer, and VerifiableConsumer classes to add and prefer the --bootstrap-server flag for defining the connection point of the Kafka cluster. This change is part of KIP-499: https://cwiki.apache.org/confluence/display/KAFKA/KIP-499+-+Unify+connection+name+flag+for+command+line+tool.

Reviewers: Ron Dagostino <rdagostino@confluent.io>, Stanislav Kozlovski <stanislav_kozlovski@outlook.com>,  Chia-Ping Tsai <chia7712@gmail.com>, Jason Gustafson <jason@confluent.io>
2020-02-13 13:44:51 -08:00
Boyang Chen 07db26c20f
KAFKA-9417: New Integration Test for KIP-447 (#8000)
This change mainly have 2 components:

1. extend the existing transactions_test.py to also try out new sendTxnOffsets(groupMetadata) API to make sure we are not introducing any regression or compatibility issue
  a. We shrink the time window to 10 seconds for the txn timeout scheduler on broker so that we could trigger expiration earlier than later

2. create a completely new system test class called group_mode_transactions_test which is more complicated than the existing system test, as we are taking rebalance into consideration and using multiple partitions instead of one. For further breakdown:
  a. The message count was done on partition level, instead of global as we need to visualize 
the per partition order throughout the test. For this sake, we extend ConsoleConsumer to print out the data partition as well to help message copier interpret the per partition data.
  b. The progress count includes the time for completing the pending txn offset expiration
  c. More visibility and feature improvements on TransactionMessageCopier to better work under either standalone or group mode.

Reviewers: Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
2020-02-12 12:34:12 -08:00
Karan Kumar c8d97c6d51
KAFKA-9375: Add names to all Connect threads (#7901)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Ryanne Dolan <ryannedolan@gmail.com>, gcsaba2
2020-01-31 18:21:21 +00:00
Ron Dagostino a3509c0870 MINOR: MiniKdc JVM shutdown hook fix (#7946)
Also made all shutdown hooks consistent and added tests

Reviewers: Ismael Juma <ismael@juma.me.uk>, Rajini Sivaram <rajinisivaram@googlemail.com>
2020-01-24 22:21:12 +00:00
Jason Gustafson 2ac78ff621
MINOR: Propagate LogContext to channel builders and SASL authenticator (#7867)
The log context is useful when debugging applications which have multiple clients. This patch propagates the context to the channel builders and the SASL authenticator.

Reviewers: Ron Dagostino <rndgstn@gmail.com>, Manikumar Reddy <manikumar.reddy@gmail.com>
2020-01-06 14:27:30 -08:00
Ismael Juma 6dc6f6a60d
KAFKA-9324: Drop support for Scala 2.11 (KIP-531) (#7859)
* 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>
2020-01-06 19:51:01 +01:00
Jason Gustafson 9d8ab3a1a2
KAFKA-8509; Add downgrade system test (#7724)
This patch adds a basic downgrade system test. It verifies that producing and consuming continues to work before and after the downgrade.

Reviewers: Ismael Juma <ismael@juma.me.uk>, David Arthur <mumrah@gmail.com>
2019-11-22 10:09:13 -08:00
Colin Patrick McCabe fe7543abd3
Revert "KAFKA-9165: Fix jersey warnings in Trogdor (#7669)" (#7721)
Reviewers: Stanislav Kozlovski <stanislav_kozlovski@outlook.com>, Jason Gustafson <jason@confluent.io>
2019-11-20 12:49:25 -08:00
David Arthur d04699486d
KAFKA-8981 Add rate limiting to NetworkDegradeSpec (#7446)
* Add rate limiting to tc

* Feedback from PR

* Add a sanity test for tc

* Add iperf to vagrant scripts

* Dynamically determine the network interface

* Add some temp code for testing on AWS

* Temp: use hostname instead of external IP

* Temp: more AWS debugging

* More AWS WIP

* More AWS temp

* Lower latency some

* AWS wip

* Trying this again now that ping should work

* Add cluster decorator to tests

* Fix broken import

* Fix device name

* Fix decorator arg

* Remove errant import

* Increase timeouts

* Fix tbf command, relax assertion on latency test

* Fix log line

* Final bit of cleanup

* Newline

* Revert Trogdor retry count

* PR feedback

* More PR feedback

* Feedback from PR

* Remove unused argument
2019-11-18 20:36:00 -05:00
Colin Patrick McCabe 69a63304de
KAFKA-9165: Fix jersey warnings in Trogdor (#7669)
Reviewers: David Arthur <mumrah@gmail.com>, Stanislav Kozlovski <stanislav_kozlovski@outlook.com>
2019-11-15 10:41:12 -08:00
Jason Gustafson 903d66e2f9 KAFKA-9079: Fix reset logic in transactional message copier
The consumer's `committed` API does not return an entry in the response map for a requested partition if there is no committed offset. The transactional message copier, which is used in the transaction system test, did not account for this. If the first transaction attempted by the copier was randomly aborted, then we would not seek to the beginning as expected, which means we would fail to copy some of the records.

This patch fixes the problem by iterating over the assignment rather than the result of `committed` when resetting offsets. It also adds enables additional logging in the transaction message copier service to make finding problems easier in the future.

Author: Jason Gustafson <jason@confluent.io>

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

Closes #7653 from hachikuji/fix-transaction-system-test
2019-11-06 15:59:51 +05:30
Guozhang Wang bcc023773f
KAFKA-8880: Add overloaded function of Consumer.committed (#7304)
1. Add the overloaded functions.
2. Update the code in Streams to use the batch API for better latency (this applies to both active StreamsTask for initialize the offsets, as well as the StandbyTasks for updating offset limits).
3. Also update all unit test to replace the deprecated APIs.

Reviewers: Christopher Pettitt <cpettitt@confluent.io>, Kamal Chandraprakash  <kamal.chandraprakash@gmail.com>, Bill Bejeck <bill@confluent.io>
2019-09-24 13:23:27 -07:00
Scott Hendricks 39afe9fe0e KAFKA-8853; Create sustained connections test for Trogdor
This creates a test that generates sustained connections against Kafka.  There
are three different components we can stress with this, KafkaConsumer,
KafkaProducer, and AdminClient.  This test tries use minimal bandwidth per
connection to reduce overhead impacts.

This test works by creating a threadpool that creates connections and then
maintains a central pool of connections at a specified keepalive rate.  The
keepalive action varies by which component is being stressed:

  * KafkaProducer:  Sends one single produce record.  The configuration for
    the produce request uses the same key/value generator as the ProduceBench
    test.

  * KafkaConsumer: Subscribes to a single partition, seeks to the end, and
    then polls a minimal number of records.  Each consumer connection is its
    own consumer group, and defaults to 1024 bytes as FETCH_MAX_BYTES to keep
    traffic to a minimum.

  * AdminClient: Makes an API call to get the nodes in the cluster.

NOTE: This test is designed to be run alongside a ProduceBench test for a
specific topic, due to the way the Consumer test polls a single partition.
There may be no data returned by the consumer test if this is run on its own.
The connection should still be kept alive, but with no data returned.

Author: Scott Hendricks <scott.hendricks@confluent.io>

Reviewers: Stanislav Kozlovski, Gwen Shapira

Closes #7289 from scott-hendricks/trunk
2019-09-08 19:49:13 -07:00
Robert Yokota add662907d MINOR: fix ProduceBenchWorker not to fail on final produce (#7254)
When sending bad records, the Trogdor task will fail if the final record produced is bad. Instead we should catch the exception to allow the task to finish since sending bad records is a valid use case.

Reviewers: Tu V. Tran <tuvtran97@gmail.com>, Guozhang Wang <wangguoz@gmail.com>
2019-08-27 09:55:51 -07:00
jolshan 2c2b30d96b MINOR: Add RandomComponentPayloadGenerator and update Trogdor documentation (#7103)
Add a new RandomComponentPayloadGenerator that gives a payload based on random selection of another PayloadGenerator.  Additionally, add an example that uses a non-default PayloadGenerator configuration to TROGDOR.md.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2019-07-31 14:00:49 -07:00
Andy Coates 2a133ba656 KAFKA-8454; Add Java AdminClient Interface (KIP-476) (#7087)
Adds an `Admin` interface as specified in [KIP-476](https://cwiki.apache.org/confluence/display/KAFKA/KIP-476%3A+Add+Java+AdminClient+Interface).

Reviewers: Ismael Juma <ismael@juma.me.uk>, Jason Gustafson <jason@confluent.io>
2019-07-22 15:47:34 -07:00
jolshan 442d36241b MINOR: add useConfiguredPartitioner and skipFlush options for ProduceBench
Add a "useConfiguredPartitioner" boolean to specify testing with the configured partitioner, rather than overriding the partitioner in the test.

Add a "skipFlush" boolean to specify skipping the flush operation when producing.  This is helpful when testing some scenarios where linger.ms is greater than 0.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2019-07-03 17:23:36 -07:00
Colin Patrick McCabe 822abe47db
MINOR: WorkerUtils#topicDescriptions must unwrap exceptions properly (#6937)
Reviewers: Ismael Juma <ismael@juma.me.uk>, Stanislav Kozlovski <stanislav_kozlovski@outlook.com>
2019-07-03 16:08:39 -07:00
David Arthur d7a5e31ca2
KAFKA-8519 Add trogdor action to slow down a network (#6912)
This adds a new Trogdor fault spec for inducing network latency on a network device for system testing. It operates very similarly to the existing network partition spec by executing the `tc` linux utility.
2019-06-21 11:30:05 -04:00
Stanislav Kozlovski 58aa04f91e MINOR: Improve Trogdor external command worker docs (#6438)
Reviewers: Colin McCabe <cmccabe@apache.org>, Xi Yang <xi@confluent.io>
2019-06-06 10:04:05 -07:00
Lee Dongjin b43f5446ac KAFKA-8316; Remove deprecated usage of Slf4jRequestLog, SslContextFactory (#6668)
* Remove deprecated class Slf4jRequestLog: use Slf4jRequestLogWriter, CustomRequestLog instread.

1. Remove '@SuppressWarnings("deprecation")' from RestServer#initializeResources, JsonRestServer#start.
2. Remove unused JsonRestServer#httpRequest.

* Fix deprecated class usage: SslContextFactory -> SslContextFactory.[Server, Client]

1. Split SSLUtils#createSslContextFactory into SSLUtils#create[Server, Client]SideSslContextFactory: each method instantiates SslContextFactory.[Server, Client], respectively.
2. SSLUtils#configureSslContextFactoryAuthentication is called from SSLUtils#createServerSideSslContextFactory only.
3. Update SSLUtilsTest following splittion; for client-side SSL Context Factory, SslContextFactory#get[Need, Want]ClientAuth is always false. (SSLUtilsTest#testCreateClientSideSslContextFactory)

Reviewers: Ismael Juma <ismael@juma.me.uk>, Jason Gustafson <jason@confluent.io>
2019-05-20 10:15:15 -07:00
Boyang Chen 9fa331b811 KAFKA-8225 & KIP-345 part-2: fencing static member instances with conflicting group.instance.id (#6650)
For static members join/rejoin, we encode the current timestamp in the new member.id. The format looks like group.instance.id-timestamp.

During consumer/broker interaction logic (Join, Sync, Heartbeat, Commit), we shall check the whether group.instance.id is known on group. If yes, we shall match the member.id stored on static membership map with the request member.id. If mismatching, this indicates a conflict consumer has used same group.instance.id, and it will receive a fatal exception to shut down.

Right now the only missing part is the system test. Will work on it offline while getting the major logic changes reviewed.

Reviewers: Ryanne Dolan <ryannedolan@gmail.com>, Jason Gustafson <jason@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
2019-05-18 07:28:36 -07:00
Ismael Juma a37282415e
MINOR: Upgrade dependencies for Kafka 2.3 (#6665)
Many patch and minor updates.

Scalatest and Jetty deprecated classes that we
use. I removed usages for the former and filed KAFKA-8316 for the latter (I
suppressed the relevant deprecation warnings until the JIRA is fixed). As
part of the scalatest fixes, I also removed `TestUtils.fail` since it duplicates
`Assertions.fail`.

I also fixed a few compiler warnings that have crept in since my last sweep.

Updates of note:
- Jetty: 9.4.14 -> 9.4.18
  * https://github.com/eclipse/jetty.project/releases/tag/jetty-9.4.15.v20190215
  * https://github.com/eclipse/jetty.project/releases/tag/jetty-9.4.16.v20190411
  * https://github.com/eclipse/jetty.project/releases/tag/jetty-9.4.17.v20190418
  * https://github.com/eclipse/jetty.project/releases/tag/jetty-9.4.17.v20190418
  * https://github.com/eclipse/jetty.project/releases/tag/jetty-9.4.18.v20190429
- zstd: 1.3.8-1 -> 1.4.0-1
  * https://github.com/facebook/zstd/releases/tag/v1.4.0
  * zstd's fastest strategy, 6-8% faster in most scenarios
- zookeeper: 3.4.13 -> 3.4.14
  * https://zookeeper.apache.org/doc/r3.4.14/releasenotes.html

### Committer Checklist (excluded from commit message)
- [ ] Verify design and implementation 
- [ ] Verify test coverage and CI build status
- [ ] Verify documentation (including upgrade notes)

Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
2019-05-03 10:35:07 -07:00
Boyang Chen 0f995ba6be KAFKA-7862 & KIP-345 part-one: Add static membership logic to JoinGroup protocol (#6177)
This is the first diff for the implementation of JoinGroup logic for static membership. The goal of this diff contains:

* Add group.instance.id to be unique identifier for consumer instances, provided by end user;
Modify group coordinator to accept JoinGroupRequest with/without static membership, refactor the logic for readability and code reusability.
* Add client side support for incorporating static membership changes, including new config for group.instance.id, apply stream thread client id by default, and new join group exception handling.
* Increase max session timeout to 30 min for more user flexibility if they are inclined to tolerate partial unavailability than burdening rebalance.
* Unit tests for each module changes, especially on the group coordinator logic. Crossing the possibilities like:
6.1 Dynamic/Static member
6.2 Known/Unknown member id
6.3 Group stable/unstable
6.4 Leader/Follower

The rest of the 345 change will be broken down to 4 separate diffs:

* Avoid kicking out members through rebalance.timeout, only do the kick out through session timeout.
* Changes around LeaveGroup logic, including version bumping, broker logic, client logic, etc.
* Admin client changes to add ability to batch remove static members
* Deprecate group.initial.rebalance.delay

Reviewers: Liquan Pei <liquanpei@gmail.com>, Stanislav Kozlovski <familyguyuser192@windowslive.com>, Jason Gustafson <jason@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
2019-04-26 11:44:38 -07:00
Bob Barrett 964e90a725 MINOR: Remove errant lock.unlock() call from RoundTripWorker (#6612)
Reviewers: Colin P. McCabe <cmccabe@apache.org>
2019-04-20 10:43:35 -07:00
Colin Patrick McCabe 04a023e302
MINOR: ConnectionStressWorker: add missing executor shutdown (#6558) 2019-04-11 11:16:06 -07:00
Colin P. Mccabe a674ded0b3 MINOR: fix throttling and status in ConnectionStressWorker
Each separate thread should have its own throttle, so that it can sleep
for an appropriate amount of time when needed.

ConnectionStressWorker should avoid recalculating the status after
shutting down the runnables.  Otherwise, if one runnable is slow to
stop, it will skew the average down in a way that doesn't reflect
reality.  This change moves the status calculation into a separate
periodic runnable that gets shut down cleanly before the other ones.

Author: Colin P. Mccabe <cmccabe@confluent.io>

Reviewers: Gwen Shapira, Stanislav Kozlovski

Closes #6533 from cmccabe/fix_connection_stress_worker
2019-04-04 14:16:56 -07:00
Stanislav Kozlovski 4825bc47a0 KAFKA-8183: Add retries to WorkerUtils#verifyTopics (#6532)
Reviewers: Colin P. McCabe <cmccabe@apache.org>
2019-04-02 17:09:40 -07:00
Colin P. Mccabe 219c22113e MINOR: Optimize ConnectionStressWorker
Optimize ConnectionStressWorker by avoiding creating a new
ChannelBuilder each time we want to open a new connection.

Author: Colin P. Mccabe <cmccabe@confluent.io>

Reviewers: Gwen Shapira

Closes #6518 from cmccabe/optimize-connection-stress-worker
2019-03-29 15:02:10 -07:00
Colin Patrick McCabe b25974c387
MINOR: WorkerUtils#abort: fix bug in abort logic (#6516)
doneFuture is supposed to be completed with an empty string (meaning success) or a non-empty string which is the error message.  Currently, due to exception.getMessage sometimes returning null or an empty string, this is not working correctly.  This patch fixes that.

Reviewers: David Arthur <mumrah@gmail.com>
2019-03-28 14:47:37 -07:00
Stanislav Kozlovski 0d55f0f3ec KAFKA-8102: Add an interval-based Trogdor transaction generator (#6444)
This patch adds a TimeIntervalTransactionsGenerator class which enables the Trogdor ProduceBench worker to commit transactions based on a configurable millisecond time interval.

Also, we now handle 409 create task responses in the coordinator command-line client by printing a more informative message

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2019-03-25 09:58:11 -07:00
Stanislav Kozlovski 6217178139 KAFKA-7819: Improve RoundTripWorker (#6187)
RoundTripWorker to should use a long field for maxMessages rather than an int.  The consumer group used should unique as well.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2019-03-21 10:03:09 -07:00
Rajini Sivaram ca6ac9393b
MINOR: Retain public constructors of classes from public API (#6455)
TopicDescription and ConsumerGroupDescription in org.apache.kafka.clients.admin. are part of the public API, so we should retain the existing public constructor. Changed the new constructor with authorized operations to be package-private to avoid maintaining more public constructors since we only expect admin client to use this.

Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
2019-03-18 08:51:50 +00:00
Stanislav Kozlovski f20f3c1a97 MINOR: Update Trogdor ConnectionStressWorker status at the end of execution (#6445)
Reviewers: Colin P. McCabe <cmccabe@apache.org>
2019-03-15 09:53:21 -07:00
Stanislav Kozlovski 9a384daf0e MINOR: Change Trogdor agent's cleanup executor to a cached thread pool (#6309)
It is best to use a growing thread pool for worker cleanups. This lets us ensure that we close workers as fast as possible and not get slowed down on blocking cleanups.

Reviewers: Colin McCabe <cmccabe@apache.org>, Jason Gustafson <jason@confluent.io>
2019-03-12 08:31:50 -07:00