Commit Graph

3325 Commits

Author SHA1 Message Date
David Jacot bbf145b1b1
KAFKA-10817; Add clusterId validation to raft Fetch handling (#10129)
This patch adds clusterId validation in the `Fetch` API as documented in KIP-595. A new error code `INCONSISTENT_CLUSTER_ID` is returned if the request clusterId does not match the value on the server. If no clusterId is provided, the request is treated as valid.

Reviewers: Jason Gustafson <jason@confluent.io>
2021-02-19 14:43:14 -08:00
Justine Olshan d030dc55ab
KAFKA-12332; Error partitions from topics with invalid IDs in LISR requests (#10143)
Changes how invalid IDs are handled in LeaderAndIsr requests. The ID check now occurs before leader epoch. If the ID exists and is invalid, the partition is ignored and a new `INCONSISTENT_TOPIC_ID` error is returned in the response.

Reviewers: Jason Gustafson <jason@confluent.io>
2021-02-19 11:08:00 -08:00
Ron Dagostino 76de61475b MINOR: Fix Raft broker restart issue when offset partitions are deferred #10155
A Raft-based broker is unable to restart if the broker defers partition
metadata changes for a __consumer_offsets topic-partition. The issue is
that GroupMetadataManager is asked to removeGroupsForPartition() upon
the broker becoming a follower, but in order for that code to function
it requires that the manager's scheduler be started. There are multiple
possible solutions here since removeGroupsForPartition() is a no-op at
this point in the broker startup cycle (nothing has been loaded, so
there is nothing to unload). We could just not invoke the callback. But
it seems more reasonable to not special-case this and instead start
ReplicaManager and the coordinators just before applying the deferred
partitions states.

We also mark deferred partitions for which we are a follower as being
online a bit earlier to avoid NotLeaderOrFollowerException that was
being thrown upon restart. Fixing this issue exposed the above issue
regarding the scheduler not being started.

Reviewers: Colin P. McCabe <cmccabe@apache.org>, Ismael Juma <ismael@juma.me.uk>
2021-02-19 10:45:20 -08:00
José Armando García Sancio e29f7a36db
KAFKA-12331: Use LEO for the base offset of LeaderChangeMessage batch (#10138)
The `KafkaMetadataLog` implementation of `ReplicatedLog` validates that batches appended using `appendAsLeader` and `appendAsFollower` have an offset that matches the LEO. This is enforced by `KafkaRaftClient` and `BatchAccumulator`. When creating control batches for the `LeaderChangeMessage` the default base offset of `0` was being used instead of using the LEO. This is fixed by:

1. Changing the implementation for `MockLog` to validate against this and throw an `RuntimeException` if this invariant is violated.
2. Always create a batch for `LeaderChangeMessage` with an offset equal to the LEO.

Reviewers: Jason Gustafson <jason@confluent.io>
2021-02-18 16:44:40 -08:00
Jason Gustafson 698319b8e2 KAFKA-12278; Ensure exposed api versions are consistent within listener (#10666)
Previously all APIs were accessible on every listener exposed by the broker, but
with KIP-500, that is no longer true.  We now have more complex requirements for
API accessibility.

For example, the KIP-500 controller exposes some APIs which are not exposed by
brokers, such as BrokerHeartbeatRequest, and does not expose most client APIs,
such as JoinGroupRequest, etc.  Similarly, the KIP-500 broker does not implement
some APIs that the ZK-based broker does, such as LeaderAndIsrRequest and
UpdateFeaturesRequest.

All of this means that we need more sophistication in how we expose APIs and
keep them consistent with the ApiVersions API. Up until now, we have been
working around this using the controllerOnly flag inside ApiKeys, but this is
not rich enough to support all of the cases listed above.  This PR introduces a
new "listeners" field to the request schema definitions.  This field is an array
of strings which indicate the listener types in which the API should be exposed.
We currently support "zkBroker", "broker", and "controller".  ("broker"
indicates the KIP-500 broker, whereas zkBroker indicates the old broker).

This PR also creates ApiVersionManager to encapsulate the creation of the
ApiVersionsResponse based on the listener type.  Additionally, it modifies
SocketServer to check the listener type of received requests before forwarding
them to the request handler.

Finally, this PR also fixes a bug in the handling of the ApiVersionsResponse
prior to authentication. Previously a static response was sent, which means that
changes to features would not get reflected. This also meant that the logic to
ensure that only the intersection of version ranges supported by the controller
would get exposed did not work. I think this is important because some clients
rely on the initial pre-authenticated ApiVersions response rather than doing a
second round after authentication as the Java client does.

One final cleanup note: I have removed the expectation that envelope requests
are only allowed on "privileged" listeners.  This made sense initially because
we expected to use forwarding before the KIP-500 controller was available. That
is not the case anymore and we expect the Envelope API to only be exposed on the
controller listener. I have nevertheless preserved the existing workarounds to
allow verification of the forwarding behavior in integration testing.

Reviewers: Colin P. McCabe <cmccabe@apache.org>, Ismael Juma <ismael@juma.me.uk>
2021-02-18 16:25:51 -08:00
Chia-Ping Tsai 97c9ae119a
HOTFIX: Fix build error caused by ControllerApisTest.scala (#10146)
Introduced by #10113. The error was:

```
14:01:06 > Task :core:compileTestScala
14:01:06 [Error] /home/jenkins/agent/workspace/LoopTest-Kafka/kafka/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala:70: the result type of an implicit conversion must be more specific than Object
```

Reviewers: Ismael Juma <ismael@juma.me.uk
2021-02-18 13:47:14 -08:00
Ron Dagostino a30f92bf59
MINOR: Add KIP-500 BrokerServer and ControllerServer (#10113)
This PR adds the KIP-500 BrokerServer and ControllerServer classes and 
makes some related changes to get them working.  Note that the ControllerServer 
does not instantiate a QuorumController object yet, since that will be added in
PR #10070.

* Add BrokerServer and ControllerServer

* Change ApiVersions#computeMaxUsableProduceMagic so that it can handle
endpoints which do not support PRODUCE (such as KIP-500 controller nodes)

* KafkaAdminClientTest: fix some lingering references to decommissionBroker
that should be references to unregisterBroker.

* Make some changes to allow SocketServer to be used by ControllerServer as
we as by the broker.

* We now return a random active Broker ID as the Controller ID in
MetadataResponse for the Raft-based case as per KIP-590.

* Add the RaftControllerNodeProvider

* Add EnvelopeUtils

* Add MetaLogRaftShim

* In ducktape, in config_property.py: use a KIP-500 compatible cluster ID.

Reviewers: Colin P. McCabe <cmccabe@apache.org>, David Arthur <mumrah@gmail.com>
2021-02-17 21:35:13 -08:00
runom 0bdb469b56
MINOR: Fix typo (thread -> threads) in MirrorMaker (#10130)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2021-02-17 12:19:20 +08:00
Chia-Ping Tsai f5c2f608b0
MINOR: use 'mapKey' to avoid unnecessary grouped data (#10082)
1. add 'mapKey=true' to DescribeLogDirsRequest
2. rename PartitionIndex to Partitions for DescribeLogDirsRequest
3. add 'mapKey=true' to ElectLeadersRequest
4. rename PartitionId to Partitions for ElectLeadersRequest
5. add 'mapKey=true' to ConsumerProtocolAssignment

Reviewers: David Jacot <djacot@confluent.io>, Ismael Juma <ismael@juma.me.uk>
2021-02-17 12:14:08 +08:00
Justine Olshan fb7da1a245
Fixed README and added clearer error message. (#10133)
The script `test-raft-server-start.sh` requires the config to be specified with `--config`. I've included this in the README and added an error message for this specific case.

Reviewers: Jason Gustafson <jason@confluent.io>
2021-02-16 15:12:29 -08:00
Ismael Juma 744d05b128
KAFKA-12327: Remove MethodHandle usage in CompressionType (#10123)
We don't really need it and it causes problems in older Android versions
and GraalVM native image usage (there are workarounds for the latter).

Move the logic to separate classes that are only invoked when the
relevant compression library is actually used. Place such classes
in their own package and enforce via checkstyle that only these
classes refer to compression library packages.

To avoid cyclic dependencies, moved `BufferSupplier` to the `utils`
package.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2021-02-14 08:12:25 -08:00
Colin Patrick McCabe 1adb580faa
MINOR: KIP-631 KafkaConfig fixes and improvements (#10114)
Add the new KIP-631 configs to KafkaConfigTest to fix the test failure.

Rename InitialBrokerRegistrationTimeoutMs to
InitialBrokerRegistrationTimeoutMsProp for consistency with the other
properties.

Add ControllerListenerNamesProp as specified in KIP-631.

Give nodeId and brokerId the same value in KafkaConfig.

Reviewers: David Arthur <mumrah@gmail.com
2021-02-11 21:35:24 -08:00
David Arthur 25555b89f5
MINOR: Add BrokerMetadataListener (#10111)
This adds BrokerMetadataListener which is responsible for processing metadata records received by the broker when running in Raft mode.

This also moves some classes that were added to the wrong folder in trunk

Reviewers: Colin P. McCabe <cmccabe@apache.org>, Ron Dagostino <rdagostino@confluent.io>
2021-02-11 12:43:21 -08:00
Colin Patrick McCabe 6b3a4553f9
MINOR: Introduce the KIP-500 Broker lifecycle manager (#10095)
Add the KIP-500 broker lifecycle manager.  It owns the broker state.  Its inputs are
messages passed in from other parts of the broker and from the controller: requests to start
up, or shut down, for example. Its output are the broker state and various futures that can
be used to wait for broker state transitions to occur.

The lifecycle manager handles registering the broker with the controller, as described in
KIP-631. After registration is complete, it handles sending periodic broker heartbeats and
processing the responses.

Reviewers: David Arthur <mumrah@gmail.com>, Ismael Juma <ismael@juma.me.uk>, Ron Dagostino <rdagostino@confluent.io>
2021-02-11 08:32:38 -08:00
Dániel Urbán 202ff6336f
KAFKA-5235: GetOffsetShell: Support for multiple topics and consumer configuration override (KIP-635) (#9430)
This patch implements KIP-635 which mainly adds support for querying offsets of multiple topics/partitions.

Reviewers: David Jacot <djacot@confluent.io>
2021-02-11 12:06:21 +01:00
Justine Olshan 39dcdeffd7
MINOR: Prevent creating partition.metadata until ID can be written (#10041)
Currently the partition.metadata file is created when the log is created. However, clusters with older inter-broker protocols will never use this file. This PR moves the creation of the file to when we write to the file.

This PR also deletes the partition.metadata file on startup if the IBP version is lower than 2.8.

Reviewers: Jun Rao <junrao@gmail.com>
2021-02-10 15:22:15 -08:00
Ron Dagostino 31a647f5ad
MINOR: Add RaftReplicaManager (#10069)
This adds the logic to apply partition metadata when consuming from the Raft-based
metadata log.

RaftReplicaManager extends ReplicaManager for now to minimize changes to existing
code for the 2.8 release. We will likely adjust this hierarchy at a later time (e.g. introducing
a trait and adding a helper to refactor common code). For now, we expose the necessary
fields and methods in ReplicaManager by changing their scope from private to protected,
and we refactor out a couple of pieces of logic that are shared between the two
implementation (stopping replicas and adding log dir fetchers).

Reviewers: Colin P. McCabe <cmccabe@apache.org>, Ismael Juma <ismael@juma.me.uk>
2021-02-10 14:37:16 -08:00
David Arthur 3a3af5e200
MINOR: Add ClientQuotaMetadataManager for processing QuotaRecord (#10101)
This PR brings in the new broker metadata processor for handling QuotaRecord-s coming from the metadata log. Also included is a new cache class to allow for fast lookups of quotas on the broker for handling DescribeClientQuotaRequest.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2021-02-10 14:04:22 -08:00
Colin Patrick McCabe 06dce721ec
MINOR: Rename DecommissionBrokers to UnregisterBrokers (#10084)
Rename DecommissionBrokers to UnregisterBrokers. Fix an incorrect JavaDoc comment
for the Admin API. Make sure that UNREGISTER_BROKER is marked as forwardable and
not as a controller-only API (since it can received by brokers).

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Ismael Juma <ismael@juma.me.uk>
2021-02-10 12:44:47 -08:00
Ismael Juma ad541b9759
MINOR: KafkaBroker.brokerState should be volatile instead of AtomicReference (#10080)
We don't need or use the additional functionality provided by
AtomicReference.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, David Arthur <mumrah@gmail.com>
2021-02-10 07:12:06 -08:00
Kamal Chandraprakash 7583e14fb2
MINOR: Remove unused parameters in functions. (#10035)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2021-02-10 11:18:13 +08:00
David Arthur e7e4252b0f
JUnit extensions for integration tests (#9986)
Adds JUnit 5 extension for running the same test with different types of clusters. 
See core/src/test/java/kafka/test/junit/README.md for details
2021-02-09 11:49:33 -05:00
dengziming 3769bc21b5
MINOR: replace hard-coding utf-8 with StandardCharsets.UTF_8 (#10079)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2021-02-09 10:06:01 +08:00
Colin Patrick McCabe 1f508ea8c3
MINOR: Add StorageTool as specified in KIP-631 (#10043)
Add StorageTool as specified in KIP-631. It can format and describe storage directories.  Fix a bug in `ZkMetaProperties#toString`.

Reviewers: David Arthur <mumrah@gmail.com>
2021-02-08 12:42:40 -08:00
Colin Patrick McCabe 1d3e293c08
MINOR: Add ClusterTool as specified in KIP-631 (#10047)
Add ClusterTool as specified in KIP-631. It can report the current cluster ID, and also send the new RPC for removing broker registrations.

Reviewers: David Arthur <mumrah@gmail.com>
2021-02-08 12:07:39 -08:00
Ron Dagostino 42a9355e60
MINOR: Defer log recovery until LogManager startup (#10039)
Currently log recovery begins as soon as we instantiate `LogManager`, but when using a
Raft-based metadata quorum we won't have configs until after we catch up on the metadata
log.  We therefore defer log recovery until we actually invoke `startup()` on the `LogManager`
instance.  This timing difference has no effect when using ZooKeeper because we
immediately invoke `startup()` on the instantiated instance, but it gives us the necessary
flexibility for accurate log recovery with updated configs when using a Raft-based metadata
quorum.

The `LogCleaner` is currently instantiated during construction just after log recovery
completes, and then it is started in `startup()`.  As an extra precaution, since we are
no longer performing recovery during construction, we both instantiate and start the
log cleaner in `startup()` after log recovery completes.

We also convert `LogManager` to use a `ConfigRepository` to load topic configs
(which can override the default log configs) instead of having a hard-coded
dependency on ZooKeeper.  We retrieve the topic configs when we invoke `startup()`
-- which again is effectively no different from a timing perspective than what we do
today for the ZooKeeper case.

One subtlety is that currently we create the log configs for every topic at this point
-- if a topic has no config overrides then we associate a copy of the default
configuration with the topic inside a map, and we retrieve the log configs for that
topic's partitions from from that map during recovery.  This PR makes a change to
this series of events as follows.  We do not associate a copy of the the default
configuration with a topic in the map if the topic has no configs set when we query
for them.  This saves some memory -- we don't unnecessarily copy the default
config many times -- but it also means we have use the default log configs for
that topic later on when recovery for each of its partitions begins.

The difference is that the default configs are dynamically reconfigurable, and they
could potentially change between the time when we invoke `startup()` and when
log recovery begins (log recovery can begin quite some time after `startup()` is
invoked if shutdown was unclean).  Prior to this patch such a change would not
be used; with this patch they could be if they happen before recovery begins.
This actually is better -- we are performing log recovery with the most recent
known defaults when a topic had no overrides at all. Also, `Partition.createLog`
has logic to handle missed config updates, so the behavior is eventually the same.

The transition of the broker state from `STARTING` to `RECOVERY` currently
happens within the `LogManager`, and it only occurs if the shutdown was
unclean.  We move this transition into the broker as it avoids passing a
reference to the broker state into the `LogManager`.  We also now always
transition the broker into the `RECOVERY` state as dictated by [the KIP-631 broker state machine](https://cwiki.apache.org/confluence/display/KAFKA/KIP-631%3A+The+Quorumbased+Kafka+Controller#KIP631:TheQuorumbasedKafkaController-TheBrokerStateMachine).

Finally, a few clean-ups were included. One worth highlighting is that `Partition`
no longer requires a `ConfigRepository`.

Reviewers: David Arthur <david.arthur@confluent.io>, Ismael Juma <ismael@juma.me.uk>
2021-02-07 15:46:41 -08:00
Boyang Chen d2cb2dc45d
KAFKA-9751: Forward CreateTopicsRequest for FindCoordinator/Metadata when topic creation is needed (#9579)
Consolidate auto topic creation logic to either forward a CreateTopicRequest or handling the creation directly as AutoTopicCreationManager, when handling FindCoordinator/Metadata request.

Co-authored-by: Jason Gustafson <jason@confluent.io>

Reviewers: Jason Gustafson <jason@confluent.io>
2021-02-06 13:04:30 -08:00
David Arthur 242f65e5ba
Refactor the MetadataCache into two implementations (#10049)
Refactor the MetadataCache into two implementations that both implement a common trait.  This will let us
continue to use the existing implementation when using ZK, but use the new implementation when in kip-500 mode.

Reviewers: Colin McCabe <cmccabe@apache.org>, Justine Olshan <jolshan@confluent.io>, Jason Gustafson <jason@confluent.io>
2021-02-05 16:25:26 -08:00
Ron Dagostino acf39fe94a
MINOR: Allow KafkaApis to be configured for Raft controller quorums (#10045)
`KafkaApis` is configured differently when it is used in a broker with a Raft-based controller quorum vs. a ZooKeeper quorum.  For example, when using Raft, `ForwardingManager` is required rather than optional, and there is no `AdminManager`, `KafkaController`, or `KafkaZkClient`.  This PR introduces `MetadataSupport` to abstract the two possibilities: `ZkSupport` and `RaftSupport`.  This provides a fluent way to decide what to do based on the type of support that `KafkaApis` has been configured with.  Certain types of requests are not supported when using raft (`AlterIsrRequest`, `UpdateMetadataRequest`, etc.), and `MetadataSupport` gives us an intuitive way to identify the constraints and requirements associated with the different configurations and react accordingly.

Existing tests are sufficient to detect bugs and regressions.

Reviewers: José Armando García Sancio <jsancio@gmail.com>, Jason Gustafson <jason@confluent.io>
2021-02-05 12:57:44 -08:00
Ron Dagostino a3305c4b8b
MINOR: Remove ZK dependency for coordinator topics' partition counts (#10008)
The group coordinator and the transaction state manager query ZooKeeper 
to retrieve the partition count for the topics they manager. Since ZooKeeper
won't be available when the broker is using a Raft-based metadata quorum,
this PR changes the startup function to provide an accessor function instead.
This will allow the ZK-based broker to continue using ZK, while the kip-500
broker will query the metadata provided by the metadata log.

Reviewers: Colin P. McCabe <cmccabe@apache.org>, David Arthur <david.arthur@confluent.io>

Co-authored-by: Ismael Juma <ismael@juma.me.uk>
2021-02-05 08:51:00 -08:00
Ron Dagostino c4ea6fb0a7
MINOR: Add ConfigRepository, use in Partition and KafkaApis (#10005)
`Partition` objects are able to retrieve topic configs when creating their log, and currently they do so with an implementation of `trait TopicConfigFetcher` that uses ZooKeeper.  ZooKeeper is not available when using a Raft-based metadata log, so we need to abstract the retrieval of configs so it can work either with or without ZooKeeper.  This PR introduces `trait ConfigRepository` with `ZkConfigRepository` and `CachedConfigRepository` implementations.  `Partition` objects now use a provided `ConfigRepository` to retrieve topic configs, and we eliminate `TopicConfigFetcher` as it is no longer needed.

`ReplicaManager` now contains an instance of `ConfigRepository` so it can provide it when creating `Partition` instances.

`KafkaApis` needs to be able to handle describe-config requests; it currently delegates that to `ZkAdminManager`, which of course queries ZooKeeper.  To make this work with or without ZooKeeper we move the logic from `ZkAdminManager` into a new `ConfigHelper` class that goes through a `ConfigRepository` instance.  We provide `KafkaApis` with such an instance, and it creates an instance of the helper so it can use that instead of going through `ZkAdminManager`.

Existing tests are sufficient to identify bugs and regressions in `Partition`, `ReplicaManager`, `KafkaApis`, and `ConfigHelper`.  The `ConfigRepository` implementations have their own unit tests.

Reviewers: Jason Gustafson <jason@confluent.io>
2021-02-04 12:58:26 -08:00
Ron Dagostino ca29727d2a
MINOR: Introduce KafkaBroker trait for use in dynamic reconfiguration (#10019)
Dynamic broker reconfiguration needs to occur for both ZooKeeper-based
brokers and brokers that use a Raft-based metadata quorum.  DynamicBrokerConfig
currently operates on KafkaServer, but it needs to operate on BrokerServer
(the broker implementation that will use the Raft metadata log) as well.
This PR introduces a KafkaBroker trait to allow dynamic reconfiguration to
work with either implementation.

Reviewers: José Armando García Sancio <jsancio@gmail.com>, Colin Patrick McCabe <cmccabe@confluent.io>, Ismael Juma <ismael@juma.me.uk>
2021-02-04 10:48:04 -08:00
dengziming 1c56abaa62
MINOR: Fix always pass unit test in MetadataRequestTest (#10033)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2021-02-04 13:51:53 +08:00
dengziming f4ddb89166
MINOR: remove unused import in TopicIdWithOldInterBrokerProtocolTest (#10037)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2021-02-04 11:07:45 +08:00
Jason Gustafson f58c2acf26
KAFKA-12250; Add metadata record serde for KIP-631 (#9998)
This patch adds a `RecordSerde` implementation for the metadata record format expected by KIP-631. 

Reviewers: Colin McCabe <cmccabe@apache.org>, Ismael Juma <mlists@juma.me.uk>
2021-02-03 16:16:35 -08:00
Colin P. Mccabe e119ea66cb KAFKA-12271 Immutable container classes to support new MetadataCache (#10018)
Three new classes are added to support the upcoming changes to MetadataCache
required for handling Raft metadata records.

Reviewers: Jason Gustafson <jason@confluent.io>

Co-authored-by: David Arthur <mumrah@gmail.com>
2021-02-03 17:08:13 -05:00
Colin Patrick McCabe 772f2cfc82
MINOR: Replace BrokerStates.scala with BrokerState.java (#10028)
Replace BrokerStates.scala with BrokerState.java, to make it easier to use from Java code if needed.  This also makes it easier to go from a numeric type to an enum.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2021-02-03 13:41:38 -08:00
Jason Gustafson 3acc193853
MINOR: Add mock implementation of `BrokerToControllerChannelManager` (#10026)
Tests involving `BrokerToControllerChannelManager` are simplified by being able to leverage `MockClient`. This patch introduces a `MockBrokerToControllerChannelManager` implementation which makes that possible.

The patch updates `ForwardingManagerTest` to use `MockBrokerToControllerChannelManager`. We also add a couple additional timeout cases, which exposed a minor bug. Previously we were using the wrong `TimeoutException`, which meant that expected timeout errors were in fact translated to `UNKNOWN_SERVER_ERROR`.

Reviewers: David Arthur <david.arthur@confluent.io>
2021-02-03 10:30:20 -08:00
Ron Dagostino 49f9229ea6
MINOR: Introduce ProducerIdGenerator trait (#10009)
`ProducerIdManager` is an existing class that talks to ZooKeeper directly.  We won't have ZooKeeper
when using a Raft-based metadata quorum, so we need an abstraction for the functionality of
generating producer IDs.  This PR introduces `ProducerIdGenerator` for this purpose, and we pass
an implementation when instantiating `TransactionCoordinator` rather than letting
`TransactionCoordinator.apply()` itself always create a ZooKeeper-based instance.

Reviewers: David Arthur <mumrah@gmail.com>, Ismael Juma <ismael@juma.me.uk>
2021-02-03 08:11:22 -08:00
Alok Nikhil 3f36f9a7ca
KAFKA-12249: Add client-side Decommission Broker RPC (KIP-500) (#9996)
Reviewers: Jason Gustafson <jason@confluent.io>
2021-02-03 05:46:46 -08:00
Ron Dagostino e31d3d4642
MINOR: Add HostedPartition.Deferred state in ReplicaManager (#10003)
Brokers receive metadata from the Raft metadata quorum very differently than they do from
ZooKeeper today, and this has implications for ReplicaManager.  In particular, when a broker
reads the metadata log it may not arrive at the ultimate state for a partition until it reads multiple
messages.  In normal operation the multiple messages associated with a state change will all
appear in a single batch, so they can and will be coalesced and applied together.  There are
circumstances where messages associated with partition state changes will appear across
multiple batches and we will be forced to coalesce these multiple batches together.  The
circumstances when this occurs are as follows:

- When the broker restarts it must "catch up" on the metadata log, and it is likely that the
broker will see multiple partition state changes for a single partition across different
batches while it is catching up.  For example, it will see the `TopicRecord` and the
`PartitionRecords` for the topic creation, and then it will see any `IsrChangeRecords`
that may have been recorded since the creation.  The broker does not know the state of
the topic partitions until it reads and coalesces all the messages.
- The broker will have to "catch up" on the metadata log if it becomes fenced and then
regains its lease and resumes communication with the metadata quorum.
- A fenced broker may ultimately have to perform a "soft restart" if it was fenced for so
long that the point at which it needs to resume fetching the metadata log has been
subsumed into a metadata snapshot and is no longer independently fetchable.  A soft
restart will entail some kind of metadata reset based on the latest available snapshot
plus a catchup phase to fetch after the snapshot end point.

The first case -- during startup -- occurs before clients are able to connect to the broker.
Clients are able to connect to the broker in the second case.  It is unclear if clients will be
able to to connect to the broker during a soft restart (the third case).

We need a way to defer the application of topic partition metadata in all of the above cases,
and while we are deferring the application of the metadata the broker will not service clients
for the affected partitions.

As a side note, it is arguable if the broker should be able to service clients while catching up
or not.  The decision to not service clients has no impact in the startup case -- clients can't
connect yet at that point anyway.  In the third case it is not yet clear what we are going to do,
but being unable to service clients while performing a soft reset seems reasonable.  In the
second case it is most likely true that we will catch up quickly; it would be unusual to
reestablish communication with the metadata quorum such that we gain a new lease and
begin to catch up only to lose our lease again.

So we need a way to defer the application of partition metadata and make those partitions
unavailable while deferring state changes.  This PR adds a new internal partition state to
ReplicaManager to accomplish this.  Currently the available partition states are simple
`Online`, `Offline` (meaning a log dir failure) and `None` (meaning we don't know about it). 
We add a new `Deferred` state.  We also rename a couple of methods that refer to
"nonOffline" partitions to instead refer to "online" partitions.

**The new `Deferred` state never happens when using ZooKeeper for metadata storage.**
Partitions can only enter the `Deferred` state when using a KIP-500 Raft metadata quorum
and one of the above 3 cases occurs.  The testing strategy is therefore to leverage existing
tests to confirm that there is no functionality change in the ZooKeeper case.  We will add
the logic for deferring/applying/reacting to deferred partition state in separate PRs since
that code will never be invoked in the ZooKeeper world.

Reviewers: Jason Gustafson <jason@confluent.io>, Ismael Juma <ismael@juma.me.uk>
2021-02-03 01:19:36 -08:00
Colin P. Mccabe 2ab5301085 MINOR: Upstream QuotaConfigs
This PR moves static property definitions for user client quotas into a
new class called QuotaConfigs in the clients module under the
o.a.k.common.config.internals package. This is needed to support the
client quotas work in the quorum based controller.

Reviewers: Colin McCabe <cmccabe@apache.org>
2021-02-02 21:29:51 -08:00
dengziming 1390d02ce4
MINOR: remove useless repeated method call in KafkaApiTest (#9988)
Reviewers: Ismael Juma <ismael@juma.me.uk>, Chia-Ping Tsai <chia7712@gmail.com>
2021-02-03 12:12:11 +08:00
Rajini Sivaram 5c562efb2d
KAFKA-10700 - Support mutual TLS authentication for SASL_SSL listeners (KIP-684) (#10007)
mTLS is enabled if listener-prefixed ssl.client.auth is configured for SASL_SSL listeners. Broker-wide ssl.client.auth is not applied to SASL_SSL listeners as before, but we now print a warning.

Reviewers: David Jacot <djacot@confluent.io>
2021-02-02 14:56:23 +00:00
Alok Nikhil a022072df3
KAFKA-12248; Add BrokerHeartbeat/BrokerRegistration RPCs for KIP-500 (#9994)
This patch adds the schemas and request/response objects for the `BrokerHeartbeat` and `BrokerRegistration` APIs that were added as part of KIP-631. These APIs are only exposed by the KIP-500 controller and not advertised by the broker.

Reviewers: Jason Gustafson <jason@confluent.io>
2021-02-01 21:59:30 -08:00
Jason Gustafson 62218a05d3
MINOR: Factor out controller node provider `BrokerToControllerChannelManager` (#10015)
This patch factors out a trait to allow for other ways to provide the controller `Node` object to `BrokerToControllerChannelManager`. In KIP-500, the controller will be provided from the Raft client and not the metadata cache.

Reviewers: David Arthur <david.arthur@confluent.io>
2021-02-01 14:30:53 -08:00
Luke Chen 7feb55731e
KAFKA-7940: Fix flaky test case in `CustomQuotaCallbackTest` (#9777)
This patch attempts to fix `CustomQuotaCallbackTest#testCustomQuotaCallback`. The test creates 99 partitions in a topic, and expects that we can get the partition info for all of them after 15 seconds. If we cannot, then we'll get the error:
```
org.scalatest.exceptions.TestFailedException: Partition [group1_largeTopic,69] metadata not propagated after 15000 ms
```
15 secs is not enough to complete the 99 partitions creation on a slow system. So, we fix it by explicitly wait until we've got the expected partition size before retrieving each partition info and we increase the wait time to 60s for all partition metadata to be propagated.

Reviewers: Jason Gustafson <jason@confluent.io>
2021-02-01 10:16:28 -08:00
feyman2016 db73d86ea6
KAFKA-10636; Bypass log validation and offset assignment for writes from the raft leader (#9739)
Since the Raft leader is already doing the work of assigning offsets and the leader epoch, we can skip the same logic in `Log.appendAsLeader`. This lets us avoid an unnecessary round of decompression.

Reviewers: dengziming <dengziming1993@gmail.com>, Jason Gustafson <jason@confluent.io>
2021-02-01 10:05:47 -08:00
Jason Gustafson 7205cd36e4
KAFKA-12236; New meta.properties logic for KIP-500 (#9967)
This patch contains the new handling of `meta.properties` required by the KIP-500 server as specified in KIP-631. When using the self-managed quorum, the `meta.properties` file is required in each log directory with the new `version` property set to 1. It must include the `cluster.id` property and it must have a `node.id` matching that in the configuration.

The behavior of `meta.properties` for the Zookeeper-based `KafkaServer` does not change. We treat `meta.properties` as optional and as if it were `version=0`. We continue to generate the clusterId and/or the brokerId through Zookeeper as needed.

Reviewers: Ismael Juma <ismael@juma.me.uk>, Chia-Ping Tsai <chia7712@gmail.com>
2021-01-30 17:05:31 -08:00
Jason Gustafson e9edf10486
KAFKA-12238; Implement `DescribeProducers` API from KIP-664 (#9979)
Implements the `DescribeProducers` API specified by KIP-664: https://cwiki.apache.org/confluence/display/KAFKA/KIP-664%3A+Provide+tooling+to+detect+and+abort+hanging+transactions.

Reviewers: David Jacot <djacot@confluent.io>
2021-01-29 17:00:14 -08:00