Commit Graph

3325 Commits

Author SHA1 Message Date
José Armando García Sancio 5b3351e10b
KAFKA-10761; Kafka Raft update log start offset (#9816)
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>
2021-01-29 14:06:01 -08:00
Justine Olshan 4f588f7ca2
KAFKA-10764: Add support for returning topic IDs on create, supplying topic IDs for delete (#9684)
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>
2021-01-29 19:40:16 +00:00
Jason Gustafson d87fc21763
MINOR: Ensure `InterBrokerSendThread` closes `NetworkClient` (#9999)
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>
2021-01-29 11:37:43 -08:00
Lee Dongjin e3ff4b0870
KAFKA-10604: Fix Streams default state.dir (#9420)
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>
2021-01-29 09:45:13 -06:00
Alok Nikhil 27a998e8a0
KAFKA-12237; Support lazy initialization of quorum voter addresses (#9985)
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>
2021-01-28 17:14:56 -08:00
John Roesler fdcf8fbf72
KAFKA-10866: Add metadata to ConsumerRecords (#9836)
Expose fetched metadata via the ConsumerRecords
object as described in KIP-695.

Reviewers: Guozhang Wang <guozhang@apache.org>
2021-01-27 18:18:38 -06:00
Kowshik Prakasam 4075a5c8e8
MINOR: Call logSegments.toBuffer only when required (#9971)
Reviewers: Ismael Juma <ismael@juma.me.uk>, Satish Duggana <satishd@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>
2021-01-27 12:43:10 +08:00
Kowshik Prakasam 7230d148f4
MINOR: Fix visibility of Log.{unflushedMessages, addSegment} methods (#9966)
Reviewers: Jun Rao <junrao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2021-01-26 13:59:33 +08:00
dengziming f4c2030b20
MINOR: Optimize assertions in unit tests (#9955)
Reviewers: David Jacot <djacot@confluent.io>
2021-01-24 08:47:37 +01:00
Jason Gustafson 6d411620dd
MINOR: A few small group coordinator cleanups (#9952)
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>
2021-01-23 13:07:31 -08:00
David Jacot 7a1d1d9a69
KAFKA-12212; Bump Metadata API version to remove `ClusterAuthorizedOperations` fields (KIP-700) (#9945)
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>
2021-01-22 09:06:37 +01:00
Jason Gustafson 9689a313f5
MINOR: Drop enable.metadata.quorum config (#9934)
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>
2021-01-21 15:16:15 -08:00
Alok Nikhil fea2f65929
MINOR: Import RaftConfig config definitions into KafkaConfig (#9916)
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>
2021-01-21 10:26:23 -08:00
David Jacot 24e7e81524
MINOR: Refactor DescribeAuthorizedOperationsTest (#9938)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2021-01-21 17:34:08 +01:00
17hao 62860fa0a1
MINOR: Remind user index file is empty when dumping LogSegment index file (#9891)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2021-01-21 12:06:35 +08:00
Justine Olshan 86b9fdef2b
KAFKA-10869: Gate topic IDs behind IBP 2.8 (KIP-516) (#9814)
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>
2021-01-20 22:32:06 +00:00
Luke Chen 918c332cbf
KAFKA-8460: produce records with current timestamp (#9877)
Reviewers: Ismael Juma <ismael@juma.me.uk>, Chia-Ping Tsai <chia7712@gmail.com>
2021-01-20 15:48:44 +08:00
David Jacot 302eee63c4
KAFKA-12204; Implement DescribeCluster API in the broker (KIP-700) (#9903)
This PR implements the DescribeCluster API in the broker.

Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>, Ismael Juma <ismael@juma.me.uk>
2021-01-19 18:43:36 +01:00
Chia-Ping Tsai f7c0b0d172
MINOR: Replace ApiVersion by auto-generated protocol (#9746)
Reviewers: Ismael Juma <ismael@juma.me.uk>
2021-01-20 00:37:45 +08:00
Geordie efff9db827
KAFKA-10841: Extract conversion from LogReadResult to FetchPartitionData (#9743)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Ismael Juma <ismael@juma.me.uk>
2021-01-18 16:55:21 -08:00
Chia-Ping Tsai 5a4bbb9f91
KAFKA-7341 Migrate core module to JUnit 5 (#9855)
Reviewers: Ismael Juma <ismael@juma.me.uk>
2021-01-19 00:00:32 +08:00
Ismael Juma 6f9e73cfd8
MINOR: Move a few more methods to AuthHelper (#9913)
And move some tests to `AuthHelperTest`.

Reviewers: David Arthur <mumrah@gmail.com>
2021-01-16 06:44:02 -08:00
Boyang Chen bfc96efa3a
KAFKA-10674: Controller API version bond with forwardable APIs (#9600)
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>
2021-01-15 20:36:25 -08:00
Jason Gustafson 8f063c15da
MINOR: Generalize server startup to make way for KIP-500 (#9883)
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>
2021-01-15 17:30:29 -08:00
Jason Gustafson 7ac06065f1
KAFKA-12161; Support raft observers with optional id (#9871)
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>
2021-01-15 14:10:17 -08:00
Colin Patrick McCabe b5c107363f
KAFKA-12208: Rename AdminManager to ZkAdminManager (#9900)
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>
2021-01-15 12:56:09 -08:00
David Arthur ba1e16f0c0
MINOR: Upstream ApisUtils from kip-500 (#9715)
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>
2021-01-15 12:28:43 -08:00
Chia-Ping Tsai 9361a43cfa
MINOR: remove unused flag 'hasIdempotentRecords' (#9884)
Reviewers: Ismael Juma <ismael@juma.me.uk>
2021-01-16 00:53:14 +08:00
Alok Nikhil c49f660c62
MINOR: Initialize QuorumState lazily in RaftClient.initialize() (#9881)
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>
2021-01-14 17:31:54 -08:00
wenbingshen 4a985b594f
MINOR: fix typo in TimeIndex (#9834)
fix typos in TimeIndex

Co-authored-by: wenbingshen <oliver.shen999@gmail.com>
Reviewers: Boyang Chen <boyang@confluent.io>
2021-01-14 13:25:48 -08:00
Luke Chen 2746af4535
MINOR: replace NotLeaderForPartitionException with NotLeaderOrFollowerException (#9885)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Ismael Juma <ismael@juma.me.uk>
2021-01-14 12:16:16 +00:00
David Arthur ee08b0b32a
Only schedule AlterIsr thread when we have an ISR change (#9749)
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.
2021-01-13 13:52:38 -05:00
Jason Gustafson 474ad1eebd
KAFKA-12168; Move envelope request parsing out of SocketServer (#9850)
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>
2021-01-12 13:32:43 -08:00
Jason Gustafson f62c2b26cc
MINOR: Factor `RaftManager` out of `TestRaftServer` (#9839)
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>
2021-01-11 09:28:12 -08:00
dengziming 119a2d9127
MINOR: Substitute assertEquals(null) with assertNull (#9852)
Reviewers: David Jacot <djacot@confluent.io>
2021-01-10 20:06:37 +01:00
Chia-Ping Tsai 913a019d6c
MINOR: replace test "expected" parameter by assertThrows (#9520)
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>
2021-01-10 20:20:13 +08:00
Jason Gustafson ea459d1457
KAFKA-12153; Update producer state before updating start/end offsets after truncation (#9838)
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>
2021-01-08 17:38:40 -08:00
David Jacot 8716ba1ff1
MINOR; Add producer id in exceptions thrown by ProducerStateManager (#9827)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Jason Gustafson <jason@confluent.io>
2021-01-08 09:46:29 +01:00
dengziming 52652a0dca
KAFKA-10779; Reassignment tool sets throttles incorrectly when overriding a reassignment (#9807)
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>
2021-01-07 15:11:21 -08:00
Manikumar Reddy f0ef5785e3 MINOR: Add a log to print acl change notification details
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
2021-01-07 16:32:22 +05:30
Geordie 71540c03b8
KAFKA-10874: Fix flaky ClientQuotasRequestTest.testAlterIpQuotasRequest (#9778)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, David Jacot <djacot@confluent.io>
2021-01-07 09:46:07 +01:00
Tom Bentley bf55afecda
KAFKA-10778; Fence appends after write failure (#9676)
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>
2021-01-06 10:06:56 -08:00
dengziming df9fc3834c
MINOR: Combine repeated top-level error test in AlterIsrManagerTest (#9649)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, David Arthur <mumrah@gmail.com>
2021-01-06 11:13:51 -05:00
Manikumar Reddy ea2423e0e3 MINOR: Update log statements in alterBrokerConfigs/alterTopicConfigs methods
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
2021-01-06 19:59:10 +05:30
Chia-Ping Tsai b80cf9c240
MINOR: make sure all dir jobs are completed (#9728)
Reviewers: Ismael Juma <ismael@juma.me.uk>
2021-01-06 14:52:15 +08:00
Jason Gustafson daad1cbf55
KAFKA-10894; Ensure PartitionInfo replicas are not null in client quota callback (#9802)
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>
2021-01-05 13:02:55 -08:00
wenbingshen 0eea5845c5
rename produce to produces (#9811)
Co-authored-by: 沈文兵 <shenwenbing@qianxin.com>

reviewers: Boyang Chen <boyang@confluent.io>
2021-01-03 21:20:48 -08:00
Attila Sasvari 2515bf2368
MINOR: fix error message in TestRaftServer.scala (#9812)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2021-01-03 21:11:31 +08:00
José Armando García Sancio 2023aed59d
KAFKA-10427: Fetch snapshot API (#9553)
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>
2020-12-28 18:37:08 -08:00
Jason Gustafson eb9fe411bb
KAFKA-10842; Use `InterBrokerSendThread` for raft's outbound network channel (#9732)
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>
2020-12-21 18:15:15 -08:00
David Arthur d3f19e4bb0
KAFKA-10825 ZooKeeper ISR manager (#9713)
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
2020-12-21 14:44:02 -05:00
Chia-Ping Tsai 4e7c789118
MINOR: refactor SelectingIterator by scala iterator (#9755)
Reviewers: Ismael Juma <ismael@juma.me.uk>
2020-12-21 16:11:51 +08:00
Chia-Ping Tsai 87260a33b0
HOTFIX: fix failed ControllerChannelManagerTest#testUpdateMetadataRequestSent (#9768)
Reviewers: David Jacot <djacot@confluent.io>
2020-12-21 12:29:10 +08:00
Justine Olshan 1dd1e7f945
KAFKA-10545: Create topic IDs and propagate to brokers (#9626)
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>
2020-12-18 22:19:50 +00:00
dengziming 5c921afa4a
KAFKA-10547; Add TopicId in MetadataResponse (#9622)
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>
2020-12-18 21:30:52 +00:00
Cheng Tan ae3a6ed990
KAKFA-10619: Idempotent producer will get authorized once it has a WRITE access to at least one topic (KIP-679) (#9485)
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>
2020-12-18 18:08:46 +00:00
Govinda Sakhare f8b26563e1
KAFKA-10850: Use primitive type to replace deprecated 'new Integer' from BrokerToControllerRequestThreadTest (#9760)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2020-12-18 00:43:21 +08:00
David Jacot 02a30a51eb
KAFKA-10740; Replace OffsetsForLeaderEpochRequest.PartitionData with automated protocol (#9689)
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>
2020-12-17 17:40:37 +01:00
Jason Gustafson 4089c36b33
KAFKA-10861; Fix race condition in flaky test `testFencingOnSendOffsets` (#9762)
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>
2020-12-16 18:31:15 -08:00
Ismael Juma 782175dfbc
MINOR: Simplify ApiKeys by relying on ApiMessageType (#9748)
* 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>
2020-12-16 06:33:10 -08:00
Jason Gustafson d072be6d90
KAFKA-10855; Fix non-local return in `KafkaApis.handle` (#9753)
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>
2020-12-15 13:07:48 -08:00
Viktor Somogyi-Vass 1aac64667f
KAFKA-6084: Propagate JSON parsing errors in ReassignPartitionsCommand (#4090)
Reviewers: Ismael Juma <ismael@juma.me.uk>
2020-12-15 10:05:40 -08:00
Luke Chen a07923c4f2
KAFKA-10854: fix flaky testConnectionRatePerIp test (#9752)
Reviewers: David Jacot <djacot@confluent.io>
2020-12-15 14:54:22 +01:00
Anastasia Vela 1a10c3445e
KAFKA-10525: Emit JSONs with new auto-generated schema (KIP-673) (#9526)
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>
2020-12-15 14:33:36 +01:00
Ismael Juma 5e5daf47ef
KAFKA-10852: AlterIsr should not be throttled (#9747)
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>
2020-12-14 22:28:47 -08:00
Chia-Ping Tsai 78b74debe4
Revert "MINOR: fix typo in `AbstractIndex.scala` (#9745)" (#9751)
This reverts commit 8577ceae87.
2020-12-15 08:39:58 +08:00
panguncle 8577ceae87
MINOR: fix typo in `AbstractIndex.scala` (#9745)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2020-12-14 15:34:10 +08:00
Kowshik Prakasam 4401f52d17
MINOR: Make Log.recordVersion private and other small cleanups (#9731)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2020-12-14 15:20:20 +08:00
Kowshik Prakasam cdf725828b
KAFKA-10832: Fix Log to use the correct ProducerStateManager instance when updating producers (#9718)
Reviewers: Ismael Juma <ismael@juma.me.uk>, Jun Rao <junrao@gmail.com>
2020-12-11 16:34:46 -08:00
David Mao b44d32dffe
KAFKA-10748: Add IP connection rate throttling metric (KIP-612) (#9685)
This PR adds the IP throttling metric as described in KIP-612.

Reviewers: Anna Povzner <anna@confluent.io>, David Jacot <djacot@confluent.io>
2020-12-10 12:24:45 +01:00
David Mao 404062d2b6
KAFKA-10747: Extend DescribeClientQuotas and AlterClientQuotas APIs to support IP connection rate quota (KIP-612) (#9628)
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>
2020-12-10 09:53:32 +01:00
dengziming 8e82eaa711
MINOR: Fix some java docs of ReplicaStateMachine (#8552)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2020-12-10 11:14:35 +08:00
bertber db79f86025
MINOR: remove duplicate code from resetByDuration (#9699)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2020-12-10 11:08:17 +08:00
Brajesh Kumar fa93982d3a
KAFKA-9892; Producer state snapshot should be forced to disk (#9621)
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>
2020-12-09 14:13:36 -08:00
Ismael Juma 1f98112e99
MINOR: Remove connection id from Send and consolidate request/message utils (#9714)
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>
2020-12-09 11:15:58 -08:00
Lincong Li ff88874e0d
KAFKA-10606: Disable auto topic creation for fetch-all-topic-metadata request (#9435)
Reviewers: Ismael Juma <ismael@juma.me.uk>, Chia-Ping Tsai <chia7712@gmail.com>
2020-12-09 23:31:04 +08:00
Boyang Chen 41ea0775e0
KAFKA-10667: add timeout for forwarding requests (#9564)
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>
2020-12-08 11:12:30 -08:00
Jason Gustafson 153bbb8ac0
MINOR: Configure reconnect backoff in `BrokerToControllerChannelManager` (#9709)
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>
2020-12-08 09:19:30 -08:00
Chia-Ping Tsai aebb0e3394
KAFKA-10264; Fix Flaky Test TransactionsTest.testBumpTransactionalEpoch (#9291)
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>
2020-12-08 09:13:32 -08:00
Kowshik Prakasam 1d84f54367
MINOR: Remove redundant default parameter values in call to LogSegment.open (#9710)
Reviewers: Jun Rao <junrao@gmail.com>
2020-12-07 16:42:19 -08:00
Ismael Juma 6f27bb02da
KAFKA-10818: Skip conversion to `Struct` when serializing generated requests/responses (#7409)
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>
2020-12-07 15:39:57 -08:00
José Armando García Sancio ab0807dd85
KAFKA-10394: Add classes to read and write snapshot for KIP-630 (#9512)
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>
2020-12-07 14:06:25 -08:00
APaMio df0c52e7fd
MINOR: a small refactor for LogManage#shutdown (#9680)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2020-12-04 21:48:01 +08:00
Prateek Agarwal 155f2c06fb
KAFKA-10803: Fix improper removal of bad dynamic config (#9682)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2020-12-04 11:59:39 +08:00
David Arthur 633f7cff19
KAFKA-10799 AlterIsr utilizes ReplicaManager ISR metrics (#9677)
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.
2020-12-03 16:11:07 -05:00
David Jacot 10364e4b0c
KAFKA-10739; Replace EpochEndOffset with automated protocol (#9630)
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>
2020-12-03 18:50:29 +01:00
Ankit Kumar 9de16bd2e6
KAFKA-10460: ReplicaListValidator format checking is incomplete (#9326)
Co-authored-by: akumar <akumar@cloudera.com>
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Viktor Somogyi-Vass <viktorsomogyi@gmail.com>
2020-12-03 14:36:48 +00:00
Rajini Sivaram 7ecc3a579a
KAFKA-10554; Perform follower truncation based on diverging epochs in Fetch response (#9382)
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>
2020-12-03 10:12:06 +00:00
Montyleo 10b07575f1
KAFKA-10794 Replica leader election is too slow in the case of too many partitions (#9675)
Co-authored-by: limengmonty <limengmonty@didichuxing.com>
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2020-12-03 10:38:27 +08:00
Chia-Ping Tsai aeeb7b2f9a
KAFKA-9263 The new hw is added to incorrect log when ReplicaAlterLogDirsThread is replacing log (fix PlaintextAdminIntegrationTest.testAlterReplicaLogDirs) (#9423)
Reviewers: Jun Rao <junrao@gmail.com>
2020-12-02 11:21:28 +08:00
Gardner Vickers 85f94d5027
KAFKA-10729; Bump remaining RPC's to use tagged fields. (#9601)
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>
2020-12-01 15:55:07 -08:00
Jason Gustafson 8839514efb
MINOR: Small cleanups in `AlterIsr` handling logic (#9663)
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>
2020-12-01 11:17:08 -08:00
Jason Gustafson e7de280b0f
KAFKA-10702; Skip bookkeeping of empty transactions (#9632)
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>
2020-11-30 14:48:28 -08:00
Chia-Ping Tsai 0a74c7d935
KAFKA-10736 Convert transaction coordinator metadata schemas to use g… (#9611)
Reviewers: David Jacot <djacot@confluent.io>
2020-11-30 17:43:42 +08:00
Tom Bentley 6b1c8f921d
KAFKA-10565: Only print console producer prompt with a tty (#9644)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2020-11-26 11:57:39 +08:00
Tom Bentley 7a49326822
MINOR: fix listeners doc to close <code> properly (#9655)
Fixes a problem with an incorrectly closed <code> in the doc for listeners.

Reviewers: Bill Bejeck <bbejeck@apache.org>
2020-11-25 14:00:02 -05:00
Ismael Juma cbf8ad277a
MINOR: Upgrade to Scala 2.13.4 (#9643)
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>
2020-11-24 10:28:53 -08:00
Rajini Sivaram ed8659b4a0
KAFKA-10727; Handle Kerberos error during re-login as transient failure in clients (#9605)
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>
2020-11-23 09:04:16 +00:00
dengziming 9fdea8a6b6
MINOR: Wrong command line suggestion in kafka.Kafka (#9637)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2020-11-23 12:02:18 +08:00
Jason Gustafson bc96a8feb5
KAFKA-10706; Ensure leader epoch cache is cleaned after truncation to end offset (#9633)
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>
2020-11-21 09:25:54 -08:00