Adds support for nonzero log start offsets.
Changes to `Log`:
1. Add a new "reason" for increasing the log start offset. This is used by `KafkaMetadataLog` when a snapshot is generated.
2. `LogAppendInfo` should return if it was rolled because of an records append. A log is rolled when a new segment is created. This is used by `KafkaMetadataLog` to in some cases delete the created segment based on the log start offset.
Changes to `KafkaMetadataLog`:
1. Update both append functions to delete old segments based on the log start offset whenever the log is rolled.
2. Update `lastFetchedEpoch` to return the epoch of the latest snapshot whenever the log is empty.
3. Add a function that empties the log whenever the latest snapshot is greater than the replicated log. This is used when first loading the `KafkaMetadataLog` and whenever the `KafkaRaftClient` downloads a snapshot from the leader.
Changes to `KafkaRaftClient`:
1. Improve `validateFetchOffsetAndEpoch` so that it can handle fetch offset and last fetched epoch that are smaller than the log start offset. This is in addition to the existing code that check for a diverging log. This is used by the raft client to determine if the Fetch response should include a diverging epoch or a snapshot id.
2. When a follower finishes fetching a snapshot from the leader fully truncate the local log.
3. When polling the current state the raft client should check if the state machine has generated a new snapshot and update the log start offset accordingly.
Reviewers: Jason Gustafson <jason@confluent.io>
Updated CreateTopicResponse, DeleteTopicsRequest/Response and added some new AdminClient methods and classes. Now the newly created topic ID will be returned in CreateTopicsResult and found in TopicAndMetadataConfig, and topics can be deleted by supplying topic IDs through deleteTopicsWithIds which will return DeleteTopicsWithIdsResult.
Reviewers: dengziming <dengziming1993@gmail.com>, Rajini Sivaram <rajinisivaram@googlemail.com>
We should ensure `NetworkClient` is closed properly when `InterBrokerSendThread` is shutdown. Also use `initiateShutdown` instead of `wakeup()` to alert polling thread.
Reviewers: David Jacot <djacot@confluent.io>
Make the default state store directory location to follow
OS-specific temporary directory settings or java.io.tmpdir
JVM parameter, with Utils#getTempDir.
Reviewers: Matthias J. Sax <mjsax@apache.org>, John Roesler <vvcephei@apache.org>
With KIP-595, we previously expect `RaftConfig` to specify the quorum voter endpoints upfront on startup. In the general case, this works fine. However, for testing where the bound port is not known ahead of time, we need a lazier approach that discovers the other voters in the quorum after startup.
In this patch, we take the voter endpoint initialization out of `KafkaRaftClient.initialize` and move it to `RaftManager`. We use a special address to indicate that the voter addresses will be provided later This approach also lends itself well to future use cases where we might discover voter addresses through an external service (for example).
Reviewers: Jason Gustafson <jason@confluent.io>
A few small cleanups in `GroupCoordinator` and related classes:
- Remove redundant `groupId` field from `MemberMetadata`
- Remove redundant `isStaticMember` field from `MemberMetadata`
- Fix broken log message in `GroupMetadata.loadGroup` and apply it to all loaded members
- Remove ancient TODOs and no-op methods from `GroupCoordinator`
- Move removal of static members into `GroupMetadata.remove`
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, David Jacot <djacot@confluent.io>
This patch bumps the version of the Metadata API and removes the `IncludeClusterAuthorizedOperations` and the `IncludeClusterAuthorizedOperations` fields from version 11 and onward.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Rajini Sivaram <rajinisivaram@googlemail.com>
The primary purpose of this patch is to remove the internal `enable.metadata.quorum` configuration. Instead, we rely on `process.roles` to determine if the self-managed quorum has been enabled. As a part of this, I've done the following:
1. Replace the notion of "disabled" APIs with "controller-only" APIs. We previously marked some APIs which were intended only for the KIP-500 as "disabled" so that they would not be unintentionally exposed. For example, the Raft quorum APIs were disabled. Marking them as "controller-only" carries the same effect, but makes the intent that they should be only exposed by the KIP-500 controller clearer.
2. Make `ForwardingManager` optional in `KafkaServer` and `KafkaApis`. Previously we used `null` if forwarding was enabled and relied on the metadata quorum check.
3. Make `zookeeper.connect` an optional configuration if `process.roles` is defined.
4. Update raft README to remove reference to `zookeeper.conntect`
Reviewers: Colin Patrick McCabe <cmccabe@confluent.io>, Boyang Chen <boyang@confluent.io>
This patch moves Raft config definitions from `RaftConfig` to `KafkaConfig`, where they are re-defined as internal configs until we are ready to expose them. It also adds the missing "controller" prefix that was added by KIP-631.
Reviewers: Jason Gustafson <jason@confluent.io>
Topics processed by the controller and topics newly created will only be given topic IDs if the inter-broker protocol version on the controller is greater than 2.8. This PR also adds a kafka config to specify whether the IBP is greater or equal to 2.8. System tests have been modified to include topic ID checks for upgrade/downgrade tests. This PR also adds a new integration test file for requests/responses that are not gated by IBP (ex: metadata)
Reviewers: dengziming <dengziming1993@gmail.com>, Lucas Bradstreet <lucas@confluent.io>, Rajini Sivaram <rajinisivaram@googlemail.com>
Get controller api version intersection setup for client queries. When the unsupported exception was hit in the EnvelopeResponse, close the client connection to let it rediscover the api version.
Reviewers: Jason Gustafson <jason@confluent.io>
This patch attempts to generalize server initialization for KIP-500. It adds a `Server` trait which `KafkaServer` extends for the legacy Zookeeper server, and a new `KafkaRaftServer` for the new server. I have also added stubs for `KafkaRaftBroker` and `KafkaRaftController` to give a clearer idea how this will be used.
Note that this patch removes `KafkaServerStartable`, which was intended to enable custom startup logic, but was not codified into an official API and is not planned to be supported after KIP-500.
Reviewers: Ismael Juma <ismael@juma.me.uk>, Colin P. McCabe <cmccabe@apache.org>
We would like to be able to use `KafkaRaftClient` for tooling/debugging use cases. For this, we need the localId to be optional so that the client can be used more like a consumer. This is already supported in the `Fetch` protocol by setting `replicaId=-1`, which the Raft implementation checks for. We just need to alter `QuorumState` so that the `localId` is optional. The main benefit of doing this is that it saves tools the need to generate an arbitrary id (which might cause conflicts given limited Int32 space) and it lets the leader avoid any local state for these observers (such as `ReplicaState` inside `LeaderState`).
Reviewers: Ismael Juma <ismael@juma.me.uk>, Boyang Chen <boyang@confluent.io>
Rename AdminManager to ZkAdminManager to emphasize the fact that it is not used by the KIP-500 code paths.
Reviewers: Ismael Juma <ismael@juma.me.uk>, Boyang Chen <boyang@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
In the [KIP-500 development branch](https://github.com/confluentinc/kafka/tree/kip-500),
we have a separate ControllerApis that shares a lot of functionality with KafkaApis. We
introduced a utility class ApisUtils to pull out the common code. Some things were moved
to RequestChannel as well.
We'd like to upstream this work now so we don't continue to diverge (since KafkaApis is
a frequently modified class). There should be no logical changes in this PR, only shuffling
code around.
Reviewers: Jason Gustafson <jason@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>, Jose Sancio <jsancio@users.noreply.github.com>, Ismael Juma <ismael@juma.me.uk>
It is helpful to delay initialization of the `RaftClient` configuration including the voter string until after construction. This helps in integration test cases where the voter ports may not be known until sockets are bound.
Reviewers: José Armando García Sancio <jsancio@users.noreply.github.com>, Jason Gustafson <jason@confluent.io>
This patch removes the periodic scheduling of a thread to send AlterISR requests and instead sends a request as soon as an ISR update is ready to send. When a response is received, the callback checks for any unsent items and will schedule another request if necessary.
Prior to this patch, envelope handling was a shared responsibility between `SocketServer` and `KafkaApis`. The former was responsible for parsing and validation, while the latter was responsible for authorization. This patch consolidates logic in `KafkaApis` so that envelope requests follow the normal request flow.
Reviewers: David Jacot <djacot@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
This patch factors out a `RaftManager` class from `TestRaftServer` which will be needed when we integrate this layer into the server. This class encapsulates the logic to build `KafkaRaftClient` as well as its IO thread.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
This PR includes following changes.
1. @Test(expected = Exception.class) is replaced by assertThrows
2. remove reference to org.scalatest.Assertions
3. change the magic code from 1 to 2 for testAppendAtInvalidOffset to test ZSTD
4. rename testMaybeAddPartitionToTransactionXXXX to testNotReadyForSendXXX
5. increase maxBlockMs from 1s to 3s to avoid unexpected timeout from TransactionsTest#testTimeout
Reviewers: Ismael Juma <ismael@confluent.io>
When we truncate the log, the first unstable offset might become valid. On the other hand, the logic in `updateHighWatermarkMetadata` assumes that the first stable offset remains at a valid position. Since this method can be reached through either `updateLogStartOffset` or `updateLogEndOffset` in the truncation paths, we need to ensure that the first unstable offset first reflects the truncated state.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Jun Rao <junrao@gmail.com>
This patch fixes a bug when overriding a reassignment in `ReassignPartitionsCommand` due to the invalid assumption that adding replicas are not included in the full replica set. It also simplifies the logic to construct the move maps and makes some stylistic improvements.
Reviewers: Jason Gustafson <jason@confluent.io>
This is similar to config change logs for topic/broker. This will be useful for debugging any acl related issues.
Author: Manikumar Reddy <manikumar.reddy@gmail.com>
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>
Closes#9841 from omkreddy/acl-change-log
This patch improves append fencing after an IO error. Previously there was a window between the time of an IO error and the time the log is taken offline in which additional appends can be attempted. This is due to the asynchronous propagation of the IO error. This patch tightens the fencing so that no additional appends will be accepted after a previous append failed with an IO error.
Reviewers: Guozhang Wang <guozhang@apache.org>, Jason Gustafson <jason@confluent.io>
Current below log statements are not useful. This PR logs readable/masked configs during alterBrokerConfigs/alterTopicConfigs method call.
`[Admin Manager on Broker 1]: Updating topic test with new configuration kafka.server.KafkaConfigc9ba35e3`
Author: Manikumar Reddy <manikumar.reddy@gmail.com>
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>, Chia-Ping Tsai <chia7712@gmail.com>
Closes#9824 from omkreddy/admin-logs
Previously offline replicas were included as `null` in the array of replicas in `PartitionInfo` when populated by the `MetadataCache` for the purpose of the client quota callback. This patch instead initializes empty non-null nodes, which is consistent with how `PartitionInfo` is constructed by the clients in `MetadataResponse`.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
Implements the code necessary for the leader to response to fetch snapshot requests and for the follower to fetch snapshots. This API is described in more detail in KIP-630: https://cwiki.apache.org/confluence/display/KAFKA/KIP-630%3A+Kafka+Raft+Snapshot. More specifically, this patch includes the following changes:
Leader Changes:
1. Raft leader response to FetchSnapshot request by reading the local snapshot and sending the requested bytes in the response. This implementation currently copies the bytes to memory. This will be fixed in a future PR.
Follower Changes:
1. Raft followers will start fetching snapshot if the leader sends a Fetch response that includes a SnapshotId.
2. Raft followers send FetchSnapshot requests if there is a pending download. The same timer is used for both Fetch and FetchSnapshot requests.
3. Raft follower handle FetchSnapshot responses by comping the bytes to the pending SnapshotWriter. This implementation doesn't fix the replicated log after the snapshot has been downloaded. This will be implemented in a future PR.
Reviewers: Jason Gustafson <jason@confluent.io>
This patch contains the following improvements:
- Separate inbound/outbound request flows so that we can open the door for concurrent inbound request handling
- Rewrite `KafkaNetworkChannel` to use `InterBrokerSendThread` which fixes a number of bugs/shortcomings
- Get rid of a lot of boilerplate conversions in `KafkaNetworkChannel`
- Improve validation of inbound responses in `KafkaRaftClient` by checking correlationId. This fixes a bug which could cause an out of order Fetch to be applied incorrectly.
Reviewers: David Arthur <mumrah@gmail.com>
ISR-related cleanup in ReplicaManager and Partition. Removes ISR change logic from ReplicaManager and adds a new ZkIsrManager class which adheres to a new AlterIsrManager trait. Unifies all of the ISR logic in Partition so we don't have separate code paths for ZK vs AlterIsr. Also removes PartitionStateStore
This change propagates topic ids to brokers in LeaderAndIsr Request. It also removes the topic name from the LeaderAndIsr Response, reorganizes the response to be sorted by topic, and includes the topic ID.
In addition, the topic ID is persisted to each replica in Log as well as in a file on disk. This file is read on startup and if the topic ID exists, it will be reloaded.
Reviewers: David Jacot <djacot@confluent.io>, dengziming <dengziming1993@gmail.com>, Nikhil Bhatia <rite2nikhil@gmail.com>, Rajini Sivaram <rajinisivaram@googlemail.com>
Includes:
- Bump the version of MetadataRequest and MetadataResponse, add topicId in MetadataResponse
- Alter describeTopic in AdminClientTopicService and ZookeeperTopicService
- TopicMetadata is cached in MetadataCache, so we need to add topicId to MetadataCache
- MetadataCache is updated by UpdateMetadataRequest, bump the version of UpdateMetadataReq and UpdateMetadataResp, add topicId in UpdateMetadataReq.
Reviewers: Justine Olshan <jolshan@confluent.io>, Rajini Sivaram <rajinisivaram@googlemail.com>
Includes:
- New API to authorize by resource type
- Default implementation for the method that supports super users and ACLs
- Optimized implementation in AclAuthorizer that supports ACLs, super users and allow.everyone.if.no.acl.found
- Benchmarks and tests
- InitProducerIdRequest authorized for Cluster:IdempotentWrite or WRITE to any topic, ProduceRequest authorized only for topic even if idempotent
Reviewers: Lucas Bradstreet <lucas@confluent.io>, Rajini Sivaram <rajinisivaram@googlemail.com>
This patch follows up https://github.com/apache/kafka/pull/9547. It refactors AbstractFetcherThread and its descendants to use `OffsetForLeaderEpochRequestData.OffsetForLeaderPartition` instead of `OffsetsForLeaderEpochRequest.PartitionData`. The patch relies on existing tests.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Jason Gustafson <jason@confluent.io>
I wasn't able to reproduce the failure locally, but it looks like there is a race condition with the sending of the records in the first producer. The test case assumes that these records have been completed before the call to `sendOffsetsToTransaction`, but they very well might not be. It is even possible for the writes from the second producer to arrive first which would then result in the test failure that we are seeing. The solution is to force the send with `flush()`.
Reviewers: Guozhang Wang <guozhang@apache.org>, Boyang Chen <boyang@confluent.io>
* The naming for `ListOffsets` was inconsistent, in some places it was `ListOffset` and in others
it was `ListOffsets`. Picked the latter since it was used in metrics and the protocol documentation
and made it consistent.
* Removed unused methods in ApiKeys.
* Deleted `CommonFields`.
* Added `lowestSupportedVersion` and `highestSupportedVersion` to `ApiMessageType`
* Removed tests in `MessageTest` that are no longer relevant.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
The non-local return when `maybeHandleInvalidEnvelope` returns true causes the default error handler to execute after a response has already been sent. This patch rewrites the check as a local return.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
This patch updates the request logger to output request and response payloads in JSON. Payloads are converted to JSON based on their auto-generated schema.
Reviewers: Lucas Bradstreet <lucas@confluent.io>, David Mao <dmao@confluent.io>, David Jacot <djacot@confluent.io>
Set it as a cluster action and update the handler in KafkaApis. We keep the `throttleTimeMs` field
since we intend to enable throttling in the future (especially relevant when we switch to the
built-in quorum mode).
Reviewers: David Arthur <mumrah@gmail.com>
This PR adds support for IP entities to the `DescribeClientQuotas` and `AlterClientQuotas` APIs. This PR also adds support for describing/altering IP quotas via `kafka-configs` tooling.
Reviewers: Brian Byrne <bbyrne@confluent.io>, Anna Povzner <anna@confluent.io>, David Jacot <djacot@confluent.io>
FileChannel.close() does not guarantee modified buffer would be written on the file system. We are changing it with force() semantics to enforce file buffer and metadata written to filesystem (FileChannel.force(true) updates buffer and metadata).
Reviewers: Jun Rao <junrao@gmail.com>, Jason Gustafson <jason@confluent.io>
Connection id is now only present in `NetworkSend`, which is now
the class used by `Selector`/`NetworkClient`/`KafkaChannel` (which
works well since `NetworkReceive` is the class used for
received data).
The previous `NetworkSend` was also responsible for adding a size
prefix. This logic is already present in `SendBuilder`, but for the
minority of cases where `SendBuilder` is not used (including
a number of tests), we now have `ByteBufferSend.sizePrefixed()`.
With regards to the request/message utilities:
* Renamed `toByteBuffer`/`toBytes` in `MessageUtil` to
`toVersionPrefixedByteBuffer`/`toVersionPrefixedBytes` for clarity.
* Introduced new `MessageUtil.toByteBuffer` that does not include
the version as the prefix.
* Renamed `serializeBody` in `AbstractRequest/Response` to
`serialize` for symmetry with `parse`.
* Introduced `RequestTestUtils` and moved relevant methods from
`TestUtils`.
* Moved `serializeWithHeader` methods that were only used in
tests to `RequestTestUtils`.
* Deleted `MessageTestUtil`.
Finally, a couple of changes to simplify coding patterns:
* Added `flip()` and `buffer()` to `ByteBufferAccessor`.
* Added `MessageSizeAccumulator.sizeExcludingZeroCopy`.
* Used lambdas instead of `TestCondition`.
* Used `Arrays.copyOf` instead of `System.arraycopy` in `MessageUtil`.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Jason Gustafson <jason@confluent.io>
add total timeout for forwarding, including the underlying broker-to-controller channel timeout setting.
Reviewers: David Arthur <mumrah@gmail.com>, Jason Gustafson <jason@confluent.io>
We should configure a reconnect backoff for controller connections to prevent tight reconnect loops when the controller cannot be reached. I have borrowed the same configuration we use in `TransactionMarkerChannelManager`.
Reviewers: David Arthur <mumrah@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>, Boyang Chen <boyang@confluent.io>
The test case sends two records before killing broker. The failure is caused when both records are NOT sent in a single batch. The failure of first record can abort second batch and then produces `KafkaException` rather than `TimeoutException`. The patch removes the second record send.
Reviewers: Jason Gustafson <jason@confluent.io>
Generated request/response classes have code to serialize/deserialize directly to
`ByteBuffer` so the intermediate conversion to `Struct` can be skipped for them.
We have recently completed the transition to generated request/response classes,
so we can also remove the `Struct` based fallbacks.
Additional noteworthy changes:
* `AbstractRequest.parseRequest` has a more efficient computation of request size that
relies on the received buffer instead of the parsed `Struct`.
* Use `SendBuilder` for `AbstractRequest/Response` `toSend`, made the superclass
implementation final and removed the overrides that are no longer necessary.
* Removed request/response constructors that assume latest version as they are unsafe
outside of tests.
* Removed redundant version fields in requests/responses.
* Removed unnecessary work in `OffsetFetchResponse`'s constructor when version >= 2.
* Made `AbstractResponse.throttleTimeMs()` abstract.
* Using `toSend` in `SaslClientAuthenticator` instead of `serialize`.
* Various changes in Request/Response classes to make them more consistent and to
rely on the Data classes as much as possible when it comes to their state.
* Remove the version argument from `AbstractResponse.toString`.
* Fix `getErrorResponse` for `ProduceRequest` and `DescribeClientQuotasRequest` to
use `ApiError` which processes the error message sent back to the clients. This was
uncovered by an accidental fix to a `RequestResponseTest` test (it was calling
`AbstractResponse.toString` instead of `AbstractResponse.toString(short)`).
Rely on existing protocol tests to ensure this refactoring does not change
observed behavior (aside from improved performance).
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
This PR adds support for generating snapshot for KIP-630.
1. Adds the interfaces `RawSnapshotWriter` and `RawSnapshotReader` and the implementations `FileRawSnapshotWriter` and `FileRawSnapshotReader` respectively. These interfaces and implementations are low level API for writing and reading snapshots. They are internal to the Raft implementation and are not exposed to the users of `RaftClient`. They operation at the `Record` level. These types are exposed to the `RaftClient` through the `ReplicatedLog` interface.
2. Adds a buffered snapshot writer: `SnapshotWriter<T>`. This type is a higher-level type and it is exposed through the `RaftClient` interface. A future PR will add the related `SnapshotReader<T>`, which will be used by the state machine to load a snapshot.
Reviewers: Jason Gustafson <jason@confluent.io>
Add small interface to Partition.scala that allows AlterIsr and ZK code paths to update the ISR metrics managed by ReplicaManager. This opens the door for consolidating even more code between the two ISR update code paths.
This patch follows up https://github.com/apache/kafka/pull/9547. It refactors KafkaApis, ReplicaManager and Partition to use `OffsetForLeaderEpochResponseData.EpochEndOffset` instead of `EpochEndOffset`. In the mean time, it removes `OffsetsForLeaderEpochRequest#epochsByTopicPartition` and `OffsetsForLeaderEpochResponse#responses` and replaces their usages to use the automated protocol directly. Finally, it removes old constructors in `OffsetsForLeaderEpochResponse`. The patch relies on existing tests.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Jason Gustafson <jason@confluent.io>
From IBP 2.7 onwards, fetch responses include diverging epoch and offset in fetch responses if lastFetchedEpoch is provided in the fetch request. This PR uses that information for truncation and avoids the additional OffsetForLeaderEpoch requests in followers when lastFetchedEpoch is known.
Co-authored-by: Jason Gustafson <jason@confluent.io>
Reviewers: Jason Gustafson <jason@confluent.io>, Nikhil Bhatia <rite2nikhil@gmail.com>
As a follow-up from [KIP-482](https://cwiki.apache.org/confluence/display/KAFKA/KIP-482%3A+The+Kafka+Protocol+should+Support+Optional+Tagged+Fields), this PR bumps the version for several
RPC's to enable tagged fields via the flexible versioning mechanism.
Additionally, a new IBP version `KAFKA_2_8_IV0` is introduced to
allow replication to take advantage of these new RPC versions for
OffsetForLeaderEpoch and ListOffset.
Reviewers: Ismael Juma <ismael@juma.me.uk>, Jason Gustafson <jason@confluent.io>
A few small cleanups in `Partition` handling of `AlterIsr`:
- Factor state update and log message into `sendAlterIsrRequest`
- Ensure illegal state error gets raised if a retry fails to be enqueued
- Always check the proposed state against the current state in `handleAlterIsrResponse`
- Add `toString` implementations to `IsrState` case classes
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
Compacted topics can accumulate a large number of empty transaction markers as the data from the transactions gets cleaned. For each transaction, there is some bookkeeping that leaders and followers must do to keep the transaction index up to date. The cost of this overhead can degrade performance when a replica needs to catch up if the log has mostly empty or small transactions. This patch improves the cost by skipping over empty transactions since these will have no effect on the last stable offset and do not need to be reflected in the transaction index.
Reviewers: Lucas Bradstreet <lucas@confluent.io>, Jun Rao <junrao@gmail.com>
Scala 2.13.4 restores default global `ExecutionContext` to 2.12 behavior
(to fix a perf regression in some use cases) and improves pattern matching
(especially exhaustiveness checking). Most of the changes are related
to the latter as I have enabled the newly introduced `-Xlint:strict-unsealed-patmat`.
More details on the code changes:
* Don't swallow exception in `ReassignPartitionsCommand.topicDescriptionFutureToState`.
* `RequestChannel.Response` should be `sealed`.
* Introduce sealed ClientQuotaManager.BaseUserEntity to avoid false positive
exhaustiveness warning.
* Handle a number of cases where pattern matches were not exhaustive:
either by marking them with @unchecked or by adding a catch-all clause.
* Workaround scalac bug related to exhaustiveness warnings in ZooKeeperClient
* Remove warning suppression annotations related to the optimizer that are no
longer needed in ConsumerGroupCommand and AclAuthorizer.
* Use `forKeyValue` in `AclAuthorizer.acls` as the scala bug preventing us from
using it seems to be fixed.
* Also update scalaCollectionCompat to 2.3.0, which includes minor improvements.
Full release notes:
* https://github.com/scala/scala/releases/tag/v2.13.4
* https://github.com/scala/scala-collection-compat/releases/tag/v2.3.0
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
We use a background thread for Kerberos to perform re-login before tickets expire. The thread performs logout() followed by login(), relying on the Java library to clear and then populate credentials in Subject. This leaves a timing window where clients fail to authenticate because credentials are not available. We cannot introduce any form of locking since authentication is performed on the network thread. So this commit treats NO_CRED as a transient failure rather than a fatal authentication exception in clients.
Reviewers: Ron Dagostino <rdagostino@confluent.io>, Manikumar Reddy <manikumar.reddy@gmail.com>
This patch fixes a liveness bug which prevents follower truncation from completing after a leader election. If there are consecutive leader elections without writing any data entries, then the leader and follower may have conflicting epoch entries at the end of the log.
The problem is the shortcut return in `Log.truncateTo` when the truncation offset is larger than or equal to the end offset, which prevents the conflicting entries from being resolved. Here we change this case to ensure `LeaderEpochFileCache.truncateFromEnd` is still called.
Reviewers: Jun Rao <junrao@gmail.com>