Commit Graph

469 Commits

Author SHA1 Message Date
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
Ismael Juma 69e591db3a
MINOR: Rewrite/Move KafkaNetworkChannel to the `raft` module (#14559)
This is now possible since `InterBrokerSend` was moved from `core` to `server-common`.
Also rewrite/move `KafkaNetworkChannelTest`.

The scala version of `KafkaNetworkChannelTest` passed with the changes here (before I
deleted it).

Reviewers: Justine Olshan <jolshan@confluent.io>, José Armando García Sancio <jsancio@users.noreply.github.com>
2023-10-16 20:10:31 -07:00
Lianet Magrans 58dfa1cc81
MINOR - KAFKA-15550: Validation for negative target times in offsetsForTimes (#14503)
The current KafkaConsumer offsetsForTimes fails with IllegalArgumentException if negative target timestamps are provided as arguments. This change includes the same validation and tests for the new consumer implementation (and some improved comments for the updateFetchPositions)

Reviewer: Lucas Brutschy <lbrutschy@confluent.io>
2023-10-13 09:59:57 +02:00
Jeff Kim 7b5d640cc6
KAFKA-14987; Implement Group/Offset expiration in the new coordinator (#14467)
This patch implements the groups and offsets expiration in the new group coordinator.

Reviewers: Ritika Reddy <rreddy@confluent.io>, David Jacot <djacot@confluent.io>
2023-10-11 23:45:13 -07:00
Mayank Shekhar Narula d817b1b590
KAFKA-15415: On producer-batch retry, skip-backoff on a new leader (#14384)
When producer-batch is being retried, new-leader is known for the partition Vs the leader used in last attempt, then it is worthwhile to retry immediately to this new leader. A partition-leader is considered to be newer, if the epoch has advanced.

Reviewers: Walker Carlson <wcarlson@apache.org>, Kirk True <kirk@kirktrue.pro>, Andrew Schofield <andrew_schofield@uk.ibm.com
2023-10-05 09:11:47 -05:00
Dongnuo Lyu a12f9f97c9
KAFKA-14506: Implement DeleteGroups API and OffsetDelete API (#14408)
This patch implements DeleteGroups and OffsetDelete API in the new group coordinator.

Reviewers: yangy0000, Ritika Reddy <rreddy@confluent.io>, Jeff Kim <jeff.kim@confluent.io>, David Jacot <djacot@confluent.io>
2023-10-04 02:30:45 -07: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
Lucas Brutschy 6263197a62
KAFKA-15326: [9/N] Start and stop executors and cornercases (#14281)
* Implements start and stop of task executors
* Introduce flush operation to keep consumer operations out of the processing threads
* Fixes corner case: handle requested unassignment during shutdown
* Fixes corner case: handle race between voluntary unassignment and requested unassigment
* Fixes corner case: task locking future completes for the empty set
* Fixes corner case: we should not reassign a task with an uncaught exception to a task executor
* Improved logging
* Number of threads controlled from outside, of the TaskManager

Reviewers: Bruno Cadonna <bruno@confluent.io>
2023-10-02 15:41:21 +02: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
Ismael Juma 7ba6d7a0b4
MINOR: Update to Scala 2.13.12 (#14430)
It offers a quickfix action for certain errors, includes a number of bug fixes and it
introduces a new warning by default (https://github.com/scala/scala/pull/10462).

In addition to the scala version bump, we also fix the new compiler warnings and
bump the scalafmt version (the previous version failed with the new scala version).

Release notes: https://github.com/scala/scala/releases/tag/v2.13.12

Reviewers: Divij Vaidya <diviv@amazon.com>, Satish Duggana <satishd@apache.org>
2023-09-24 06:05:12 -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
Tyler Bertrand eea1854479
KAFKA-15476: Resolves cache misses in checkstyle (#14344)
Resolves cache misses in checkstyle tasks due to absolute paths in configProperties.

Sets configDirectory extension property, which is made available by the checkstyle plugin as ${config_loc} in the checkstyle xml files, as shown in the Checkstyle Gradle docs. The absolute paths set in configProperties are then replaced by relative paths from configDirectory. Because the header and suppression config file names are static and only referenced once, these were removed from configProperties and the file names are given directly in checkstyle.xml

Reviewers: Divij Vaidya <diviv@amazon.com>
2023-09-19 10:51:57 +02:00
Kirk True e1dc6d9f34
KAFKA-14274 [2-5/7]: Introduction of more infrastructure for forthcoming fetch request manager (#14359)
This continues the work of providing the groundwork for the fetch
refactoring work by introducing some new classes and refactoring the
existing code to use the new classes where applicable.

Changes:

* Minor clean up of the events classes to make data immutable,
  private, and implement toString().
* Added IdempotentCloser which prevents a resource from being closed
  more than once. It's general enough that it could be used elsewhere
  in the project, but it's limited to the consumer internals for now.
* Split core Fetcher code into classes to buffer raw results
  (FetchBuffer) and to collect raw results into ConsumerRecords
  (FetchCollector). These can be tested and changed in isolation from
  the core fetcher logic.
* Added NodeStatusDetector which abstracts methods from
  ConsumerNetworkClient so that it and NetworkClientDelegate can be
  used in AbstractFetch via the interface instead of using
  ConsumerNetworkClient directly.

Reviewers: Jun Rao <junrao@gmail.com>
2023-09-16 09:15:37 -07:00
zhaohaidao f309299f3c
KAFKA-14503: Implement ListGroups (#14271)
This patch implements the ListGroups API in the new group coordinator.

Reviewers: David Jacot <djacot@confluent.io>
2023-09-14 23:45:03 -07:00
Jeff Kim e9057aab37
KAFKA-14502; Implement LeaveGroup protocol in new GroupCoordinator (#14147)
This patch implements the LeaveGroup API in the new group coordinator.

Reviewers: David Jacot <djacot@confluent.io>
2023-09-13 01:43:37 -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
Andrew Schofield b49013b73e
KAFKA-9800: Exponential backoff for Kafka clients - KIP-580 (#14111)
Implementation of KIP-580 to add exponential back-off to situations in which retry.backoff.ms
is used to delay backoff attempts. This KIP adds exponential backoff behavior with a maximum
controlled by a new config retry.backoff.max.ms, together with a +/- 20% of jitter to spread the
retry attempts of the client fleet.

Reviewers: Mayank Shekhar Narula <mayanks.narula@gmail.com>, Milind Luthra <i.milind.luthra@gmail.com>, Kirk True <kirk@mustardgrain.com>, Jun Rao<junrao@gmail.com>
2023-09-05 11:57:51 -07:00
Kamal Chandraprakash 4590d565ef
KAFKA-15399: Enable OffloadAndConsumeFromLeader test (#14285)
Reviewers: Divij Vaidya <diviv@amazon.com>, Christo Lolov <lolovc@amazon.com>, Satish Duggana <satishd@apache.org>
2023-08-28 12:29:50 +02:00
Satish Duggana d4ab3ae85a
KAFKA-14888: Added remote log segments retention mechanism based on time and size. (#13561)
This change introduces a remote log segment segment retention cleanup mechanism.

RemoteLogManager runs retention cleanup activity tasks on each leader replica. It assesses factors such as overall size and retention duration, subsequently removing qualified segments from remote storage. This process also involves adjusting the log-start-offset within the UnifiedLog accordingly. It also cleans up the segments which have epochs earlier than the earliest leader epoch in the current leader. 

Co-authored-by: Satish Duggana <satishd@apache.org>
Co-authored-by: Kamal Chandraprakash <kamal.chandraprakash@gmail.com>

Reviewers: Jun Rao <junrao@gmail.com>, Divij Vaidya <diviv@amazon.com, Luke Chen <showuon@gmail.com>, Kamal Chandraprakash <kamal.chandraprakash@gmail.com>, Christo Lolov <lolovc@amazon.com>, Jorge Esteban Quilcate Otoya <quilcate.jorge@gmail.com>, Alexandre Dupriez <alexandre.dupriez@gmail.com>, Nikhil Ramakrishnan <ramakrishnan.nikhil@gmail.com>
2023-08-25 05:27:59 +05:30
Kamal Chandraprakash 6492164d9c
KAFKA-15167: Tiered Storage Test Harness Framework (#14116)
`TieredStorageTestHarness` is a base class for integration tests exercising the tiered storage functionality. This uses  `LocalTieredStorage` instance as the second-tier storage system and `TopicBasedRemoteLogMetadataManager` as the remote log metadata manager.

Co-authored-by: Alexandre Dupriez <alexandre.dupriez@gmail.com>
Co-authored-by: Kamal Chandraprakash <kamal.chandraprakash@gmail.com>
2023-08-20 17:15:52 +05:30
Proven Provenzano c2759df067
KAFKA-15219: KRaft support for DelegationTokens (#14083)
Reviewers: David Arthur <mumrah@gmail.com>, Ron Dagostino <rndgstn@gmail.com>, Manikumar Reddy <manikumar.reddy@gmail.com>, Viktor Somogyi <viktor.somogyi@cloudera.com>
2023-08-19 14:01:08 -04: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
Greg Harris ab60bce090
KAFKA-15239: Fix ThroughputThrottler import-control (#14188)
Reviewers: Chris Egerton <chrise@aiven.io>
2023-08-10 16:53:49 -07:00
Nikolay ddeb89f4a9
KAFKA-14595: Move AdminUtils to server-common (#14096)
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2023-08-09 10:32:45 +02:00
Luke Chen 748175ce62
KAFKA-15189: only init remote topic metrics when enabled (#14133)
Only initialize remote topic metrics when system-wise remote storage is enabled to avoid impacting performance for existing brokers. Also add tests.

Reviewers: Divij Vaidya <diviv@amazon.com>, Kamal Chandraprakash <kamal.chandraprakash@gmail.com>
2023-08-05 13:00:16 +08:00
Ivan Yurchenko b3db905b27
KAFKA-15107: Support custom metadata for remote log segment (#13984)
* KAFKA-15107: Support custom metadata for remote log segment

This commit does the changes discussed in the KIP-917. Mainly, changes the `RemoteStorageManager` interface in order to return `CustomMetadata` and then ensures these custom metadata are stored, propagated, (de-)serialized correctly along with the standard metadata throughout the whole lifecycle. It introduces the `remote.log.metadata.custom.metadata.max.size` to limit the custom metadata size acceptable by the broker and stop uploading in case a piece of metadata exceeds this limit.

On testing:
1. `RemoteLogManagerTest` checks the case when a piece of custom metadata is larger than the configured limit.
2. `RemoteLogSegmentMetadataTest` checks if `createWithUpdates` works correctly, including custom metadata.
3. `RemoteLogSegmentMetadataTransformTest`, `RemoteLogSegmentMetadataSnapshotTransformTest`, and `RemoteLogSegmentMetadataUpdateTransformTest` were added to test the corresponding class (de-)serialization, including custom metadata.
4. `FileBasedRemoteLogMetadataCacheTest` checks if custom metadata are being correctly saved and loaded to a file (indirectly, via `equals`).
5. `RemoteLogManagerConfigTest` checks if the configuration setting is handled correctly.

Reviewers: Luke Chen <showuon@gmail.com>, Satish Duggana <satishd@apache.org>, Divij Vaidya <diviv@amazon.com>
2023-08-04 18:23:25 +05:30
Jeff Kim 19f9e1e6d0
KAFKA-14501: Implement Heartbeat protocol in new GroupCoordinator (#14056)
This patch implements the existing Heartbeat API in the new Group Coordinator.

Reviewers: David Jacot <djacot@confluent.io>
2023-07-28 15:13:27 +02:00
Hao Li ed44bcd71b
KAFKA-15022: [3/N] use graph to compute rack aware assignment for active stateful tasks (#14030)
Part of KIP-925.

Reviewers: Matthias J. Sax <matthias@confluent.io>
2023-07-26 16:02:52 -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
Colin Patrick McCabe c7de30f38b
KAFKA-15183: Add more controller, loader, snapshot emitter metrics (#14010)
Implement some of the metrics from KIP-938: Add more metrics for
measuring KRaft performance.

Add these metrics to QuorumControllerMetrics:
    kafka.controller:type=KafkaController,name=TimedOutBrokerHeartbeatCount
    kafka.controller:type=KafkaController,name=EventQueueOperationsStartedCount
    kafka.controller:type=KafkaController,name=EventQueueOperationsTimedOutCount
    kafka.controller:type=KafkaController,name=NewActiveControllersCount

Create LoaderMetrics with these new metrics:
    kafka.server:type=MetadataLoader,name=CurrentMetadataVersion
    kafka.server:type=MetadataLoader,name=HandleLoadSnapshotCount

Create SnapshotEmitterMetrics with these new metrics:
    kafka.server:type=SnapshotEmitter,name=LatestSnapshotGeneratedBytes
    kafka.server:type=SnapshotEmitter,name=LatestSnapshotGeneratedAgeMs

Reviewers: Ron Dagostino <rndgstn@gmail.com>
2023-07-24 21:13:58 -07:00
David Jacot 2528dd4116
KAFKA-14499: [2/N] Add OffsetCommit record & related (#14047)
This patch does a few things:
1) It introduces the `OffsetAndMetadata` class which hold the committed offsets in the group coordinator.
2) It adds methods to deal with OffsetCommit records to `RecordHelpers`.
3) It adds `MetadataVersion#offsetCommitValueVersion` to get the version of the OffsetCommit value record that should be used.

Reviewers: Jeff Kim <jeff.kim@confluent.io>, David Arthur <mumrah@gmail.com>, Justine Olshan <jolshan@confluent.io>
2023-07-21 20:09:06 +02:00
Yash Mayya 4daeb2714c
KAFKA-13431 (KIP-793): Expose the original pre-transform topic partition and offset in sink records (#14024)
Reviewers: Greg Harris <greg.harris@aiven.io>, Chris Egerton <chrise@aiven.io>
2023-07-21 12:06:01 -04:00
Luke Chen 27ea025e33
KAFKA-15176: add tests for tiered storage metrics (#13999)
Added tests for metrics:
1. RemoteLogReaderTaskQueueSize
2. RemoteLogReaderAvgIdlePercent
3. RemoteLogManagerTasksAvgIdlePercent

Also, added tests for OffsetOutOfRangeException will be thrown while reading logs

Reviewers: Christo Lolov <christololov@gmail.com>, Satish Duggana <satishd@apache.org>
2023-07-21 10:30:33 +08: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
Jeff Kim a500c3ecf9
KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator (#13870)
This patch implements the existing JoinGroup protocol within the new group coordinator. 

Some notable differences:
* Methods return a CoordinatorResult to the runtime framework, which includes records to append to the log as well as a future to complete after the append succeeds/fails.
* The coordinator runtime ensures that only a single thread will be processing a group at any given time, therefore there is no more locking on groups.
* Instead of using on purgatories, we rely on the Timer interface to schedule/cancel delayed operations.

Reviewers: David Jacot <djacot@confluent.io>
2023-07-19 09:15:13 +02:00
Abhijeet Kumar fd3b1137d2
KAFKA-14953: Add tiered storage related metrics (#13944)
* KAFKA-14953: Adding RemoteLogManager metrics
In this PR, I have added the following metrics that are related to tiered storage mentioned in[ KIP-405](https://cwiki.apache.org/confluence/display/KAFKA/KIP-405%3A+Kafka+Tiered+Storage).
|Metric|Description|
|-----------------------------------------|--------------------------------------------------------------|
| RemoteReadRequestsPerSec                    | Number of remote storage read requests per second               |
| RemoteWriteRequestsPerSec                    | Number of remote storage write requests per second              |
| RemoteBytesInPerSec                                | Number of bytes read from remote storage per second           |
| RemoteReadErrorsPerSec                          | Number of remote storage read errors per second                   |
| RemoteBytesOutPerSec                             | Number of bytes copied to remote storage per second            |
| RemoteWriteErrorsPerSec                          | Number of remote storage write errors per second                   |
| RemoteLogReaderTaskQueueSize             | Number of remote storage read tasks pending for execution.  |
| RemoteLogReaderAvgIdlePercent             | Average idle percent of the remote storage reader thread pool|
| RemoteLogManagerTasksAvgIdlePercent | Average idle percent of RemoteLogManager thread pool          |

Added unit tests for all the rate metrics.

Reviewers: Luke Chen <showuon@gmail.com>, Divij Vaidya <diviv@amazon.com>, Kamal Chandraprakash<kamal.chandraprakash@gmail.com>, Jorge Esteban Quilcate Otoya <quilcate.jorge@gmail.com>, Staniel Yao<yaolixinylx@gmail.com>, hudeqi<1217150961@qq.com>, Satish Duggana <satishd@apache.org>
2023-07-18 20:16:19 +05:30
Omnia G H Ibrahim 0c6b1a4e9a
KAFKA-14737: Move kafka.utils.json to server-common (#13585)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Federico Valeri <fedevaleri@gmail.com>
2023-07-18 11:02:40 +02:00
vamossagar12 fa5b493241
KAFKA-14647: Move TopicFilter to server-common/utils (#13158)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Federico Valeri <fedevaleri@gmail.com>
2023-07-18 10:38:56 +02:00
David Jacot 32ff347b2c
KAFKA-14462; [23/23] Wire GroupCoordinatorService in BrokerServer (#13991)
This patch wires the new group coordinator in BrokerServer (KRaft only). With this, it is now possible to run a cluster with the new group coordinator and to use the ConsumerGroupHeartbeat API by specifying the following two properties:
- group.coordinator.new.enable = true (to enable the new group coordinator)
- unstable.api.versions.enable = true (to enable unreleased APIs)

Note that the new group coordinator does not support all the existing APIs yet.

Reviewers: Jeff Kim <jeff.kim@confluent.io>, Justine Olshan <jolshan@confluent.io>
2023-07-14 17:41:06 +02:00
Satish Duggana 7e2f878713
KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module. (#13275)
KAFKA-14522 Rewrite and Move of RemoteIndexCache to storage module.
Cleanedup index file suffix usages and other minor cleanups

Reviewers: Jun Rao <junrao@gmail.com>, Ismael Juma <ismael@juma.me.uk>, Luke Chen <showuon@gmail.com>, Divij Vaidya <diviv@amazon.com>, Kamal Chandraprakash<kamal.chandraprakash@gmail.com>, Jorge Esteban Quilcate Otoya <quilcate.jorge@gmail.com>
2023-07-11 23:55:23 +05:30
David Jacot bd1f02b2be
MINOR: Move MockTimer to server-common (#13954)
This patch rewrites MockTimer in Java and moves it from core to server-common. This continues the work started in https://github.com/apache/kafka/pull/13820.

Reviewers: Divij Vaidya <diviv@amazon.com>
2023-07-06 14:56:05 +02:00
David Jacot 98fbd8afc7
KAFKA-14462; [20/N] Refresh subscription metadata on new metadata image (#13901)
This patch adds (1) the logic to propagate a new MetadataImage to the running coordinators; and (2) the logic to ensure that all the consumer groups subscribed to topics with changes will refresh their subscriptions metadata on the next heartbeat. In the mean time, it ensures that freshly loaded consumer groups also refresh their subscriptions metadata on the next heartbeat.

Reviewers: Jeff Kim <jeff.kim@confluent.io>, Justine Olshan <jolshan@confluent.io>
2023-07-05 18:28:38 +02:00
Kirk True a81f35c1c8
KAFKA-14831: Illegal state errors should be fatal in transactional producer (#13591)
Poison the transaction manager if we detect an illegal transition in the Sender thread. A ThreadLocal in is stored in TransactionManager so that the Sender can inform TransactionManager which thread it's using.

Reviewers: Daniel Urban <durban@cloudera.com>, Justine Olshan <jolshan@confluent.io>, Jason Gustafson <jason@confluent.io>
2023-06-29 11:21:15 -07:00
Yash Mayya 6e72986949
KAFKA-14784: Connect offset reset REST API (#13818)
Reviewers: Chris Egerton <chrise@aiven.io>
2023-06-23 13:27:46 -04:00
David Jacot a81486e4f8
KAFKA-14462; [18/N] Add GroupCoordinatorService (#13812)
This patch introduces the GroupCoordinatorService. This is the new (incomplete) implementation of the group coordinator based on the coordinator runtime introduced in https://github.com/apache/kafka/pull/13795.

Reviewers: Divij Vaidya <diviv@amazon.com>, Justine Olshan <jolshan@confluent.io>
2023-06-22 09:06:10 +02:00
Dimitar Dimitrov b100f1efac
KAFKA-15087 Move/rewrite InterBrokerSendThread to server-commons (#13856)
The Java rewrite is kept relatively close to the Scala original
to minimize potential newly introduced bugs and to make reviewing
simpler. The following details might be of note:
- The `Logging` trait moved to InterBrokerSendThread with the
rewrite of ShutdownableThread has been similarly moved to any
subclasses that currently use it. InterBrokerSendThread's own
logging has been made to use ShutdownableThread's logger which
mimics the prefix/log identifier that the trait provided.
- The case RequestAndCompletionHandler class has been made a
separate POJO class and the internal-use UnsentRequests class
has been kept as a static nested class.
- The relatively commonly used but internal (not part of the
public API) clients classes that InterBrokerSendThread relies on
have been allowlisted in the server-common import control.
- The accompanying test class has also been moved and rewritten
with one new test added and most of the pre-existing tests made
stricter.

Reviewers: David Jacot <djacot@confluent.io>
2023-06-20 16:50:46 +02:00
Colin P. McCabe cd3c0ab1a3 KAFKA-15060: fix the ApiVersionManager interface
This PR expands the scope of ApiVersionManager a bit to include returning the current
MetadataVersion and features that are in effect. This is useful in general because that information
needs to be returned in an ApiVersionsResponse. It also allows us to fix the ApiVersionManager
interface so that all subclasses implement all methods of the interface. Having subclasses that
don't implement some methods is dangerous because they could cause exceptions at runtime in
unexpected scenarios.

On the KRaft controller, we were previously performing a read operation in the QuorumController
thread to get the current metadata version and features. With this PR, we now read a volatile
variable maintained by a separate MetadataVersionContextPublisher object. This will improve
performance and simplify the code. It should not change the guarantees we are providing; in both
the old and new scenarios, we need to be robust against version skew scenarios during updates.

Add a Features class which just has a 3-tuple of metadata version, features, and feature epoch.
Remove MetadataCache.FinalizedFeaturesAndEpoch, since it just duplicates the Features class.
(There are some additional feature-related classes that can be consolidated in in a follow-on PR.)

Create a java class, EndpointReadyFutures, for managing the futures associated with individual
authorizer endpoints. This avoids code duplication between ControllerServer and BrokerServer and
makes this code unit-testable.

Reviewers: David Arthur <mumrah@gmail.com>, dengziming <dengziming1993@gmail.com>, Luke Chen <showuon@gmail.com>
2023-06-19 16:46:44 -07:00
David Jacot 7eea2a3908
MINOR: Move MockTime to server-common (#13823)
This patch rewrite `MockTime` in Java and moves it to `server-common` module. This is a prerequisite to move `MockTimer` later on to `server-common` as well. 

Reviewers: David Arthur <mumrah@gmail.com>
2023-06-09 08:54:25 +02:00
Lianet Magrans 4af4bccbbf
KAFKA-14966: Extract OffsetFetcher reusable logic (#13815)
The OffsetFetcher is internally used by the KafkaConsumer to fetch offsets, validate and reset positions. For the new KafkaConsumer with a refactored threading model, similar functionality will be needed.

This is an initial refactoring for extracting logic from the OffsetFetcher, that will be reused by the new consumer implementation. No changes to the existing logic, just extracting classes, functions or pieces of logic.

All the functionality moved out of the OffsetFetcher is already covered by tests in OffsetFetcherTest and FetcherTest. There were no individual tests for the extracted functions, so no tests were migrated.

Reviewers: Jun Rao <junrao@gmail.com>
2023-06-08 14:03:45 -07:00