Commit Graph

10645 Commits

Author SHA1 Message Date
Ismael Juma e2678d57d0
KAFKA-14472: Move TransactionIndex and related to storage module (#12996)
For broader context on this change, please check:

* KAFKA-14470: Move log layer to storage module

Reviewers: Jun Rao <junrao@gmail.com>, Satish Duggana <satishd@apache.org>
2022-12-19 11:31:37 -08:00
David Arthur 802fb11d4e
KAFKA-14529 Use the MetadataVersion from ClusterConfig in ZK tests (#13020)
Reviewers: Ismael Juma <ismael@juma.me.uk>
2022-12-19 14:30:21 -05:00
Bill Bejeck ea65d74f6b
MINOR: No error with zero results state query (#13002)
This PR updates StateQueryResult.getOnlyPartitionResult() to not throw an IllegaArgumentException when there are 0 query results.

Added a test that will fail without this patch

Reviewers: John Roesler<vvcephei@apache.org>
2022-12-19 13:39:06 -05:00
Philip Nee 4548c272ae
KAFKA-14264; New logic to discover group coordinator (#12862)
[KAFKA-14264](https://issues.apache.org/jira/browse/KAFKA-14264)
In this patch, we refactored the existing FindCoordinator mechanism. In particular, we first centralize all of the network operation (send, poll) in `NetworkClientDelegate`, then we introduced a RequestManager interface that is responsible to handle the timing of different kind of requests, based on the implementation.  In this path, we implemented a `CoordinatorRequestManager` which determines when to create an `UnsentRequest` upon polling the request manager.

Reviewers: Jason Gustafson <jason@confluent.io>
2022-12-19 09:48:52 -08:00
Justine Olshan 22bb3e7a58
KAFKA-14417: Address incompatible error code returned by broker from `InitProducerId` (#12968)
Older clients can not handle the `REQUEST_TIMED_OUT` error that is returned from `InitProducerId` when the next producerId block cannot be fetched from the controller. In this patch, we return `COORDINATOR_LOAD_IN_PROGRESS` instead which is retriable.

Reviewers: Jason Gustafson <jason@confluent.io>
2022-12-19 09:33:11 -08:00
Satish Duggana e3cb2deff6
KAFKA-14466: Move ClassloaderAwareRemoteStorageManager to storage module (#13013)
Reviewers: Ismael Juma <ismael@juma.me.uk>
2022-12-19 06:37:33 -08:00
David Jacot 28f29a719d
MINOR: Serialize response in KafkaApisTest (#12865)
Every request used in KafkaApiTests is serialized before hitting KafkaApis. This ensures that backward compatibility is handled correctly, especially ignorable fields. We don't do the same for the responses though. This patch fixes this and refactor how we get back the response in tests.

Reviewers: Mickael Maison <mickael.maison@gmail.com>
2022-12-19 11:30:23 +01:00
Ismael Juma 8d09fc9935
MINOR: Fix compiler errors in `RemoteIndex*` related to `OffsetPosition` (#13008)
#11390 and #12993 were merged in relatively quick succession, which resulted in compiler errors that weren't present when each change was on top of trunk.

Reviewers: Mickael Maison <mickael.maison@gmail.com>
2022-12-17 20:25:33 +01: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 95dc9d9eed
Move IndexEntry and related to storage module (#12993)
For broader context on this change, please check:

* KAFKA-14470: Move log layer to storage module

Reviewers: dengziming <dengziming1993@gmail.com>
2022-12-17 10:07:11 -08:00
Satish Duggana 7146ac57ba
[KAFKA-13369] Follower fetch protocol changes for tiered storage. (#11390)
This PR implements the follower fetch protocol as mentioned in KIP-405.

Added a new version for ListOffsets protocol to receive local log start offset on the leader replica. This is used by follower replicas to find the local log star offset on the leader.

Added a new version for FetchRequest protocol to receive OffsetMovedToTieredStorageException error. This is part of the enhanced fetch protocol as described in KIP-405.

We introduced a new field locaLogStartOffset to maintain the log start offset in the local logs. Existing logStartOffset will continue to be the log start offset of the effective log that includes the segments in remote storage.

When a follower receives OffsetMovedToTieredStorage, then it tries to build the required state from the leader and remote storage so that it can be ready to move to fetch state.

Introduced RemoteLogManager which is responsible for

initializing RemoteStorageManager and RemoteLogMetadataManager instances.
receives any leader and follower replica events and partition stop events and act on them
also provides APIs to fetch indexes, metadata about remote log segments.
Followup PRs will add more functionality like copying segments to tiered storage, retention checks to clean local and remote log segments. This will change the local log start offset and make sure the follower fetch protocol works fine for several cases.

You can look at the detailed protocol changes in KIP: https://cwiki.apache.org/confluence/display/KAFKA/KIP-405%3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-FollowerReplication

Co-authors: satishd@apache.org, kamal.chandraprakash@gmail.com, yingz@uber.com

Reviewers: Kowshik Prakasam <kprakasam@confluent.io>, Cong Ding <cong@ccding.com>, Tirtha Chatterjee <tirtha.p.chatterjee@gmail.com>, Yaodong Yang <yangyaodong88@gmail.com>, Divij Vaidya <diviv@amazon.com>, Luke Chen <showuon@gmail.com>, Jun Rao <junrao@gmail.com>
2022-12-17 09:36:44 -08:00
Kirk True f247aac96a
KAFKA-14496: Wrong Base64 encoder used by OIDC OAuthBearerLoginCallbackHandler (#13000)
The OAuth code to generate the Authentication header was incorrectly
using the URL-safe base64 encoder. For client IDs and/or secrets with
dashes and/or plus signs would not be encoded correctly, leading to the
OAuth server to reject the credentials.

This change uses the correct base64 encoder, per RFC-7617.

Co-authored-by: Endre Vig <vendre@gmail.com>
2022-12-16 19:44:41 +05:30
Daniel Scanteianu e3585a4cd5
MINOR: Document Offset and Partition 0-indexing, fix typo (#12753)
Add comments to clarify that both offsets and partitions are 0-indexed, and fix a minor typo. Clarify which offset will be retrieved by poll() after seek() is used in various circumstances. Also added integration tests.

Reviewers: Luke Chen <showuon@gmail.com>
2022-12-16 17:12:40 +08:00
Colin Patrick McCabe 29c09e2ca1
MINOR: ControllerServer should use the new metadata loader and snapshot generator (#12983)
This PR introduces the new metadata loader and snapshot generator. For the time being, they are
only used by the controller, but a PR for the broker will come soon.

The new metadata loader supports adding and removing publishers dynamically. (In contrast, the old
loader only supported adding a single publisher.) It also passes along more information about each
new image that is published. This information can be found in the LogDeltaManifest and
SnapshotManifest classes.

The new snapshot generator replaces the previous logic for generating snapshots in
QuorumController.java and associated classes. The new generator is intended to be shared between
the broker and the controller, so it is decoupled from both.

There are a few small changes to the old snapshot generator in this PR. Specifically, we move the
batch processing time and batch size metrics out of BrokerMetadataListener.scala and into
BrokerServerMetrics.scala.

Finally, fix a case where we are using 'is' rather than '==' for a numeric comparison in
snapshot_test.py.

Reviewers: David Arthur <mumrah@gmail.com>
2022-12-15 16:53:07 -08:00
vamossagar12 a46d16e7ab
Removing Multicasting partitioner for IQ (#12977)
Follow up PR for KIP-837. We don't want to allow multicasting for IQ. This PR imposes that restriction.

Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>
2022-12-15 15:09:41 -08:00
Akhilesh C 8b045dcbf6
KAFKA-14446: API forwarding support from zkBrokers to the Controller (#12961)
This PR enables brokers which are upgrading from ZK mode to KRaft mode to forward certain metadata
change requests to the controller instead of applying them directly through ZK. To faciliate this,
we now support EnvelopeRequest on zkBrokers (instead of only on KRaft nodes.)

In BrokerToControllerChannelManager, we can now reinitialize our NetworkClient. This is needed to
handle the case when we transition from forwarding requests to a ZK-based broker over the
inter-broker listener, to forwarding requests to a quorum node over the controller listener.

In MetadataCache.scala, distinguish between KRaft and ZK controller nodes with a new type,
CachedControllerId.

In LeaderAndIsrRequest, StopReplicaRequest, and UpdateMetadataRequest, switch from sending both a
zk and a KRaft controller ID to sending a single controller ID plus a boolean to express whether it
is KRaft. The previous scheme was ambiguous as to whether the system was in KRaft or ZK mode when
both IDs were -1 (although this case is unlikely to come up in practice). The new scheme avoids
this ambiguity and is simpler to understand.

Reviewers: dengziming <dengziming1993@gmail.com>, David Arthur <mumrah@gmail.com>, Colin P. McCabe <cmccabe@apache.org>
2022-12-15 14:16:41 -08:00
Chris Egerton dd76872264
KAFKA-13709: Add docs for exactly-once support in Connect (#12941)
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2022-12-15 11:20:49 -05:00
Greg Harris dcc02346c5
KAFKA-13881: Add Clients package description javadocs (#12895)
Reviewers: Mickael Maison <mickael.maison@gmail.com>

, Tom Bentley <tbentley@redhat.com>
2022-12-15 10:03:44 +01:00
David Jacot f8556fe791
KAFKA-14367; Add `DeleteGroups` to the new `GroupCoordinator` interface (#12858)
This patch adds `deleteGroups` to the new `GroupCoordinator` interface and updates `KafkaApis` to use it.

Reviewers: Omnia G H Ibrahim <o.g.h.ibrahim@gmail.com>, Justine Olshan <jolshan@confluent.io>, Jason Gustafson <jason@confluent.io>
2022-12-15 09:29:56 +01:00
Matthias J. Sax de088a2e97 MINOR: update Streams upgrade guide for 3.1 release (#12926)
Reviewers: John Roesler <john@confluent.io>, Bill Bejeck <bill@confluent.io>
2022-12-14 14:35:18 -05:00
Hao Li 9b23d9305d
KAFKA-14395: add config to configure client supplier (#12944)
Implements KIP-884.

Reviewers: Matthias J. Sax <matthias@confluent.io>
2022-12-14 09:17:27 -08:00
David Arthur 87b9c572c6
MINOR: Change KRaft ZK controller registration algorithm (#12973)
Increment the value in "/controller_epoch" when registering a KRaft controller as the active controller. Use the "kraftControllerEpoch" stored under "/controller" to ensure we are registering a newer KRaft controller.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2022-12-14 08:37:06 -05:00
Artem Livshits 7d1e37ea7e
MINOR: Fix FetchFromFollowerIntegrationTest.testFetchFromFollowerWithRoll (#12987)
The test was added with a fix to KAFKA-14379, the problem was that the replication factor for the offset topic was 1 and consumer group coordinator got unavailable when one of the brokers got shut down.

Reviewers: David Jacot <djacot@confluent.io>
2022-12-14 08:30:15 +01:00
David Arthur 67c72596af
KAFKA-14448 Let ZK brokers register with KRaft controller (#12965)
Prior to starting a KIP-866 migration, the ZK brokers must register themselves with the active
KRaft controller. The controller waits for all brokers to register in order to verify that all the
brokers can

A) Communicate with the quorum
B) Have the migration config enabled
C) Have the proper IBP set

This patch uses the new isMigratingZkBroker field in BrokerRegistrationRequest and
RegisterBrokerRecord. The type was changed from int8 to bool for BrokerRegistrationRequest (a
mistake from #12860). The ZK brokers use the existing BrokerLifecycleManager class to register and
heartbeat with the controllers.

Reviewers: Mickael Maison <mickael.maison@gmail.com>, Colin P. McCabe <cmccabe@apache.org>
2022-12-13 13:15:21 -08:00
Jason Gustafson 26a4d42072
MINOR: Pass snapshot ID directly in `RaftClient.createSnapshot` (#12981)
Let `RaftClient.createSnapshot` take the snapshotId directly instead of the committed offset/epoch (which may not exist). 

Reviewers: José Armando García Sancio <jsancio@apache.org>
2022-12-13 10:44:56 -08:00
vamossagar12 73ea6986df
KAFKA-13602: Remove unwanted logging in RecordCollectorImpl.java (#12985)
There is unwanted logging introduced by #12803 as pointed out in this comment: #12803 (comment). This PR removes it.

Reviewers: Lucas Brutschy <lbrutschy@confluent.io>, Bruno Cadonna <cadonna@apache.org>
2022-12-13 16:36:00 +01:00
Greg Harris 526af63cfe
KAFKA-13881: Add Connect package infos (#12937)
Reviewers: Chris Egerton <chrise@aiven.io>
2022-12-13 09:39:10 -05:00
Junyang Liu 7ff779f3f4
KAFKA-14285: Delete quota node in zookeeper when configs are empty (#12729)
The PR resolve issue KAFKA-14285. After doing changeConfigs, check and clean quota nodes if configs are empty, to avoid infinite increasement of quota nodes in zookeeper

Reviewers: Luke Chen <showuon@gmail.com>, Igor Soarez <soarez@apple.com>, dengziming <dengziming1993@gmail.com>
2022-12-13 17:04:11 +08:00
David Jacot 4a9c0fa4a4
KAFKA-14367; Add `DescribeGroups` to the new `GroupCoordinator` interface (#12855)
This patch adds `describeGroups` to the new `GroupCoordinator` interface and updates `KafkaApis` to use it.

Reviewers: Justine Olshan <jolshan@confluent.io>, Jason Gustafson <jason@confluent.io>
2022-12-13 09:19:21 +01:00
David Jacot 2935a52073
MINOR: Small refactor in KafkaApis.handleHeartbeatRequest (#12978)
This is a small follow-up to https://github.com/apache/kafka/pull/12848. 

Reviewers: Justine Olshan <jolshan@confluent.io>, Jason Gustafson <jason@confluent.io>
2022-12-13 09:18:05 +01:00
José Armando García Sancio 3541d5ab18
MINOR; Improve high watermark log messages (#12975)
While debugging KRaft and the metadata state machines it is helpful to always log the first time the replica discovers the high watermark. All other updates to the high watermark are logged at trace because they are more frequent and less useful.

Reviewers: Luke Chen <showuon@gmail.com>
2022-12-12 16:32:16 -08:00
Colin Patrick McCabe b2dea17041
MINOR: Introduce MetadataProvenance and ImageReWriter (#12964)
Introduce MetadataProvenance to encapsulate the three-tuple of (offset, epoch, timestamp) that is
associated with each MetadataImage, as well as each on-disk snapshot. Also introduce a builder
for MetadataDelta.

Remove offset and epoch tracking from MetadataDelta. We do not really need to know this information
until we are creating the final MetadataImage object. Therefore, this bookkeeping should be done by
the metadata loading code, not inside the delta code, like the other bookkeeping. This simplifies a
lot of tests, as well as simplifying RecordTestUtils.  It also makes more sense for snapshots, where
the offset and epoch are the same for every record.

Add ImageReWriter, an ImageWriter that applies records to a MetadataDelta. This is useful when you
need to create a MetadataDelta object that holds the contents of a MetadataImage. This will be
used in the new image loader code (coming soon).

Add ImageWriterOptionsTest to test ImageWriterOptions.

Reviewers: David Arthur <mumrah@gmail.com>
2022-12-12 09:52:06 -08:00
Artem Livshits 43f39c2e60
KAFKA-14379: Consumer should refresh preferred read replica on update metadata (#12956)
The consumer (fetcher) used to refresh the preferred read replica on
three conditions:
    
1. the consumer receives an OFFSET_OUT_OF_RANGE error
2. the follower does not exist in the client's metadata (i.e., offline)
3. after metadata.max.age.ms (5 min default)
    
For other errors, it will continue to reach to the possibly unavailable
follower and only after 5 minutes will it refresh the preferred read
replica and go back to the leader.
    
Another problem is that the client might have stale metadata and not
send fetches to preferred replica, even after the leader redirects to
the preferred replica.
    
A specific example is when a partition is reassigned. the consumer will
get NOT_LEADER_OR_FOLLOWER which triggers a metadata update but the
preferred read replica will not be refreshed as the follower is still
online. it will continue to reach out to the old follower until the
preferred read replica expires.
    
The consumer can instead refresh its preferred read replica whenever it
makes a metadata update request, so when the consumer receives i.e.
NOT_LEADER_OR_FOLLOWER it can find the new preferred read replica without
waiting for the expiration.
    
Generally, we will rely on the leader to choose the correct preferred
read replica and have the consumer fail fast (clear preferred read replica
cache) on errors and reach out to the leader.

Co-authored-by: Jeff Kim <jeff.kim@confluent.io>

Reviewers: David Jacot <djacot@confluent.io>, Jason Gustafson <jason@confluent.io>
2022-12-12 09:55:22 +01:00
vamossagar12 2fa1879247
KAFKA-14454: Making unique StreamsConfig for tests (#12971)
Newly added test KTableKTableForeignKeyInnerJoinCustomPartitionerIntegrationTest#shouldThrowIllegalArgumentExceptionWhenCustomPartionerReturnsMultiplePartitions as part of KIP-837 passes when run individually but fails when is part of IT class and hence is marked as Ignored.

That seemed to have been because of the way StreamsConfig was being initialised so any new test would have used the same names. Because of which the second test never got to the desired state. With this PR, every test gets a unique app name which seems to have fixed the issue. Also, a couple of cosmetic changes

Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>
2022-12-09 17:51:42 -08:00
Ron Dagostino 1aa1984b5f
KAFKA-14392: Fix overly long request timeouts in BrokerToControllerChannelManager (#12856)
In BrokerToControllerChannelManager, set the request timeout to the minimum of the retry timeout
and the controller socket timeout. This fixes some cases where we were unintentionally setting an
overly long request timeout. 

Also, the channel manager used by the BrokerLifecycleManager should set a retry timeout equal to
half of the broker session timeout, rather than the entire broker session timeout, to allow for a
retransmission if the initial attempt fails.

These two fixes should address some cases where heartbeat broker requests were not being resent
in a timely fashion after a network glitch.

Reviewers: Colin P. McCabe <cmccabe@apache.org>, José Armando García Sancio <jsancio@apache.org>
2022-12-09 17:00:19 -08:00
A. Sophie Blee-Goldman d9b139220e
KAFKA-14318: KIP-878, Introduce partition autoscaling configs (#12962)
First PR for KIP-878: Internal Topic Autoscaling for Kafka Streams

Introduces two new configs related to autoscaling in Streams: a feature flag and retry timeout. This PR just adds the configs and gets them passed through to the Streams assignor where they'll ultimately be needed/used

Reviewers: Bill Bejeck <bill@confluent.io>, Walker Carlson <wcarlson@confluent.io>
2022-12-09 15:02:36 -08:00
Rohan 175d491470
MINOR: add docs table entries for new metrics (#12934)
Adds docs for KIP-761.

Reviewers: Anurag Bandyopadhyay (@Anuragkillswitch), Matthias J. Sax <matthias@confluent.io>
2022-12-09 10:28:31 -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
David Jacot f9a09fdd29
MINOR: Small refactor in DescribeGroupsResponse (#12970)
This patch does a few cleanups:
* It removes `DescribeGroupsResponse.fromError` and pushes its logic to `DescribeGroupsRequest.getErrorResponse` to be consistent with how we implemented the other requests/responses.
* It renames `DescribedGroup.forError` to `DescribedGroup.groupError`.

The patch relies on existing tests.

Reviewers: Mickael Maison <mickael.maison@gmail.com>
2022-12-09 14:15:53 +01:00
David Jacot a0c19c05ef
KAFKA-14425; The Kafka protocol should support nullable structs (#12932)
This patch adds support for nullable structs in the Kafka protocol as described in KIP-893 - https://cwiki.apache.org/confluence/display/KAFKA/KIP-893%3A+The+Kafka+protocol+should+support+nullable+structs.

Reviewers: Colin Patrick McCabe <cmccabe@apache.org>
2022-12-08 20:54:29 +01:00
Purshotam Chauhan c6590ee28b KAFKA-14435: Fix `allow.everyone.if.no.acl.found` config behavior for StandardAuthorizer
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>, Colin Patrick McCabe <cmccabe@apache.org>
2022-12-09 00:58:13 +05:30
David Arthur d40561e90a
KAFKA-14427 ZK client support for migrations (#12946)
This patch adds support for reading and writing ZooKeeper metadata during a KIP-866 migration.

For reading metadata from ZK, methods from KafkaZkClient and ZkData are reused to ensure we are decoding the JSON consistently.

For writing metadata, we use a new multi-op transaction that ensures only a single controller is writing to ZK. This is similar to the existing multi-op transaction that KafkaController uses, but it also includes a check on the new "/migration" ZNode. The transaction consists of three operations:

* CheckOp on /controller_epoch
* SetDataOp on /migration with zkVersion
* CreateOp/SetDataOp/DeleteOp (the actual operation being applied)

In the case of a batch of operations (such as topic creation), only the final MultiOp has a SetDataOp on /migration while the other requests use a CheckOp (similar to /controller_epoch).

Reviewers: Colin Patrick McCabe <cmccabe@apache.org>, dengziming <dengziming1993@gmail.com>
2022-12-08 13:14:01 -05:00
A. Sophie Blee-Goldman c1a54671e8
MINOR: Bump trunk to 3.5.0-SNAPSHOT (#12960)
Version bumps in trunk after the creation of the 3.4 branch.

Reviewers: Ismael Juma <ismael@juma.me.uk>
2022-12-07 18:29:20 -08:00
Lucas Brutschy 36a2f7bfd0
KAFKA-14432: RocksDBStore relies on finalizers to not leak memory (#12935)
RocksDBStore relied on finalizers to not leak memory (and leaked memory after the upgrade to RocksDB 7).
The problem was that every call to options.statistics creates a new wrapper object that needs to be finalized.

I simplified the logic a bit and moved the ownership of the statistics from ValueProvider to RocksDBStore.

Reviewers: Bruno Cadonna <cadonna@apache.org>, Anna Sophie Blee-Goldman <ableegoldman@apache.org>, Christo Lolov <lolovc@amazon.com>
2022-12-07 18:25:58 -08:00
Greg Harris 83732489ab
KAFKA-14443: Close topic creation Admin clients in MM2 connectors (#12955)
Reviewers: Omnia G H Ibrahim <o.g.h.ibrahim@gmail.com>, Chris Egerton <chrise@aiven.io>
2022-12-07 16:59:46 -05:00
David Jacot 854dfb5ffc
KAFKA-14367; Add `ListGroups` to the new `GroupCoordinator` interface (#12853)
This patch adds `listGroups` to the new `GroupCoordinator` interface and updates `KafkaApis` to use it.

Reviewers: Justine Olshan <jolshan@confluent.io>, Jason Gustafson <jason@confluent.io>
2022-12-07 20:42:42 +01:00
Colin Patrick McCabe 100e874671
MINOR: Move dynamic config logic to DynamicConfigPublisher (#12958)
Split out the logic for applying dynamic configurations to a KafkaConfig object from
BrokerMetadataPublisher into a new class, DynamicConfigPublisher. This will allow the
ControllerServer to also run this code, in a follow-up change.

Create separate KafkaConfig objects in BrokerServer versus ControllerServer. This is necessary
because the controller will apply configuration changes as soon as its raft client catches up to
the high water mark, whereas the broker will wait for the active controller to acknowledge it has
caught up in a heartbeat response. So when running in combined mode, we want two separate
KafkaConfig objects that are changed at different times.

Minor changes: improve the error message when catching up broker metadata fails. Fix incorrect
indentation in checkstyle/import-control.xml. Invoke AppInfoParser.unregisterAppInfo from
SharedServer.stop so that it happens only when both the controller and broker have shut down.

Reviewers: David Arthur <mumrah@gmail.com>
2022-12-07 10:43:34 -08:00
Gantigmaa Selenge 42cfd57a24
MINOR: Fix some compiler warnings (#12912)
Reviewers: Ismael Juma <ismael@juma.me.uk>, Luke Chen <showuon@gmail.com>
2022-12-07 21:23:01 +08:00
Rajini Sivaram d23ce20bdf
KAFKA-14352: Rack-aware consumer partition assignment protocol changes (KIP-881) (#12954)
Reviewers: David Jacot <djacot@confluent.io>
2022-12-07 11:41:21 +00:00
Lucia Cerchie 923fea583b
KAFKA-14260: add `synchronized` to `prefixScan` method (#12893)
As a result of "14260: InMemoryKeyValueStore iterator still throws ConcurrentModificationException", I'm adding synchronized to prefixScan as an alternative to going back to the ConcurrentSkipList.

Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>
2022-12-06 19:39:32 -08:00