KAFKA-15369: Implement KIP-919: Allow AC to Talk Directly with Controllers

Implement KIP-919: Allow AdminClient to Talk Directly with the KRaft Controller Quorum and add
Controller Registration.

This KIP adds a new version of DescribeClusterRequest which is supported by KRaft controllers.
It also teaches AdminClient how to use this new DESCRIBE_CLUSTER request to talk directly with the
controller quorum.

In order to share the DESCRIBE_CLUSTER logic between broker and controller, this PR factors it out
into AuthHelper.computeDescribeClusterResponse.

On the controller side, the controllers now try to register themselves with the current active
controller, by sending a CONTROLLER_REGISTRATION request. This, in turn, is converted into a
RegisterControllerRecord by the active controller. ClusterImage, ClusterDelta, and all other
associated classes have been upgraded to propagate the new metadata.

This KIP adds three new errors codes: MISMATCHED_ENDPOINT_TYPE, UNSUPPORTED_ENDPOINT_TYPE, and
UNKNOWN_CONTROLLER_ID. The endpoint type errors can be returned from DescribeClusterRequest
This commit is contained in:
Colin P. McCabe 2023-08-04 09:45:16 -07:00
parent 6bd17419b7
commit 0069fcea13
61 changed files with 2864 additions and 400 deletions

View File

@ -0,0 +1,47 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.clients.admin;
/**
* Identifies the endpoint type, as specified by KIP-919.
*/
public enum EndpointType {
UNKNOWN((byte) 0),
BROKER((byte) 1),
CONTROLLER((byte) 2);
private final byte id;
EndpointType(byte id) {
this.id = id;
}
public byte id() {
return id;
}
public static EndpointType fromId(byte id) {
if (id == BROKER.id) {
return BROKER;
} else if (id == CONTROLLER.id) {
return CONTROLLER;
} else {
return UNKNOWN;
}
}
}

View File

@ -0,0 +1,23 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.common.errors;
public class MismatchedEndpointTypeException extends ApiException {
public MismatchedEndpointTypeException(String message) {
super(message);
}
}

View File

@ -0,0 +1,23 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.common.errors;
public class UnknownControllerIdException extends ApiException {
public UnknownControllerIdException(String message) {
super(message);
}
}

View File

@ -0,0 +1,23 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.common.errors;
public class UnsupportedEndpointTypeException extends ApiException {
public UnsupportedEndpointTypeException(String message) {
super(message);
}
}

View File

@ -111,7 +111,8 @@ public enum ApiKeys {
DESCRIBE_TRANSACTIONS(ApiMessageType.DESCRIBE_TRANSACTIONS),
LIST_TRANSACTIONS(ApiMessageType.LIST_TRANSACTIONS),
ALLOCATE_PRODUCER_IDS(ApiMessageType.ALLOCATE_PRODUCER_IDS, true, true),
CONSUMER_GROUP_HEARTBEAT(ApiMessageType.CONSUMER_GROUP_HEARTBEAT);
CONSUMER_GROUP_HEARTBEAT(ApiMessageType.CONSUMER_GROUP_HEARTBEAT),
CONTROLLER_REGISTRATION(ApiMessageType.CONTROLLER_REGISTRATION);
private static final Map<ApiMessageType.ListenerType, EnumSet<ApiKeys>> APIS_BY_LISTENER =
new EnumMap<>(ApiMessageType.ListenerType.class);

View File

@ -78,6 +78,7 @@ import org.apache.kafka.common.errors.LeaderNotAvailableException;
import org.apache.kafka.common.errors.ListenerNotFoundException;
import org.apache.kafka.common.errors.LogDirNotFoundException;
import org.apache.kafka.common.errors.MemberIdRequiredException;
import org.apache.kafka.common.errors.MismatchedEndpointTypeException;
import org.apache.kafka.common.errors.NetworkException;
import org.apache.kafka.common.errors.NewLeaderElectedException;
import org.apache.kafka.common.errors.NoReassignmentInProgressException;
@ -118,6 +119,7 @@ import org.apache.kafka.common.errors.TransactionCoordinatorFencedException;
import org.apache.kafka.common.errors.TransactionalIdAuthorizationException;
import org.apache.kafka.common.errors.TransactionalIdNotFoundException;
import org.apache.kafka.common.errors.UnacceptableCredentialException;
import org.apache.kafka.common.errors.UnknownControllerIdException;
import org.apache.kafka.common.errors.UnknownLeaderEpochException;
import org.apache.kafka.common.errors.UnknownMemberIdException;
import org.apache.kafka.common.errors.UnknownProducerIdException;
@ -129,6 +131,7 @@ import org.apache.kafka.common.errors.UnstableOffsetCommitException;
import org.apache.kafka.common.errors.UnsupportedAssignorException;
import org.apache.kafka.common.errors.UnsupportedByAuthenticationException;
import org.apache.kafka.common.errors.UnsupportedCompressionTypeException;
import org.apache.kafka.common.errors.UnsupportedEndpointTypeException;
import org.apache.kafka.common.errors.UnsupportedForMessageFormatException;
import org.apache.kafka.common.errors.UnsupportedSaslMechanismException;
import org.apache.kafka.common.errors.UnsupportedVersionException;
@ -380,7 +383,10 @@ public enum Errors {
FENCED_MEMBER_EPOCH(110, "The member epoch is fenced by the group coordinator. The member must abandon all its partitions and rejoin.", FencedMemberEpochException::new),
UNRELEASED_INSTANCE_ID(111, "The instance ID is still used by another member in the consumer group. That member must leave first.", UnreleasedInstanceIdException::new),
UNSUPPORTED_ASSIGNOR(112, "The assignor or its version range is not supported by the consumer group.", UnsupportedAssignorException::new),
STALE_MEMBER_EPOCH(113, "The member epoch is stale. The member must retry after receiving its updated member epoch via the ConsumerGroupHeartbeat API.", StaleMemberEpochException::new);
STALE_MEMBER_EPOCH(113, "The member epoch is stale. The member must retry after receiving its updated member epoch via the ConsumerGroupHeartbeat API.", StaleMemberEpochException::new),
MISMATCHED_ENDPOINT_TYPE(114, "The request was sent to an endpoint of the wrong type.", MismatchedEndpointTypeException::new),
UNSUPPORTED_ENDPOINT_TYPE(115, "This endpoint type is not supported yet.", UnsupportedEndpointTypeException::new),
UNKNOWN_CONTROLLER_ID(116, "This controller ID is not known.", UnknownControllerIdException::new);
private static final Logger log = LoggerFactory.getLogger(Errors.class);

View File

@ -0,0 +1,73 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.common.requests;
import org.apache.kafka.common.message.ControllerRegistrationRequestData;
import org.apache.kafka.common.message.ControllerRegistrationResponseData;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors;
import java.nio.ByteBuffer;
public class ControllerRegistrationRequest extends AbstractRequest {
public static class Builder extends AbstractRequest.Builder<ControllerRegistrationRequest> {
private final ControllerRegistrationRequestData data;
public Builder(ControllerRegistrationRequestData data) {
super(ApiKeys.BROKER_HEARTBEAT);
this.data = data;
}
@Override
public ControllerRegistrationRequest build(short version) {
return new ControllerRegistrationRequest(data, version);
}
@Override
public String toString() {
return data.toString();
}
}
private final ControllerRegistrationRequestData data;
public ControllerRegistrationRequest(ControllerRegistrationRequestData data, short version) {
super(ApiKeys.CONTROLLER_REGISTRATION, version);
this.data = data;
}
@Override
public ControllerRegistrationRequestData data() {
return data;
}
@Override
public ControllerRegistrationResponse getErrorResponse(int throttleTimeMs, Throwable e) {
Errors error = Errors.forException(e);
return new ControllerRegistrationResponse(new ControllerRegistrationResponseData()
.setThrottleTimeMs(throttleTimeMs)
.setErrorCode(error.code())
.setErrorMessage(error.message()));
}
public static ControllerRegistrationRequest parse(ByteBuffer buffer, short version) {
return new ControllerRegistrationRequest(
new ControllerRegistrationRequestData(new ByteBufferAccessor(buffer), version),
version);
}
}

View File

@ -0,0 +1,61 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.common.requests;
import org.apache.kafka.common.message.ControllerRegistrationResponseData;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors;
import java.nio.ByteBuffer;
import java.util.Collections;
import java.util.Map;
public class ControllerRegistrationResponse extends AbstractResponse {
private final ControllerRegistrationResponseData data;
public ControllerRegistrationResponse(ControllerRegistrationResponseData data) {
super(ApiKeys.CONTROLLER_REGISTRATION);
this.data = data;
}
@Override
public ControllerRegistrationResponseData data() {
return data;
}
@Override
public int throttleTimeMs() {
return data.throttleTimeMs();
}
@Override
public void maybeSetThrottleTimeMs(int throttleTimeMs) {
data.setThrottleTimeMs(throttleTimeMs);
}
@Override
public Map<Errors, Integer> errorCounts() {
return Collections.singletonMap(Errors.forCode(data.errorCode()), 1);
}
public static ControllerRegistrationResponse parse(ByteBuffer buffer, short version) {
return new ControllerRegistrationResponse(
new ControllerRegistrationResponseData(new ByteBufferAccessor(buffer), version));
}
}

View File

@ -0,0 +1,53 @@
// Licensed to the Apache Software Foundation (ASF) under one or more
// contributor license agreements. See the NOTICE file distributed with
// this work for additional information regarding copyright ownership.
// The ASF licenses this file to You under the Apache License, Version 2.0
// (the "License"); you may not use this file except in compliance with
// the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
{
"apiKey": 70,
"type": "request",
"listeners": ["controller"],
"name": "ControllerRegistrationRequest",
"validVersions": "0",
"flexibleVersions": "0+",
"fields": [
{ "name": "ControllerId", "type": "int32", "versions": "0+",
"about": "The ID of the controller to register." },
{ "name": "ActiveControllerEpoch", "type": "int32", "versions": "0+",
"about": "The epoch of the current active controller." },
{ "name": "IncarnationId", "type": "uuid", "versions": "0+",
"about": "The controller incarnation ID, which is unique to each process run." },
{ "name": "ZkMigrationReady", "type": "bool", "versions": "0+",
"about": "Set if the required configurations for ZK migration are present." },
{ "name": "Listeners", "type": "[]Listener",
"about": "The listeners of this controller", "versions": "0+", "fields": [
{ "name": "Name", "type": "string", "versions": "0+", "mapKey": true,
"about": "The name of the endpoint." },
{ "name": "Host", "type": "string", "versions": "0+",
"about": "The hostname." },
{ "name": "Port", "type": "uint16", "versions": "0+",
"about": "The port." },
{ "name": "SecurityProtocol", "type": "int16", "versions": "0+",
"about": "The security protocol." }
]},
{ "name": "Features", "type": "[]Feature",
"about": "The features on this controller", "versions": "0+", "fields": [
{ "name": "Name", "type": "string", "versions": "0+", "mapKey": true,
"about": "The feature name." },
{ "name": "MinSupportedVersion", "type": "int16", "versions": "0+",
"about": "The minimum supported feature level." },
{ "name": "MaxSupportedVersion", "type": "int16", "versions": "0+",
"about": "The maximum supported feature level." }
]}
]
}

View File

@ -0,0 +1,30 @@
// Licensed to the Apache Software Foundation (ASF) under one or more
// contributor license agreements. See the NOTICE file distributed with
// this work for additional information regarding copyright ownership.
// The ASF licenses this file to You under the Apache License, Version 2.0
// (the "License"); you may not use this file except in compliance with
// the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
{
"apiKey": 70,
"type": "response",
"name": "ControllerRegistrationResponse",
"validVersions": "0",
"flexibleVersions": "0+",
"fields": [
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "0+",
"about": "Duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." },
{ "name": "ErrorCode", "type": "int16", "versions": "0+",
"about": "The response error code." },
{ "name": "ErrorMessage", "type": "string", "nullableVersions": "0+", "versions": "0+",
"about": "The response error message, or null if there was no error." }
]
}

View File

@ -16,12 +16,17 @@
{
"apiKey": 60,
"type": "request",
"listeners": ["zkBroker", "broker"],
"listeners": ["zkBroker", "broker", "controller"],
"name": "DescribeClusterRequest",
"validVersions": "0",
//
// Version 1 adds EndpointType for KIP-919 support.
//
"validVersions": "0-1",
"flexibleVersions": "0+",
"fields": [
{ "name": "IncludeClusterAuthorizedOperations", "type": "bool", "versions": "0+",
"about": "Whether to include cluster authorized operations." }
"about": "Whether to include cluster authorized operations." },
{ "name": "EndpointType", "type": "int8", "versions": "1+", "default": "1",
"about": "The endpoint type to describe. 1=brokers, 2=controllers." }
]
}

View File

@ -17,7 +17,11 @@
"apiKey": 60,
"type": "response",
"name": "DescribeClusterResponse",
"validVersions": "0",
//
// Version 1 adds the EndpointType field, and makes MISMATCHED_ENDPOINT_TYPE and
// UNSUPPORTED_ENDPOINT_TYPE valid top-level response error codes.
//
"validVersions": "0-1",
"flexibleVersions": "0+",
"fields": [
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "0+",
@ -26,6 +30,8 @@
"about": "The top-level error code, or 0 if there was no error" },
{ "name": "ErrorMessage", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null",
"about": "The top-level error message, or null if there was no error." },
{ "name": "EndpointType", "type": "int8", "versions": "1+", "default": "1",
"about": "The endpoint type that was described. 1=brokers, 2=controllers." },
{ "name": "ClusterId", "type": "string", "versions": "0+",
"about": "The cluster ID that responding broker belongs to." },
{ "name": "ControllerId", "type": "int32", "versions": "0+", "default": "-1", "entityType": "brokerId",

View File

@ -0,0 +1,48 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.clients.admin;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.Timeout;
import static org.junit.jupiter.api.Assertions.assertEquals;
@Timeout(60)
public class EndpointTypeTest {
@Test
public void testRoundTripBroker() {
testRoundTrip(EndpointType.BROKER);
}
@Test
public void testRoundTripController() {
testRoundTrip(EndpointType.CONTROLLER);
}
@Test
public void testUnknown() {
assertEquals(EndpointType.UNKNOWN, EndpointType.fromId((byte) 0));
assertEquals(EndpointType.UNKNOWN, EndpointType.fromId((byte) 3));
}
private void testRoundTrip(EndpointType type) {
byte id = type.id();
assertEquals(type, EndpointType.fromId(id));
}
}

View File

@ -22,9 +22,14 @@ import java.util.Collections
import kafka.network.RequestChannel
import kafka.security.authorizer.AclEntry
import kafka.utils.CoreUtils
import org.apache.kafka.clients.admin.EndpointType
import org.apache.kafka.common.acl.AclOperation
import org.apache.kafka.common.acl.AclOperation.DESCRIBE
import org.apache.kafka.common.errors.ClusterAuthorizationException
import org.apache.kafka.common.requests.RequestContext
import org.apache.kafka.common.message.DescribeClusterResponseData
import org.apache.kafka.common.message.DescribeClusterResponseData.DescribeClusterBrokerCollection
import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.requests.{DescribeClusterRequest, RequestContext}
import org.apache.kafka.common.resource.Resource.CLUSTER_NAME
import org.apache.kafka.common.resource.ResourceType.CLUSTER
import org.apache.kafka.common.resource.{PatternType, Resource, ResourcePattern, ResourceType}
@ -35,7 +40,6 @@ import scala.collection.{Map, Seq}
import scala.jdk.CollectionConverters._
class AuthHelper(authorizer: Option[Authorizer]) {
def authorize(requestContext: RequestContext,
operation: AclOperation,
resourceType: ResourceType,
@ -130,4 +134,57 @@ class AuthHelper(authorizer: Option[Authorizer]) {
}
}
def computeDescribeClusterResponse(
request: RequestChannel.Request,
expectedEndpointType: EndpointType,
clusterId: String,
getNodes: () => DescribeClusterBrokerCollection,
getControllerId: () => Int
): DescribeClusterResponseData = {
val describeClusterRequest = request.body[DescribeClusterRequest]
val requestEndpointType = EndpointType.fromId(describeClusterRequest.data().endpointType())
if (requestEndpointType.equals(EndpointType.UNKNOWN)) {
return new DescribeClusterResponseData().
setErrorCode(if (request.header.data().requestApiVersion() == 0) {
Errors.INVALID_REQUEST.code()
} else {
Errors.UNSUPPORTED_ENDPOINT_TYPE.code()
}).
setErrorMessage("Unsupported endpoint type " + describeClusterRequest.data().endpointType().toInt)
} else if (!expectedEndpointType.equals(requestEndpointType)) {
return new DescribeClusterResponseData().
setErrorCode(if (request.header.data().requestApiVersion() == 0) {
Errors.INVALID_REQUEST.code()
} else {
Errors.MISMATCHED_ENDPOINT_TYPE.code()
}).
setErrorMessage("The request was sent to an endpoint of type " + expectedEndpointType +
", but we wanted an endpoint of type " + requestEndpointType)
}
var clusterAuthorizedOperations = Int.MinValue // Default value in the schema
// get cluster authorized operations
if (describeClusterRequest.data.includeClusterAuthorizedOperations) {
if (authorize(request.context, DESCRIBE, CLUSTER, CLUSTER_NAME))
clusterAuthorizedOperations = authorizedOperations(request, Resource.CLUSTER)
else
clusterAuthorizedOperations = 0
}
// Get the node list and the controller ID.
val nodes = getNodes()
val controllerId = getControllerId()
// If the provided controller ID is not in the node list, return -1 instead
// to avoid confusing the client. This could happen in a case where we know
// the controller ID, but we don't yet have KIP-919 information about that
// controller.
val effectiveControllerId = if (nodes.find(controllerId) == null) {
-1
} else {
controllerId
}
new DescribeClusterResponseData().
setClusterId(clusterId).
setControllerId(effectiveControllerId).
setClusterAuthorizedOperations(clusterAuthorizedOperations).
setBrokers(nodes)
}
}

View File

@ -26,7 +26,7 @@ import kafka.network.RequestChannel
import kafka.raft.RaftManager
import kafka.server.QuotaFactory.QuotaManagers
import kafka.utils.Logging
import org.apache.kafka.clients.admin.AlterConfigOp
import org.apache.kafka.clients.admin.{AlterConfigOp, EndpointType}
import org.apache.kafka.common.Uuid.ZERO_UUID
import org.apache.kafka.common.acl.AclOperation.{ALTER, ALTER_CONFIGS, CLUSTER_ACTION, CREATE, DELETE, DESCRIBE, DESCRIBE_CONFIGS}
import org.apache.kafka.common.config.ConfigResource
@ -46,9 +46,10 @@ import org.apache.kafka.common.requests._
import org.apache.kafka.common.resource.Resource.CLUSTER_NAME
import org.apache.kafka.common.resource.ResourceType.{CLUSTER, TOPIC}
import org.apache.kafka.common.utils.Time
import org.apache.kafka.common.{Node, Uuid}
import org.apache.kafka.common.Uuid
import org.apache.kafka.controller.ControllerRequestContext.requestTimeoutMsToDeadlineNs
import org.apache.kafka.controller.{Controller, ControllerRequestContext}
import org.apache.kafka.image.publisher.ControllerRegistrationsPublisher
import org.apache.kafka.metadata.{BrokerHeartbeatReply, BrokerRegistrationReply}
import org.apache.kafka.server.authorizer.Authorizer
import org.apache.kafka.server.common.ApiMessageAndVersion
@ -59,7 +60,8 @@ import scala.jdk.CollectionConverters._
/**
* Request handler for Controller APIs
*/
class ControllerApis(val requestChannel: RequestChannel,
class ControllerApis(
val requestChannel: RequestChannel,
val authorizer: Option[Authorizer],
val quotas: QuotaManagers,
val time: Time,
@ -67,8 +69,9 @@ class ControllerApis(val requestChannel: RequestChannel,
val raftManager: RaftManager[ApiMessageAndVersion],
val config: KafkaConfig,
val metaProperties: MetaProperties,
val controllerNodes: Seq[Node],
val apiVersionManager: ApiVersionManager) extends ApiRequestHandler with Logging {
val controllerRegistrations: ControllerRegistrationsPublisher,
val apiVersionManager: ApiVersionManager
) extends ApiRequestHandler with Logging {
this.logIdent = s"[ControllerApis nodeId=${config.nodeId}] "
val authHelper = new AuthHelper(authorizer)
@ -111,6 +114,7 @@ class ControllerApis(val requestChannel: RequestChannel,
case ApiKeys.DELETE_ACLS => aclApis.handleDeleteAcls(request)
case ApiKeys.ELECT_LEADERS => handleElectLeaders(request)
case ApiKeys.UPDATE_FEATURES => handleUpdateFeatures(request)
case ApiKeys.DESCRIBE_CLUSTER => handleDescribeCluster(request)
case _ => throw new ApiException(s"Unsupported ApiKey ${request.context.header.apiKey}")
}
@ -818,6 +822,20 @@ class ControllerApis(val requestChannel: RequestChannel,
}
}
def handleControllerRegistration(request: RequestChannel.Request): CompletableFuture[Unit] = {
val registrationRequest = request.body[ControllerRegistrationRequest]
authHelper.authorizeClusterOperation(request, CLUSTER_ACTION)
val context = new ControllerRequestContext(request.context.header.data, request.context.principal,
OptionalLong.empty())
controller.registerController(context, registrationRequest.data)
.thenApply[Unit] { _ =>
requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
new ControllerRegistrationResponse(new ControllerRegistrationResponseData().
setThrottleTimeMs(requestThrottleMs)))
}
}
def handleAlterPartitionReassignments(request: RequestChannel.Request): CompletableFuture[Unit] = {
val alterRequest = request.body[AlterPartitionReassignmentsRequest]
authHelper.authorizeClusterOperation(request, ALTER)
@ -887,4 +905,21 @@ class ControllerApis(val requestChannel: RequestChannel,
}
}
}
def handleDescribeCluster(request: RequestChannel.Request): CompletableFuture[Unit] = {
// Unlike on the broker, DESCRIBE_CLUSTER on the controller requires a high level of
// permissions (ALTER on CLUSTER).
authHelper.authorizeClusterOperation(request, ALTER)
val response = authHelper.computeDescribeClusterResponse(
request,
EndpointType.CONTROLLER,
metaProperties.clusterId,
() => controllerRegistrations.describeClusterControllers(request.context.listenerName()),
() => raftManager.leaderAndEpoch.leaderId().orElse(-1)
)
requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
new DescribeClusterResponse(response.setThrottleTimeMs(requestThrottleMs)))
CompletableFuture.completedFuture[Unit](())
}
}

View File

@ -0,0 +1,314 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.server
import java.util
import java.util.concurrent.TimeUnit.MILLISECONDS
import kafka.utils.Logging
import org.apache.kafka.clients.ClientResponse
import org.apache.kafka.common.Uuid
import org.apache.kafka.common.message.ControllerRegistrationRequestData.ListenerCollection
import org.apache.kafka.common.message.ControllerRegistrationRequestData
import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.requests.{ControllerRegistrationRequest, ControllerRegistrationResponse}
import org.apache.kafka.metadata.VersionRange
import org.apache.kafka.common.utils.{ExponentialBackoff, LogContext, Time}
import org.apache.kafka.image.loader.LoaderManifest
import org.apache.kafka.image.{MetadataDelta, MetadataImage}
import org.apache.kafka.image.publisher.MetadataPublisher
import org.apache.kafka.queue.EventQueue.DeadlineFunction
import org.apache.kafka.queue.{EventQueue, KafkaEventQueue}
import org.apache.kafka.server.common.MetadataVersion
import scala.jdk.CollectionConverters._
/**
* The broker lifecycle manager 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.
*
* This code uses an event queue paradigm. Modifications get translated into events, which
* are placed on the queue to be processed sequentially. As described in the JavaDoc for
* each variable, most mutable state can be accessed only from that event queue thread.
* In some cases we expose a volatile variable which can be read from any thread, but only
* written from the event queue thread.
*/
class ControllerRegistrationManager(
val config: KafkaConfig,
val clusterId: String,
val time: Time,
val threadNamePrefix: String,
val supportedFeatures: util.Map[String, VersionRange],
val controllerEpochProvider: () => Int,
val incarnationId: Uuid
) extends Logging with MetadataPublisher {
override def name(): String = "ControllerRegistrationManager"
val nodeId: Int = config.nodeId
private def logPrefix(): String = {
val builder = new StringBuilder("[ControllerRegistrationManager")
builder.append(" id=").append(config.nodeId)
builder.append(" incarnation=").append(incarnationId)
builder.append("] ")
builder.toString()
}
val logContext = new LogContext(logPrefix())
this.logIdent = logContext.logPrefix()
val listenerCollection = {
val collection = new ListenerCollection()
config.controllerListeners.foreach(endPoint => {
collection.add(new ControllerRegistrationRequestData.Listener().
setHost(endPoint.host).
setName(endPoint.listenerName.value()).
setPort(endPoint.port).
setSecurityProtocol(endPoint.securityProtocol.id))
})
collection
}
/**
* The exponential backoff to use for resending communication.
*/
private val resendExponentialBackoff =
new ExponentialBackoff(100, 2, 120000L, 0.02)
/**
* The number of RPCs that we are waiting for. Only read or written from the event queue thread.
*/
var pendingRpcs = 0L
/**
* The number of RPCs that we successfully sent.
* Only read or written from the event queue thread.
*/
var successfulRpcs = 0L
/**
* The number of RPCs that we failed to send, or got back a failure response for. This is
* cleared after a success. Only read or written from the event queue thread.
*/
var failedRpcs = 0L
/**
* The current metadata version that is in effect. Only read or written from the event queue thread.
*/
private var metadataVersion: MetadataVersion = MetadataVersion.MINIMUM_KRAFT_VERSION
/**
* True if we're registered. Only read or written from the event queue thread.
*/
var registered: Boolean = false
/**
* The channel manager, or null if this manager has not been started yet. This variable
* can only be read or written from the event queue thread.
*/
private var _channelManager: BrokerToControllerChannelManager = _
/**
* The event queue.
*/
private[server] val eventQueue = new KafkaEventQueue(time,
logContext,
threadNamePrefix + "registration-manager-",
new ShutdownEvent())
private class ShutdownEvent extends EventQueue.Event {
override def run(): Unit = {
try {
info(s"shutting down.")
if (_channelManager != null) {
_channelManager.shutdown()
_channelManager = null
}
} catch {
case t: Throwable => error("ControllerRegistrationManager.stop error", t)
}
}
}
/**
* Start the BrokerLifecycleManager.
*
* @param channelManager The brokerToControllerChannelManager to use.
*/
def start(channelManager: BrokerToControllerChannelManager): Unit = {
eventQueue.append(() => {
try {
info(s"initialized channel manager.")
_channelManager = channelManager
maybeSendControllerRegistration()
} catch {
case t: Throwable => error("start error", t)
}
})
}
/**
* Start shutting down the BrokerLifecycleManager, but do not block.
*/
def beginShutdown(): Unit = {
eventQueue.beginShutdown("beginShutdown");
}
/**
* Shut down the BrokerLifecycleManager and block until all threads are joined.
*/
override def close(): Unit = {
beginShutdown()
eventQueue.close()
}
override def onMetadataUpdate(
delta: MetadataDelta,
newImage: MetadataImage,
manifest: LoaderManifest
): Unit = {
if (delta.featuresDelta() != null ||
(delta.clusterDelta() != null && delta.clusterDelta().changedControllers().containsKey(nodeId))) {
eventQueue.append(new MetadataUpdateEvent(delta, newImage))
}
}
private class MetadataUpdateEvent(
delta: MetadataDelta,
newImage: MetadataImage
) extends EventQueue.Event {
override def run(): Unit = {
try {
if (delta.featuresDelta() != null) {
metadataVersion = newImage.features().metadataVersion()
}
if (delta.clusterDelta() != null) {
if (delta.clusterDelta().changedControllers().containsKey(nodeId)) {
val curRegistration = newImage.cluster().controllers().get(nodeId)
if (curRegistration == null) {
info(s"Registration removed for this node ID.")
registered = false
} else if (!curRegistration.incarnationId().equals(incarnationId)) {
info(s"Found registration for ${curRegistration.incarnationId()} instead of our incarnation.")
registered = false
} else {
info(s"Our registration has been persisted to the metadata log.")
registered = true
}
}
}
maybeSendControllerRegistration()
} catch {
case t: Throwable => error("onMetadataUpdate error", t)
}
}
}
private def maybeSendControllerRegistration(): Unit = {
if (registered) {
debug("maybeSendControllerRegistration: controller is already registered.")
} else if (_channelManager == null) {
debug("maybeSendControllerRegistration: cannot register yet because the channel manager has " +
"not been initialized.")
} else if (!metadataVersion.isControllerRegistrationSupported) {
info("maybeSendControllerRegistration: cannot register yet because the metadata version is " +
s"still $metadataVersion, which does not support KIP-919 controller registration.")
} else if (pendingRpcs > 0) {
info("maybeSendControllerRegistration: waiting for the previous RPC to complete.");
} else {
sendControllerRegistration()
}
}
private def sendControllerRegistration(): Unit = {
val features = new ControllerRegistrationRequestData.FeatureCollection()
supportedFeatures.asScala.foreach {
case (name, range) => features.add(new ControllerRegistrationRequestData.Feature().
setName(name).
setMinSupportedVersion(range.min()).
setMaxSupportedVersion(range.max()))
}
val data = new ControllerRegistrationRequestData().
setControllerId(nodeId).
setActiveControllerEpoch(controllerEpochProvider()).
setFeatures(features).
setIncarnationId(incarnationId).
setListeners(listenerCollection)
info(s"sendControllerRegistration: attempting to send $data")
_channelManager.sendRequest(new ControllerRegistrationRequest.Builder(data),
new RegistrationResponseHandler())
pendingRpcs = pendingRpcs + 1
}
private class RegistrationResponseHandler extends ControllerRequestCompletionHandler {
override def onComplete(response: ClientResponse): Unit = {
pendingRpcs = pendingRpcs - 1
if (response.authenticationException() != null) {
error(s"RegistrationResponseHandler: authentication error", response.authenticationException())
scheduleNextCommunicationAfterFailure()
} else if (response.versionMismatch() != null) {
error(s"RegistrationResponseHandler: unsupported API version error", response.versionMismatch())
scheduleNextCommunicationAfterFailure()
} else if (response.responseBody() == null) {
error(s"RegistrationResponseHandler: unknown error")
scheduleNextCommunicationAfterFailure()
} else if (!response.responseBody().isInstanceOf[ControllerRegistrationResponse]) {
error(s"RegistrationResponseHandler: invalid response type error")
scheduleNextCommunicationAfterFailure()
} else {
val message = response.responseBody().asInstanceOf[ControllerRegistrationResponse]
val errorCode = Errors.forCode(message.data().errorCode())
if (errorCode == Errors.NONE) {
successfulRpcs = successfulRpcs + 1
failedRpcs = 0
info(s"RegistrationResponseHandler: controller acknowledged ControllerRegistrationRequest.")
} else {
info(s"RegistrationResponseHandler: controller returned error $errorCode " +
s"(${message.data().errorMessage()})")
scheduleNextCommunicationAfterFailure()
}
}
}
override def onTimeout(): Unit = {
error(s"RegistrationResponseHandler: channel manager timed out before sending the request.")
scheduleNextCommunicationAfterFailure()
}
}
private def scheduleNextCommunicationAfterFailure(): Unit = {
val delayMs = resendExponentialBackoff.backoff(failedRpcs)
failedRpcs = failedRpcs + 1
scheduleNextCommunication(delayMs)
}
private def scheduleNextCommunication(intervalMs: Long): Unit = {
trace(s"Scheduling next communication at ${intervalMs} ms from now.")
val deadlineNs = time.nanoseconds() + MILLISECONDS.toNanos(intervalMs)
eventQueue.scheduleDeferred("communication",
new DeadlineFunction(deadlineNs),
() => maybeSendControllerRegistration())
}
}

View File

@ -34,10 +34,10 @@ import org.apache.kafka.common.message.ApiMessageType.ListenerType
import org.apache.kafka.common.security.scram.internals.ScramMechanism
import org.apache.kafka.common.security.token.delegation.internals.DelegationTokenCache
import org.apache.kafka.common.utils.LogContext
import org.apache.kafka.common.{ClusterResource, Endpoint}
import org.apache.kafka.common.{ClusterResource, Endpoint, Uuid}
import org.apache.kafka.controller.metrics.{ControllerMetadataMetricsPublisher, QuorumControllerMetrics}
import org.apache.kafka.controller.{Controller, QuorumController, QuorumFeatures}
import org.apache.kafka.image.publisher.MetadataPublisher
import org.apache.kafka.image.publisher.{ControllerRegistrationsPublisher, MetadataPublisher}
import org.apache.kafka.metadata.KafkaConfigSchema
import org.apache.kafka.metadata.authorizer.ClusterMetadataAuthorizer
import org.apache.kafka.metadata.bootstrap.BootstrapMetadata
@ -91,6 +91,7 @@ class ControllerServer(
private val metricsGroup = new KafkaMetricsGroup(this.getClass)
val config = sharedServer.controllerConfig
val logContext = new LogContext(s"[ControllerServer id=${config.nodeId}] ")
val time = sharedServer.time
def metrics = sharedServer.metrics
def raftManager: KafkaRaftManager[ApiMessageAndVersion] = sharedServer.raftManager
@ -116,7 +117,15 @@ class ControllerServer(
var migrationSupport: Option[ControllerMigrationSupport] = None
def kafkaYammerMetrics: KafkaYammerMetrics = KafkaYammerMetrics.INSTANCE
val metadataPublishers: util.List[MetadataPublisher] = new util.ArrayList[MetadataPublisher]()
val featuresPublisher = new FeaturesPublisher()
val featuresPublisher = new FeaturesPublisher(logContext)
val controllerRegistrationsPublisher = new ControllerRegistrationsPublisher()
val registrationManager = new ControllerRegistrationManager(config,
clusterId,
time,
s"controller-${config.nodeId}-",
QuorumFeatures.defaultFeatureMap(),
() => raftManager.client.leaderAndEpoch().epoch(),
Uuid.randomUuid())
private def maybeChangeStatus(from: ProcessStatus, to: ProcessStatus): Boolean = {
lock.lock()
@ -136,7 +145,7 @@ class ControllerServer(
if (!maybeChangeStatus(SHUTDOWN, STARTING)) return
val startupDeadline = Deadline.fromDelay(time, config.serverMaxStartupTimeMs, TimeUnit.MILLISECONDS)
try {
this.logIdent = new LogContext(s"[ControllerServer id=${config.nodeId}] ").logPrefix()
this.logIdent = logContext.logPrefix()
info("Starting controller")
config.dynamicConfig.initialize(zkClientOpt = None)
@ -200,10 +209,9 @@ class ControllerServer(
sharedServer.controllerQuorumVotersFuture,
startupDeadline, time)
val controllerNodes = RaftConfig.voterConnectionsToNodes(voterConnections)
val quorumFeatures = QuorumFeatures.create(config.nodeId,
sharedServer.raftManager.apiVersions,
val quorumFeatures = new QuorumFeatures(config.nodeId,
QuorumFeatures.defaultFeatureMap(),
controllerNodes)
controllerNodes.asScala.map(node => Integer.valueOf(node.id())).asJava)
val controllerBuilder = {
val leaderImbalanceCheckIntervalNs = if (config.autoLeaderRebalanceEnable) {
@ -292,7 +300,7 @@ class ControllerServer(
raftManager,
config,
sharedServer.metaProps,
controllerNodes.asScala.toSeq,
controllerRegistrationsPublisher,
apiVersionManager)
controllerApisHandlerPool = new KafkaRequestHandlerPool(config.nodeId,
socketServer.dataPlaneRequestChannel,
@ -305,6 +313,9 @@ class ControllerServer(
// Set up the metadata features publisher.
metadataPublishers.add(featuresPublisher)
// Set up the controller registrations publisher.
metadataPublishers.add(controllerRegistrationsPublisher)
// Set up the dynamic config publisher. This runs even in combined mode, since the broker
// has its own separate dynamic configuration object.
metadataPublishers.add(new DynamicConfigPublisher(

View File

@ -27,7 +27,7 @@ import kafka.utils.Implicits._
import kafka.utils.{CoreUtils, Logging}
import org.apache.kafka.admin.AdminUtils
import org.apache.kafka.clients.admin.AlterConfigOp.OpType
import org.apache.kafka.clients.admin.{AlterConfigOp, ConfigEntry}
import org.apache.kafka.clients.admin.{AlterConfigOp, ConfigEntry, EndpointType}
import org.apache.kafka.common.acl.AclOperation._
import org.apache.kafka.common.acl.AclOperation
import org.apache.kafka.common.config.ConfigResource
@ -3449,43 +3449,34 @@ class KafkaApis(val requestChannel: RequestChannel,
}
def handleDescribeCluster(request: RequestChannel.Request): Unit = {
val describeClusterRequest = request.body[DescribeClusterRequest]
var clusterAuthorizedOperations = Int.MinValue // Default value in the schema
// get cluster authorized operations
if (describeClusterRequest.data.includeClusterAuthorizedOperations) {
if (authHelper.authorize(request.context, DESCRIBE, CLUSTER, CLUSTER_NAME))
clusterAuthorizedOperations = authHelper.authorizedOperations(request, Resource.CLUSTER)
else
clusterAuthorizedOperations = 0
val response = authHelper.computeDescribeClusterResponse(
request,
EndpointType.BROKER,
clusterId,
() => {
val brokers = new DescribeClusterResponseData.DescribeClusterBrokerCollection()
metadataCache.getAliveBrokerNodes(request.context.listenerName).foreach { node =>
brokers.add(new DescribeClusterResponseData.DescribeClusterBroker().
setBrokerId(node.id).
setHost(node.host).
setPort(node.port).
setRack(node.rack))
}
val brokers = metadataCache.getAliveBrokerNodes(request.context.listenerName)
val controllerId = {
metadataCache.getControllerId.flatMap {
case ZkCachedControllerId(id) => Some(id)
case KRaftCachedControllerId(_) => metadataCache.getRandomAliveBrokerId
brokers
},
() => {
metadataCache.getControllerId match {
case Some(value) =>
value match {
case ZkCachedControllerId (id) => id
case KRaftCachedControllerId (_) => metadataCache.getRandomAliveBrokerId.getOrElse(- 1)
}
case None => -1
}
}
requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => {
val data = new DescribeClusterResponseData()
.setThrottleTimeMs(requestThrottleMs)
.setClusterId(clusterId)
.setControllerId(controllerId.getOrElse(MetadataResponse.NO_CONTROLLER_ID))
.setClusterAuthorizedOperations(clusterAuthorizedOperations)
brokers.foreach { broker =>
data.brokers.add(new DescribeClusterResponseData.DescribeClusterBroker()
.setBrokerId(broker.id)
.setHost(broker.host)
.setPort(broker.port)
.setRack(broker.rack))
}
new DescribeClusterResponse(data)
})
)
requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
new DescribeClusterResponse(response.setThrottleTimeMs(requestThrottleMs)))
}
def handleEnvelope(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = {

View File

@ -117,6 +117,6 @@ public class ClusterTestExtensionsTest {
@ClusterTest
public void testDefaults(ClusterConfig config) {
Assertions.assertEquals(MetadataVersion.IBP_3_6_IV1, config.metadataVersion());
Assertions.assertEquals(MetadataVersion.IBP_3_6_IV2, config.metadataVersion());
}
}

View File

@ -34,6 +34,7 @@ import org.apache.kafka.common.message.AlterUserScramCredentialsRequestData;
import org.apache.kafka.common.message.AlterUserScramCredentialsResponseData;
import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
import org.apache.kafka.common.message.BrokerRegistrationRequestData;
import org.apache.kafka.common.message.ControllerRegistrationRequestData;
import org.apache.kafka.common.message.CreatePartitionsRequestData.CreatePartitionsTopic;
import org.apache.kafka.common.message.CreatePartitionsResponseData.CreatePartitionsTopicResult;
import org.apache.kafka.common.message.CreateTopicsRequestData;
@ -480,6 +481,14 @@ public class MockController implements Controller {
return CompletableFuture.completedFuture(results);
}
@Override
public CompletableFuture<Void> registerController(
ControllerRequestContext context,
ControllerRegistrationRequestData request
) {
throw new UnsupportedOperationException();
}
@Override
public void beginShutdown() {
this.active = false;

View File

@ -41,6 +41,6 @@ public @interface ClusterTest {
String name() default "";
SecurityProtocol securityProtocol() default SecurityProtocol.PLAINTEXT;
String listener() default "";
MetadataVersion metadataVersion() default MetadataVersion.IBP_3_6_IV1;
MetadataVersion metadataVersion() default MetadataVersion.IBP_3_6_IV2;
ClusterConfigProperty[] serverProperties() default {};
}

View File

@ -12,14 +12,14 @@
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
log4j.rootLogger=OFF, stdout
log4j.rootLogger=DEBUG, stdout
log4j.appender.stdout=org.apache.log4j.ConsoleAppender
log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
log4j.appender.stdout.layout.ConversionPattern=[%d] %p %m (%c:%L)%n
log4j.logger.kafka=WARN
log4j.logger.org.apache.kafka=WARN
log4j.logger.kafka=DEBUG
log4j.logger.org.apache.kafka=DEBUG
# zkclient can be verbose, during debugging it is common to adjust it separately

View File

@ -24,6 +24,7 @@ import org.apache.kafka.metadata.BrokerRegistration
import org.junit.jupiter.api.Assertions.{assertFalse, assertTrue}
import org.junit.jupiter.api.Test
import java.util.Collections
import scala.jdk.CollectionConverters._
class MigrationPropagatorTest {
@ -39,7 +40,7 @@ class MigrationPropagatorTest {
def brokersToClusterImage(brokers: Seq[BrokerRegistration]): ClusterImage = {
val brokerMap = brokers.map(broker => Integer.valueOf(broker.id()) -> broker).toMap.asJava
new ClusterImage(brokerMap)
new ClusterImage(brokerMap, Collections.emptyMap)
}
@Test

View File

@ -18,28 +18,15 @@
package kafka.server
import java.util.{Collections, Properties}
import java.util.concurrent.atomic.{AtomicLong, AtomicReference}
import kafka.utils.TestUtils
import org.apache.kafka.clients.{Metadata, MockClient, NodeApiVersions}
import org.apache.kafka.common.config.SaslConfigs
import org.apache.kafka.common.Node
import org.apache.kafka.common.internals.ClusterResourceListeners
import org.apache.kafka.common.message.ApiVersionsResponseData.ApiVersion
import org.apache.kafka.common.message.BrokerRegistrationRequestData.{Listener, ListenerCollection}
import org.apache.kafka.common.message.{BrokerHeartbeatResponseData, BrokerRegistrationResponseData}
import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.protocol.ApiKeys.{BROKER_HEARTBEAT, BROKER_REGISTRATION}
import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.requests.{AbstractRequest, BrokerHeartbeatRequest, BrokerHeartbeatResponse, BrokerRegistrationResponse}
import org.apache.kafka.common.security.auth.SecurityProtocol
import org.apache.kafka.common.utils.LogContext
import org.apache.kafka.metadata.BrokerState
import org.apache.kafka.server.util.MockTime
import org.junit.jupiter.api.{Test, Timeout}
import org.junit.jupiter.api.Assertions._
import scala.jdk.CollectionConverters._
@Timeout(value = 12)
class BrokerLifecycleManagerTest {
@ -54,57 +41,16 @@ class BrokerLifecycleManagerTest {
properties
}
class SimpleControllerNodeProvider extends ControllerNodeProvider {
val node = new AtomicReference[Node](null)
def listenerName: ListenerName = new ListenerName("PLAINTEXT")
def securityProtocol: SecurityProtocol = SecurityProtocol.PLAINTEXT
def saslMechanism: String = SaslConfigs.DEFAULT_SASL_MECHANISM
override def getControllerInfo(): ControllerInformation = ControllerInformation(Option(node.get()),
listenerName, securityProtocol, saslMechanism, isZkController = false)
}
class BrokerLifecycleManagerTestContext(properties: Properties) {
val config = new KafkaConfig(properties)
val time = new MockTime(1, 1)
val highestMetadataOffset = new AtomicLong(0)
val metadata = new Metadata(1000, 1000, new LogContext(), new ClusterResourceListeners())
val mockClient = new MockClient(time, metadata)
val controllerNodeProvider = new SimpleControllerNodeProvider()
val nodeApiVersions = NodeApiVersions.create(Seq(BROKER_REGISTRATION, BROKER_HEARTBEAT).map {
apiKey => new ApiVersion().setApiKey(apiKey.id).
setMinVersion(apiKey.oldestVersion()).setMaxVersion(apiKey.latestVersion())
}.toList.asJava)
val mockChannelManager = new MockBrokerToControllerChannelManager(mockClient,
time, controllerNodeProvider, nodeApiVersions)
val clusterId = "x4AJGXQSRnephtTZzujw4w"
val advertisedListeners = new ListenerCollection()
config.effectiveAdvertisedListeners.foreach { ep =>
advertisedListeners.add(new Listener().setHost(ep.host).
setName(ep.listenerName.value()).
setPort(ep.port.shortValue()).
setSecurityProtocol(ep.securityProtocol.id))
}
def poll(): Unit = {
mockClient.wakeup()
mockChannelManager.poll()
}
}
@Test
def testCreateAndClose(): Unit = {
val context = new BrokerLifecycleManagerTestContext(configProperties)
val context = new RegistrationTestContext(configProperties)
val manager = new BrokerLifecycleManager(context.config, context.time, "create-and-close-", isZkBroker = false)
manager.close()
}
@Test
def testCreateStartAndClose(): Unit = {
val context = new BrokerLifecycleManagerTestContext(configProperties)
val context = new RegistrationTestContext(configProperties)
val manager = new BrokerLifecycleManager(context.config, context.time, "create-start-and-close-", isZkBroker = false)
assertEquals(BrokerState.NOT_RUNNING, manager.state)
manager.start(() => context.highestMetadataOffset.get(),
@ -119,7 +65,7 @@ class BrokerLifecycleManagerTest {
@Test
def testSuccessfulRegistration(): Unit = {
val context = new BrokerLifecycleManagerTestContext(configProperties)
val context = new RegistrationTestContext(configProperties)
val manager = new BrokerLifecycleManager(context.config, context.time, "successful-registration-", isZkBroker = false)
val controllerNode = new Node(3000, "localhost", 8021)
context.controllerNodeProvider.node.set(controllerNode)
@ -138,7 +84,7 @@ class BrokerLifecycleManagerTest {
@Test
def testRegistrationTimeout(): Unit = {
val context = new BrokerLifecycleManagerTestContext(configProperties)
val context = new RegistrationTestContext(configProperties)
val controllerNode = new Node(3000, "localhost", 8021)
val manager = new BrokerLifecycleManager(context.config, context.time, "registration-timeout-", isZkBroker = false)
context.controllerNodeProvider.node.set(controllerNode)
@ -180,7 +126,7 @@ class BrokerLifecycleManagerTest {
@Test
def testControlledShutdown(): Unit = {
val context = new BrokerLifecycleManagerTestContext(configProperties)
val context = new RegistrationTestContext(configProperties)
val manager = new BrokerLifecycleManager(context.config, context.time, "controlled-shutdown-", isZkBroker = false)
val controllerNode = new Node(3000, "localhost", 8021)
context.controllerNodeProvider.node.set(controllerNode)

View File

@ -46,10 +46,11 @@ import org.apache.kafka.common.protocol.{ApiKeys, ApiMessage, Errors}
import org.apache.kafka.common.requests._
import org.apache.kafka.common.resource.{PatternType, Resource, ResourcePattern, ResourceType}
import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol}
import org.apache.kafka.common.utils.MockTime;
import org.apache.kafka.common.utils.MockTime
import org.apache.kafka.common.{ElectionType, Uuid}
import org.apache.kafka.controller.ControllerRequestContextUtil.ANONYMOUS_CONTEXT
import org.apache.kafka.controller.{Controller, ControllerRequestContext, ResultOrError}
import org.apache.kafka.image.publisher.ControllerRegistrationsPublisher
import org.apache.kafka.server.authorizer.{Action, AuthorizableRequestContext, AuthorizationResult, Authorizer}
import org.apache.kafka.server.common.{ApiMessageAndVersion, Features, MetadataVersion, ProducerIdsBlock}
import org.junit.jupiter.api.Assertions._
@ -154,7 +155,7 @@ class ControllerApisTest {
raftManager,
new KafkaConfig(props),
MetaProperties("JgxuGe9URy-E-ceaL04lEw", nodeId = nodeId),
Seq.empty,
new ControllerRegistrationsPublisher(),
new SimpleApiVersionManager(
ListenerType.CONTROLLER,
true,

View File

@ -0,0 +1,238 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.server
import org.apache.kafka.common.{Node, Uuid}
import org.apache.kafka.common.message.ControllerRegistrationResponseData
import org.apache.kafka.common.metadata.{FeatureLevelRecord, RegisterControllerRecord}
import org.apache.kafka.common.requests.ControllerRegistrationResponse
import org.apache.kafka.image.loader.{LogDeltaManifest, SnapshotManifest}
import org.apache.kafka.image.{MetadataDelta, MetadataImage, MetadataProvenance}
import org.apache.kafka.metadata.{RecordTestUtils, VersionRange}
import org.apache.kafka.raft.LeaderAndEpoch
import org.apache.kafka.server.common.MetadataVersion
import org.apache.kafka.test.TestUtils
import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertTrue}
import org.junit.jupiter.api.{Test, Timeout}
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.ValueSource
import java.util
import java.util.{OptionalInt, Properties}
import java.util.concurrent.{CompletableFuture, TimeUnit}
@Timeout(value = 60)
class ControllerRegistrationManagerTest {
private val controller1 = new Node(1, "localhost", 7000)
private def configProperties = {
val properties = new Properties()
properties.setProperty(KafkaConfig.LogDirsProp, "/tmp/foo")
properties.setProperty(KafkaConfig.ProcessRolesProp, "controller")
properties.setProperty(KafkaConfig.ListenerSecurityProtocolMapProp, s"CONTROLLER:PLAINTEXT")
properties.setProperty(KafkaConfig.ListenersProp, s"CONTROLLER://localhost:0")
properties.setProperty(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER")
properties.setProperty(KafkaConfig.NodeIdProp, "1")
properties.setProperty(KafkaConfig.QuorumVotersProp, s"1@localhost:8000,2@localhost:5000,3@localhost:7000")
properties
}
private def createSupportedFeatures(
highestSupportedMetadataVersion: MetadataVersion
): java.util.Map[String, VersionRange] = {
val results = new util.HashMap[String, VersionRange]()
results.put(MetadataVersion.FEATURE_NAME, VersionRange.of(
MetadataVersion.MINIMUM_KRAFT_VERSION.featureLevel(),
highestSupportedMetadataVersion.featureLevel()))
results
}
private def newControllerRegistrationManager(
context: RegistrationTestContext,
): ControllerRegistrationManager = {
new ControllerRegistrationManager(context.config,
context.clusterId,
context.time,
"controller-registration-manager-test-",
createSupportedFeatures(MetadataVersion.IBP_3_6_IV2),
() => context.controllerEpoch.get(),
RecordTestUtils.createTestControllerRegistration(1, false).incarnationId())
}
private def registered(manager: ControllerRegistrationManager): Boolean = {
val registered = new CompletableFuture[Boolean]
manager.eventQueue.append(() => {
registered.complete(manager.registered)
})
registered.get(30, TimeUnit.SECONDS)
}
private def rpcStats(manager: ControllerRegistrationManager): (Long, Long, Long) = {
val failedAttempts = new CompletableFuture[(Long, Long, Long)]
manager.eventQueue.append(() => {
failedAttempts.complete((manager.pendingRpcs, manager.successfulRpcs, manager.failedRpcs))
})
failedAttempts.get(30, TimeUnit.SECONDS)
}
private def doMetadataUpdate(
prevImage: MetadataImage,
manager: ControllerRegistrationManager,
metadataVersion: MetadataVersion,
registrationModifier: RegisterControllerRecord => Option[RegisterControllerRecord]
): MetadataImage = {
val delta = new MetadataDelta.Builder().
setImage(prevImage).
build()
if (!prevImage.features().metadataVersion().equals(metadataVersion)) {
delta.replay(new FeatureLevelRecord().
setName(MetadataVersion.FEATURE_NAME).
setFeatureLevel(metadataVersion.featureLevel()))
}
if (metadataVersion.isControllerRegistrationSupported) {
for (i <- Seq(1, 2, 3)) {
registrationModifier(RecordTestUtils.createTestControllerRegistration(i, false)).foreach {
registration => delta.replay(registration)
}
}
}
val provenance = new MetadataProvenance(100, 200, 300)
val newImage = delta.apply(provenance)
val manifest = if (!prevImage.features().metadataVersion().equals(metadataVersion)) {
new SnapshotManifest(provenance, 1000)
} else {
new LogDeltaManifest(provenance,
new LeaderAndEpoch(OptionalInt.of(1), 100),
1,
100,
200)
}
manager.onMetadataUpdate(delta, newImage, manifest)
newImage
}
@Test
def testCreateAndClose(): Unit = {
val context = new RegistrationTestContext(configProperties)
val manager = newControllerRegistrationManager(context)
assertFalse(registered(manager))
assertEquals((0, 0, 0), rpcStats(manager))
manager.close()
}
@Test
def testCreateStartAndClose(): Unit = {
val context = new RegistrationTestContext(configProperties)
val manager = newControllerRegistrationManager(context)
try {
manager.start(context.mockChannelManager)
assertFalse(registered(manager))
assertEquals((0, 0, 0), rpcStats(manager))
} finally {
manager.close()
}
}
@ParameterizedTest
@ValueSource(booleans = Array(false, true))
def testRegistration(metadataVersionSupportsRegistration: Boolean): Unit = {
val context = new RegistrationTestContext(configProperties)
val metadataVersion = if (metadataVersionSupportsRegistration) {
MetadataVersion.IBP_3_6_IV2
} else {
MetadataVersion.IBP_3_6_IV0
}
val manager = newControllerRegistrationManager(context)
try {
if (!metadataVersionSupportsRegistration) {
context.mockClient.prepareUnsupportedVersionResponse(_ => true)
} else {
context.controllerNodeProvider.node.set(controller1)
}
manager.start(context.mockChannelManager)
assertFalse(registered(manager))
assertEquals((0, 0, 0), rpcStats(manager))
val image = doMetadataUpdate(MetadataImage.EMPTY,
manager,
metadataVersion,
r => if (r.controllerId() == 1) None else Some(r))
if (!metadataVersionSupportsRegistration) {
assertFalse(registered(manager))
assertEquals((0, 0, 0), rpcStats(manager))
} else {
TestUtils.retryOnExceptionWithTimeout(30000, () => {
assertEquals((1, 0, 0), rpcStats(manager))
})
context.mockClient.prepareResponseFrom(new ControllerRegistrationResponse(
new ControllerRegistrationResponseData()), controller1)
TestUtils.retryOnExceptionWithTimeout(30000, () => {
context.mockChannelManager.poll()
assertEquals((0, 1, 0), rpcStats(manager))
})
assertFalse(registered(manager))
doMetadataUpdate(image,
manager,
metadataVersion,
r => Some(r))
assertTrue(registered(manager))
}
} finally {
manager.close()
}
}
@Test
def testWrongIncarnationId(): Unit = {
val context = new RegistrationTestContext(configProperties)
val manager = newControllerRegistrationManager(context)
try {
context.controllerNodeProvider.node.set(controller1)
manager.start(context.mockChannelManager)
context.mockClient.prepareResponseFrom(new ControllerRegistrationResponse(
new ControllerRegistrationResponseData()), controller1)
var image = doMetadataUpdate(MetadataImage.EMPTY,
manager,
MetadataVersion.IBP_3_6_IV2,
r => if (r.controllerId() == 1) None else Some(r))
TestUtils.retryOnExceptionWithTimeout(30000, () => {
context.mockChannelManager.poll()
assertEquals((0, 1, 0), rpcStats(manager))
})
image = doMetadataUpdate(image,
manager,
MetadataVersion.IBP_3_6_IV2,
r => Some(r.setIncarnationId(new Uuid(456, r.controllerId()))))
TestUtils.retryOnExceptionWithTimeout(30000, () => {
assertEquals((1, 1, 0), rpcStats(manager))
})
context.mockClient.prepareResponseFrom(new ControllerRegistrationResponse(
new ControllerRegistrationResponseData()), controller1)
doMetadataUpdate(image,
manager,
MetadataVersion.IBP_3_6_IV2,
r => Some(r))
TestUtils.retryOnExceptionWithTimeout(30000, () => {
context.mockChannelManager.poll()
assertEquals((0, 2, 0), rpcStats(manager))
assertTrue(registered(manager))
})
} finally {
manager.close()
}
}
}

View File

@ -0,0 +1,78 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.server
import org.apache.kafka.clients.{Metadata, MockClient, NodeApiVersions}
import org.apache.kafka.common.config.SaslConfigs
import org.apache.kafka.common.Node
import org.apache.kafka.common.internals.ClusterResourceListeners
import org.apache.kafka.common.message.ApiVersionsResponseData.ApiVersion
import org.apache.kafka.common.message.BrokerRegistrationRequestData.{Listener, ListenerCollection}
import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.protocol.ApiKeys.{BROKER_HEARTBEAT, BROKER_REGISTRATION}
import org.apache.kafka.common.security.auth.SecurityProtocol
import org.apache.kafka.common.utils.LogContext
import org.apache.kafka.server.util.MockTime
import java.util.Properties
import java.util.concurrent.atomic.{AtomicInteger, AtomicLong, AtomicReference}
import scala.jdk.CollectionConverters._
class SimpleControllerNodeProvider extends ControllerNodeProvider {
val node = new AtomicReference[Node](null)
def listenerName: ListenerName = new ListenerName("PLAINTEXT")
def securityProtocol: SecurityProtocol = SecurityProtocol.PLAINTEXT
def saslMechanism: String = SaslConfigs.DEFAULT_SASL_MECHANISM
override def getControllerInfo(): ControllerInformation = ControllerInformation(Option(node.get()),
listenerName, securityProtocol, saslMechanism, isZkController = false)
}
class RegistrationTestContext(
properties: Properties
) {
val config = new KafkaConfig(properties)
val time = new MockTime(1, 1)
val highestMetadataOffset = new AtomicLong(0)
val metadata = new Metadata(1000, 1000, new LogContext(), new ClusterResourceListeners())
val mockClient = new MockClient(time, metadata)
val controllerNodeProvider = new SimpleControllerNodeProvider()
val nodeApiVersions = NodeApiVersions.create(Seq(BROKER_REGISTRATION, BROKER_HEARTBEAT).map {
apiKey => new ApiVersion().setApiKey(apiKey.id).
setMinVersion(apiKey.oldestVersion()).setMaxVersion(apiKey.latestVersion())
}.toList.asJava)
val mockChannelManager = new MockBrokerToControllerChannelManager(mockClient,
time, controllerNodeProvider, nodeApiVersions)
val clusterId = "x4AJGXQSRnephtTZzujw4w"
val advertisedListeners = new ListenerCollection()
val controllerEpoch = new AtomicInteger(123)
config.effectiveAdvertisedListeners.foreach { ep =>
advertisedListeners.add(new Listener().setHost(ep.host).
setName(ep.listenerName.value()).
setPort(ep.port.shortValue()).
setSecurityProtocol(ep.securityProtocol.id))
}
def poll(): Unit = {
mockClient.wakeup()
mockChannelManager.poll()
}
}

View File

@ -25,11 +25,15 @@ import org.apache.kafka.common.errors.InconsistentClusterIdException;
import org.apache.kafka.common.errors.StaleBrokerEpochException;
import org.apache.kafka.common.errors.UnsupportedVersionException;
import org.apache.kafka.common.message.BrokerRegistrationRequestData;
import org.apache.kafka.common.message.ControllerRegistrationRequestData;
import org.apache.kafka.common.metadata.BrokerRegistrationChangeRecord;
import org.apache.kafka.common.metadata.FenceBrokerRecord;
import org.apache.kafka.common.metadata.RegisterBrokerRecord;
import org.apache.kafka.common.metadata.RegisterBrokerRecord.BrokerEndpoint;
import org.apache.kafka.common.metadata.RegisterBrokerRecord.BrokerFeature;
import org.apache.kafka.common.metadata.RegisterControllerRecord;
import org.apache.kafka.common.metadata.RegisterControllerRecord.ControllerEndpointCollection;
import org.apache.kafka.common.metadata.RegisterControllerRecord.ControllerFeatureCollection;
import org.apache.kafka.common.metadata.UnfenceBrokerRecord;
import org.apache.kafka.common.metadata.UnregisterBrokerRecord;
import org.apache.kafka.common.protocol.ApiMessage;
@ -40,6 +44,7 @@ import org.apache.kafka.metadata.BrokerRegistration;
import org.apache.kafka.metadata.BrokerRegistrationFencingChange;
import org.apache.kafka.metadata.BrokerRegistrationInControlledShutdownChange;
import org.apache.kafka.metadata.BrokerRegistrationReply;
import org.apache.kafka.metadata.ControllerRegistration;
import org.apache.kafka.metadata.FinalizedControllerFeatures;
import org.apache.kafka.metadata.VersionRange;
import org.apache.kafka.metadata.placement.ReplicaPlacer;
@ -51,6 +56,7 @@ import org.apache.kafka.timeline.SnapshotRegistry;
import org.apache.kafka.timeline.TimelineHashMap;
import org.slf4j.Logger;
import java.util.AbstractMap;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.Iterator;
@ -241,6 +247,11 @@ public class ClusterControlManager {
*/
private final boolean zkMigrationEnabled;
/**
* Maps controller IDs to controller registrations.
*/
private final TimelineHashMap<Integer, ControllerRegistration> controllerRegistrations;
private ClusterControlManager(
LogContext logContext,
String clusterId,
@ -263,6 +274,7 @@ public class ClusterControlManager {
this.readyBrokersFuture = Optional.empty();
this.featureControl = featureControl;
this.zkMigrationEnabled = zkMigrationEnabled;
this.controllerRegistrations = new TimelineHashMap<>(snapshotRegistry, 0);
}
ReplicaPlacer replicaPlacer() {
@ -382,6 +394,37 @@ public class ClusterControlManager {
return ControllerResult.atomicOf(records, new BrokerRegistrationReply(brokerEpoch));
}
ControllerResult<Void> registerController(ControllerRegistrationRequestData request) {
if (!featureControl.metadataVersion().isControllerRegistrationSupported()) {
throw new UnsupportedVersionException("The current MetadataVersion is too old to " +
"support controller registrations.");
}
ControllerEndpointCollection endpoints = new ControllerEndpointCollection();
request.listeners().forEach(listener -> {
endpoints.add(new RegisterControllerRecord.ControllerEndpoint().
setHost(listener.host()).
setName(listener.name()).
setPort(listener.port()).
setSecurityProtocol(listener.securityProtocol()));
});
ControllerFeatureCollection features = new ControllerFeatureCollection();
request.features().forEach(feature -> {
features.add(new RegisterControllerRecord.ControllerFeature().
setName(feature.name()).
setMaxSupportedVersion(feature.maxSupportedVersion()).
setMinSupportedVersion(feature.minSupportedVersion()));
});
List<ApiMessageAndVersion> records = new ArrayList<>();
records.add(new ApiMessageAndVersion(new RegisterControllerRecord().
setControllerId(request.controllerId()).
setIncarnationId(request.incarnationId()).
setZkMigrationReady(request.zkMigrationReady()).
setEndPoints(endpoints).
setFeatures(features),
(short) 0));
return ControllerResult.atomicOf(records, null);
}
BrokerFeature processRegistrationFeature(
int brokerId,
FinalizedControllerFeatures finalizedFeatures,
@ -541,6 +584,15 @@ public class ClusterControlManager {
}
}
public void replay(RegisterControllerRecord record) {
ControllerRegistration newRegistration = new ControllerRegistration.Builder(record).build();
ControllerRegistration prevRegistration =
controllerRegistrations.put(record.controllerId(), newRegistration);
log.info("Replayed RegisterControllerRecord contaning {}.{}", newRegistration,
prevRegistration == null ? "" :
" Previous incarnation was " + prevRegistration.incarnationId());
}
Iterator<UsableBroker> usableBrokers() {
if (heartbeatManager == null) {
throw new RuntimeException("ClusterControlManager is not active.");
@ -615,4 +667,44 @@ public class ClusterControlManager {
readyBrokersFuture = Optional.empty();
}
}
Iterator<Entry<Integer, Map<String, VersionRange>>> brokerSupportedFeatures() {
return new Iterator<Entry<Integer, Map<String, VersionRange>>>() {
private final Iterator<BrokerRegistration> iter = brokerRegistrations.values().iterator();
@Override
public boolean hasNext() {
return iter.hasNext();
}
@Override
public Entry<Integer, Map<String, VersionRange>> next() {
BrokerRegistration registration = iter.next();
return new AbstractMap.SimpleImmutableEntry<>(registration.id(),
registration.supportedFeatures());
}
};
}
Iterator<Entry<Integer, Map<String, VersionRange>>> controllerSupportedFeatures() {
if (!featureControl.metadataVersion().isControllerRegistrationSupported()) {
throw new UnsupportedVersionException("The current MetadataVersion is too old to " +
"support controller registrations.");
}
return new Iterator<Entry<Integer, Map<String, VersionRange>>>() {
private final Iterator<ControllerRegistration> iter = controllerRegistrations.values().iterator();
@Override
public boolean hasNext() {
return iter.hasNext();
}
@Override
public Entry<Integer, Map<String, VersionRange>> next() {
ControllerRegistration registration = iter.next();
return new AbstractMap.SimpleImmutableEntry<>(registration.id(),
registration.supportedFeatures());
}
};
}
}

View File

@ -0,0 +1,30 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.controller;
import java.util.Iterator;
import java.util.Map.Entry;
import java.util.Map;
import org.apache.kafka.metadata.VersionRange;
public interface ClusterSupportDescriber {
Iterator<Entry<Integer, Map<String, VersionRange>>> brokerSupported();
Iterator<Entry<Integer, Map<String, VersionRange>>> controllerSupported();
}

View File

@ -30,6 +30,7 @@ import org.apache.kafka.common.message.AlterUserScramCredentialsRequestData;
import org.apache.kafka.common.message.AlterUserScramCredentialsResponseData;
import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
import org.apache.kafka.common.message.BrokerRegistrationRequestData;
import org.apache.kafka.common.message.ControllerRegistrationRequestData;
import org.apache.kafka.common.message.CreatePartitionsRequestData.CreatePartitionsTopic;
import org.apache.kafka.common.message.CreatePartitionsResponseData.CreatePartitionsTopicResult;
import org.apache.kafka.common.message.CreateTopicsRequestData;
@ -345,6 +346,19 @@ public interface Controller extends AclMutator, AutoCloseable {
boolean validateOnly
);
/**
* Attempt to register the given controller.
*
* @param context The controller request context.
* @param request The registration request.
*
* @return A future yielding the broker registration reply.
*/
CompletableFuture<Void> registerController(
ControllerRequestContext context,
ControllerRegistrationRequestData request
);
/**
* Begin shutting down, but don't block. You must still call close to clean up all
* resources.

View File

@ -19,6 +19,8 @@ package org.apache.kafka.controller;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map.Entry;
import java.util.Map;
@ -26,7 +28,6 @@ import java.util.Optional;
import java.util.TreeMap;
import java.util.function.Consumer;
import org.apache.kafka.clients.ApiVersions;
import org.apache.kafka.clients.admin.FeatureUpdate;
import org.apache.kafka.common.metadata.FeatureLevelRecord;
import org.apache.kafka.common.metadata.ZkMigrationStateRecord;
@ -49,13 +50,23 @@ import static org.apache.kafka.controller.QuorumController.MAX_RECORDS_PER_USER_
public class FeatureControlManager {
public static class Builder {
private LogContext logContext = null;
private SnapshotRegistry snapshotRegistry = null;
private QuorumFeatures quorumFeatures = null;
private MetadataVersion metadataVersion = MetadataVersion.latest();
private MetadataVersion minimumBootstrapVersion = MetadataVersion.MINIMUM_BOOTSTRAP_VERSION;
private ClusterSupportDescriber clusterSupportDescriber = new ClusterSupportDescriber() {
@Override
public Iterator<Entry<Integer, Map<String, VersionRange>>> brokerSupported() {
return Collections.<Integer, Map<String, VersionRange>>emptyMap().entrySet().iterator();
}
@Override
public Iterator<Entry<Integer, Map<String, VersionRange>>> controllerSupported() {
return Collections.<Integer, Map<String, VersionRange>>emptyMap().entrySet().iterator();
}
};
Builder setLogContext(LogContext logContext) {
this.logContext = logContext;
@ -82,19 +93,28 @@ public class FeatureControlManager {
return this;
}
Builder setClusterSupportDescriber(ClusterSupportDescriber clusterSupportDescriber) {
this.clusterSupportDescriber = clusterSupportDescriber;
return this;
}
public FeatureControlManager build() {
if (logContext == null) logContext = new LogContext();
if (snapshotRegistry == null) snapshotRegistry = new SnapshotRegistry(logContext);
if (quorumFeatures == null) {
quorumFeatures = new QuorumFeatures(0, new ApiVersions(), QuorumFeatures.defaultFeatureMap(),
Collections.emptyList());
Map<String, VersionRange> localSupportedFeatures = new HashMap<>();
localSupportedFeatures.put(MetadataVersion.FEATURE_NAME, VersionRange.of(
MetadataVersion.MINIMUM_KRAFT_VERSION.featureLevel(),
MetadataVersion.latest().featureLevel()));
quorumFeatures = new QuorumFeatures(0, localSupportedFeatures, Collections.singletonList(0));
}
return new FeatureControlManager(
logContext,
quorumFeatures,
snapshotRegistry,
metadataVersion,
minimumBootstrapVersion
minimumBootstrapVersion,
clusterSupportDescriber
);
}
}
@ -126,12 +146,18 @@ public class FeatureControlManager {
*/
private final MetadataVersion minimumBootstrapVersion;
/**
* Gives information about the supported versions in the cluster.
*/
private final ClusterSupportDescriber clusterSupportDescriber;
private FeatureControlManager(
LogContext logContext,
QuorumFeatures quorumFeatures,
SnapshotRegistry snapshotRegistry,
MetadataVersion metadataVersion,
MetadataVersion minimumBootstrapVersion
MetadataVersion minimumBootstrapVersion,
ClusterSupportDescriber clusterSupportDescriber
) {
this.log = logContext.logger(FeatureControlManager.class);
this.quorumFeatures = quorumFeatures;
@ -139,6 +165,7 @@ public class FeatureControlManager {
this.metadataVersion = new TimelineObject<>(snapshotRegistry, metadataVersion);
this.minimumBootstrapVersion = minimumBootstrapVersion;
this.migrationControlState = new TimelineObject<>(snapshotRegistry, ZkMigrationState.NONE);
this.clusterSupportDescriber = clusterSupportDescriber;
}
ControllerResult<Map<String, ApiError>> updateFeatures(
@ -194,23 +221,11 @@ public class FeatureControlManager {
"A feature version cannot be less than 0.");
}
Optional<String> reasonNotSupported = quorumFeatures.reasonNotSupported(featureName, newVersion);
Optional<String> reasonNotSupported = reasonNotSupported(featureName, newVersion);
if (reasonNotSupported.isPresent()) {
return invalidUpdateVersion(featureName, newVersion, reasonNotSupported.get());
}
for (Entry<Integer, Map<String, VersionRange>> brokerEntry : brokersAndFeatures.entrySet()) {
VersionRange brokerRange = brokerEntry.getValue().get(featureName);
if (brokerRange == null) {
return invalidUpdateVersion(featureName, newVersion,
"Broker " + brokerEntry.getKey() + " does not support this feature.");
} else if (!brokerRange.contains(newVersion)) {
return invalidUpdateVersion(featureName, newVersion,
"Broker " + brokerEntry.getKey() + " does not support the given " +
"version. It supports " + brokerRange.min() + " to " + brokerRange.max() + ".");
}
}
if (newVersion < currentVersion) {
if (upgradeType.equals(FeatureUpdate.UpgradeType.UPGRADE)) {
return invalidUpdateVersion(featureName, newVersion,
@ -234,9 +249,64 @@ public class FeatureControlManager {
}
}
private Optional<String> reasonNotSupported(
String featureName,
short newVersion
) {
int numBrokersChecked = 0;
int numControllersChecked = 0;
Optional<String> reason = quorumFeatures.reasonNotLocallySupported(featureName, newVersion);
if (reason.isPresent()) return reason;
numControllersChecked++;
for (Iterator<Entry<Integer, Map<String, VersionRange>>> iter =
clusterSupportDescriber.brokerSupported();
iter.hasNext(); ) {
Entry<Integer, Map<String, VersionRange>> entry = iter.next();
reason = QuorumFeatures.reasonNotSupported(newVersion,
"broker" + entry.getKey(),
entry.getValue().getOrDefault(featureName, QuorumFeatures.DISABLED));
if (reason.isPresent()) return reason;
numBrokersChecked++;
}
String registrationSuffix = "";
HashSet<Integer> foundControllers = new HashSet<>();
foundControllers.add(quorumFeatures.nodeId());
if (metadataVersion.get().isControllerRegistrationSupported()) {
for (Iterator<Entry<Integer, Map<String, VersionRange>>> iter =
clusterSupportDescriber.controllerSupported();
iter.hasNext(); ) {
Entry<Integer, Map<String, VersionRange>> entry = iter.next();
if (entry.getKey() == quorumFeatures.nodeId()) {
// No need to re-check the features supported by this controller, since we
// already checked that above.
continue;
}
reason = QuorumFeatures.reasonNotSupported(newVersion,
"controller " + entry.getKey(),
entry.getValue().getOrDefault(featureName, QuorumFeatures.DISABLED));
if (reason.isPresent()) return reason;
foundControllers.add(entry.getKey());
numControllersChecked++;
}
for (int id : quorumFeatures.quorumNodeIds()) {
if (!foundControllers.contains(id)) {
return Optional.of("controller " + id + " has not registered, and may not " +
"support this feature");
}
}
} else {
registrationSuffix = " Note: unable to verify controller support in the current " +
"MetadataVersion.";
}
log.info("Verified that {} broker(s) and {} controller(s) supported changing {} to " +
"feature level {}.{}", numBrokersChecked, numControllersChecked, featureName,
newVersion, registrationSuffix);
return Optional.empty();
}
private ApiError invalidUpdateVersion(String feature, short version, String message) {
String errorMessage = String.format("Invalid update version %d for feature %s. %s", version, feature, message);
log.debug(errorMessage);
log.warn(errorMessage);
return new ApiError(Errors.INVALID_UPDATE_VERSION, errorMessage);
}
@ -273,7 +343,7 @@ public class FeatureControlManager {
// This is a downgrade
boolean metadataChanged = MetadataVersion.checkIfMetadataChanged(currentVersion, newVersion);
if (!metadataChanged) {
log.info("Downgrading metadata.version from {} to {}.", currentVersion, newVersion);
log.warn("Downgrading metadata.version from {} to {}.", currentVersion, newVersion);
} else if (allowUnsafeDowngrade) {
return invalidMetadataVersion(newVersionLevel, "Unsafe metadata downgrade is not supported " +
"in this version.");
@ -283,7 +353,7 @@ public class FeatureControlManager {
"UNSAFE_DOWNGRADE if you want to force the downgrade to proceed.");
}
} else {
log.info("Upgrading metadata.version from {} to {}.", currentVersion, newVersion);
log.warn("Upgrading metadata.version from {} to {}.", currentVersion, newVersion);
}
recordConsumer.accept(new ApiMessageAndVersion(
@ -296,7 +366,7 @@ public class FeatureControlManager {
private ApiError invalidMetadataVersion(short version, String message) {
String errorMessage = String.format("Invalid metadata.version %d. %s", version, message);
log.error(errorMessage);
log.warn(errorMessage);
return new ApiError(Errors.INVALID_UPDATE_VERSION, errorMessage);
}

View File

@ -39,6 +39,7 @@ import org.apache.kafka.common.message.AlterUserScramCredentialsRequestData;
import org.apache.kafka.common.message.AlterUserScramCredentialsResponseData;
import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
import org.apache.kafka.common.message.BrokerRegistrationRequestData;
import org.apache.kafka.common.message.ControllerRegistrationRequestData;
import org.apache.kafka.common.message.CreatePartitionsRequestData.CreatePartitionsTopic;
import org.apache.kafka.common.message.CreatePartitionsResponseData.CreatePartitionsTopicResult;
import org.apache.kafka.common.message.CreateTopicsRequestData;
@ -64,6 +65,7 @@ import org.apache.kafka.common.metadata.PartitionChangeRecord;
import org.apache.kafka.common.metadata.PartitionRecord;
import org.apache.kafka.common.metadata.ProducerIdsRecord;
import org.apache.kafka.common.metadata.RegisterBrokerRecord;
import org.apache.kafka.common.metadata.RegisterControllerRecord;
import org.apache.kafka.common.metadata.RemoveAccessControlEntryRecord;
import org.apache.kafka.common.metadata.RemoveTopicRecord;
import org.apache.kafka.common.metadata.UserScramCredentialRecord;
@ -86,6 +88,7 @@ import org.apache.kafka.metadata.BrokerHeartbeatReply;
import org.apache.kafka.metadata.BrokerRegistrationReply;
import org.apache.kafka.metadata.FinalizedControllerFeatures;
import org.apache.kafka.metadata.KafkaConfigSchema;
import org.apache.kafka.metadata.VersionRange;
import org.apache.kafka.metadata.bootstrap.BootstrapMetadata;
import org.apache.kafka.metadata.migration.ZkMigrationState;
import org.apache.kafka.metadata.migration.ZkRecordConsumer;
@ -121,6 +124,7 @@ import java.util.Collection;
import java.util.Collections;
import java.util.EnumSet;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map.Entry;
import java.util.Map;
@ -424,6 +428,18 @@ public final class QuorumController implements Controller {
}
}
class QuorumClusterSupportDescriber implements ClusterSupportDescriber {
@Override
public Iterator<Entry<Integer, Map<String, VersionRange>>> brokerSupported() {
return clusterControl.brokerSupportedFeatures();
}
@Override
public Iterator<Entry<Integer, Map<String, VersionRange>>> controllerSupported() {
return clusterControl.controllerSupportedFeatures();
}
}
public static final String CONTROLLER_THREAD_SUFFIX = "QuorumControllerEventHandler";
private OptionalInt latestController() {
@ -1496,6 +1512,9 @@ public final class QuorumController implements Controller {
case ABORT_TRANSACTION_RECORD:
offsetControl.replay((AbortTransactionRecord) message, offset);
break;
case REGISTER_CONTROLLER_RECORD:
clusterControl.replay((RegisterControllerRecord) message);
break;
default:
throw new RuntimeException("Unhandled record type " + type);
}
@ -1575,6 +1594,11 @@ public final class QuorumController implements Controller {
*/
private final ClientQuotaControlManager clientQuotaControlManager;
/**
* Describes the feature versions in the cluster.
*/
private final QuorumClusterSupportDescriber clusterSupportDescriber;
/**
* An object which stores the controller's view of the cluster.
* This must be accessed only by the event queue thread.
@ -1741,6 +1765,7 @@ public final class QuorumController implements Controller {
setLogContext(logContext).
setSnapshotRegistry(snapshotRegistry).
build();
this.clusterSupportDescriber = new QuorumClusterSupportDescriber();
this.featureControl = new FeatureControlManager.Builder().
setLogContext(logContext).
setQuorumFeatures(quorumFeatures).
@ -1751,6 +1776,7 @@ public final class QuorumController implements Controller {
// are all treated as 3.0IV1. In newer versions the metadata.version will be specified
// by the log.
setMetadataVersion(MetadataVersion.MINIMUM_KRAFT_VERSION).
setClusterSupportDescriber(clusterSupportDescriber).
build();
this.clusterControl = new ClusterControlManager.Builder().
setLogContext(logContext).
@ -2131,6 +2157,16 @@ public final class QuorumController implements Controller {
});
}
@Override
public CompletableFuture<Void> registerController(
ControllerRequestContext context,
ControllerRegistrationRequestData request
) {
return appendWriteEvent("registerController", context.deadlineNs(),
() -> clusterControl.registerController(request),
EnumSet.of(RUNS_IN_PREMIGRATION));
}
@Override
public CompletableFuture<List<AclCreateResult>> createAcls(
ControllerRequestContext context,

View File

@ -17,57 +17,41 @@
package org.apache.kafka.controller;
import org.apache.kafka.clients.ApiVersions;
import org.apache.kafka.clients.NodeApiVersions;
import org.apache.kafka.common.Node;
import org.apache.kafka.common.feature.SupportedVersionRange;
import org.apache.kafka.metadata.ControllerRegistration;
import org.apache.kafka.metadata.VersionRange;
import org.apache.kafka.server.common.MetadataVersion;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Optional;
import java.util.stream.Collectors;
/**
* A holder class of the local node's supported feature flags as well as the ApiVersions of other nodes.
* A holder class of the local node's supported feature flags as well as the quorum node IDs.
*/
public class QuorumFeatures {
private static final VersionRange DISABLED = VersionRange.of(0, 0);
private static final Logger log = LoggerFactory.getLogger(QuorumFeatures.class);
public final class QuorumFeatures {
public static final VersionRange DISABLED = VersionRange.of(0, 0);
private final int nodeId;
private final ApiVersions apiVersions;
private final Map<String, VersionRange> localSupportedFeatures;
private final List<Integer> quorumNodeIds;
QuorumFeatures(
int nodeId,
ApiVersions apiVersions,
Map<String, VersionRange> localSupportedFeatures,
List<Integer> quorumNodeIds
static public Optional<String> reasonNotSupported(
short newVersion,
String what,
VersionRange range
) {
this.nodeId = nodeId;
this.apiVersions = apiVersions;
this.localSupportedFeatures = Collections.unmodifiableMap(localSupportedFeatures);
this.quorumNodeIds = Collections.unmodifiableList(quorumNodeIds);
if (!range.contains(newVersion)) {
if (range.max() == (short) 0) {
return Optional.of(what + " does not support this feature.");
} else {
return Optional.of(what + " only supports versions " + range);
}
public static QuorumFeatures create(
int nodeId,
ApiVersions apiVersions,
Map<String, VersionRange> localSupportedFeatures,
Collection<Node> quorumNodes
) {
List<Integer> nodeIds = quorumNodes.stream().map(Node::id).collect(Collectors.toList());
return new QuorumFeatures(nodeId, apiVersions, localSupportedFeatures, nodeIds);
}
return Optional.empty();
}
public static Map<String, VersionRange> defaultFeatureMap() {
@ -78,80 +62,92 @@ public class QuorumFeatures {
return features;
}
/**
* Return the reason a specific feature level is not supported, or Optional.empty if it is supported.
*
* @param featureName The feature name.
* @param level The feature level.
* @return The reason why the feature level is not supported, or Optional.empty if it is supported.
*/
public Optional<String> reasonNotSupported(String featureName, short level) {
VersionRange localRange = localSupportedFeatures.getOrDefault(featureName, DISABLED);
if (!localRange.contains(level)) {
if (localRange.equals(DISABLED)) {
return Optional.of("Local controller " + nodeId + " does not support this feature.");
} else {
return Optional.of("Local controller " + nodeId + " only supports versions " + localRange);
}
}
List<String> missing = new ArrayList<>();
for (int id : quorumNodeIds) {
if (nodeId == id) {
continue; // We get the local node's features from localSupportedFeatures.
}
NodeApiVersions nodeVersions = apiVersions.get(Integer.toString(id));
if (nodeVersions == null) {
missing.add(Integer.toString(id));
continue;
}
SupportedVersionRange supportedRange = nodeVersions.supportedFeatures().get(featureName);
VersionRange range = supportedRange == null ? DISABLED :
VersionRange.of(supportedRange.min(), supportedRange.max());
if (!range.contains(level)) {
if (range.equals(DISABLED)) {
return Optional.of("Controller " + id + " does not support this feature.");
} else {
return Optional.of("Controller " + id + " only supports versions " + range);
}
}
}
if (!missing.isEmpty()) {
log.info("Unable to get feature level information for controller(s): " + String.join(", ", missing));
}
return Optional.empty();
public QuorumFeatures(
int nodeId,
Map<String, VersionRange> localSupportedFeatures,
List<Integer> quorumNodeIds
) {
this.nodeId = nodeId;
this.localSupportedFeatures = Collections.unmodifiableMap(localSupportedFeatures);
this.quorumNodeIds = Collections.unmodifiableList(quorumNodeIds);
}
VersionRange localSupportedFeature(String featureName) {
return localSupportedFeatures.getOrDefault(featureName, DISABLED);
public int nodeId() {
return nodeId;
}
boolean isControllerId(int nodeId) {
public Map<String, VersionRange> localSupportedFeatures() {
return localSupportedFeatures;
}
public List<Integer> quorumNodeIds() {
return quorumNodeIds;
}
public VersionRange localSupportedFeature(String name) {
return localSupportedFeatures.getOrDefault(name, DISABLED);
}
public boolean isControllerId(int nodeId) {
return quorumNodeIds.contains(nodeId);
}
// check if all controller nodes are ZK Migration ready
public Optional<String> reasonAllControllersZkMigrationNotReady() {
List<String> missingApiVers = new ArrayList<>();
List<String> zkMigrationNotReady = new ArrayList<>();
for (int id : quorumNodeIds) {
if (nodeId == id) {
continue; // No need to check local node because the KraftMigrationDriver will be created only when migration config set
}
NodeApiVersions nodeVersions = apiVersions.get(Integer.toString(id));
if (nodeVersions == null) {
missingApiVers.add(String.valueOf(id));
} else if (!nodeVersions.zkMigrationEnabled()) {
zkMigrationNotReady.add(String.valueOf(id));
}
}
boolean isReady = missingApiVers.isEmpty() && zkMigrationNotReady.isEmpty();
if (!isReady) {
String zkMigrationNotReadyMsg = zkMigrationNotReady.isEmpty() ? "" : "Nodes don't enable `zookeeper.metadata.migration.enable`: " + zkMigrationNotReady + ".";
String missingApiVersionMsg = missingApiVers.isEmpty() ? "" : " Missing apiVersion from nodes: " + missingApiVers;
return Optional.of(zkMigrationNotReadyMsg + missingApiVersionMsg);
public Optional<String> reasonNotLocallySupported(
String featureName,
short newVersion
) {
return reasonNotSupported(newVersion,
"Local controller " + nodeId,
localSupportedFeature(featureName));
}
public Optional<String> reasonAllControllersZkMigrationNotReady(
MetadataVersion metadataVersion,
Map<Integer, ControllerRegistration> controllers
) {
if (!metadataVersion.isMigrationSupported()) {
return Optional.of("Metadata version too low at " + metadataVersion);
} else if (!metadataVersion.isControllerRegistrationSupported()) {
return Optional.empty();
}
for (int quorumNodeId : quorumNodeIds) {
ControllerRegistration registration = controllers.get(quorumNodeId);
if (registration == null) {
return Optional.of("No registration found for controller " + quorumNodeId);
} else if (!registration.zkMigrationReady()) {
return Optional.of("Controller " + quorumNodeId + " has not enabled " +
"zookeeper.metadata.migration.enable");
}
}
return Optional.empty();
}
@Override
public int hashCode() {
return Objects.hash(nodeId, localSupportedFeatures, quorumNodeIds);
}
@Override
public boolean equals(Object o) {
if (o == null || !(o.getClass().equals(QuorumFeatures.class))) return false;
QuorumFeatures other = (QuorumFeatures) o;
return nodeId == other.nodeId &&
localSupportedFeatures.equals(other.localSupportedFeatures) &&
quorumNodeIds.equals(other.quorumNodeIds);
}
@Override
public String toString() {
List<String> features = new ArrayList<>();
localSupportedFeatures.entrySet().forEach(f -> features.add(f.getKey() + ": " + f.getValue()));
features.sort(String::compareTo);
List<String> nodeIds = new ArrayList<>();
quorumNodeIds.forEach(id -> nodeIds.add("" + id));
nodeIds.sort(String::compareTo);
return "QuorumFeatures" +
"(nodeId=" + nodeId +
", localSupportedFeatures={" + features + "}" +
", quorumNodeIds=[" + nodeIds + "]" +
")";
}
}

View File

@ -20,11 +20,13 @@ package org.apache.kafka.image;
import org.apache.kafka.common.metadata.BrokerRegistrationChangeRecord;
import org.apache.kafka.common.metadata.FenceBrokerRecord;
import org.apache.kafka.common.metadata.RegisterBrokerRecord;
import org.apache.kafka.common.metadata.RegisterControllerRecord;
import org.apache.kafka.common.metadata.UnfenceBrokerRecord;
import org.apache.kafka.common.metadata.UnregisterBrokerRecord;
import org.apache.kafka.metadata.BrokerRegistration;
import org.apache.kafka.metadata.BrokerRegistrationFencingChange;
import org.apache.kafka.metadata.BrokerRegistrationInControlledShutdownChange;
import org.apache.kafka.metadata.ControllerRegistration;
import org.apache.kafka.server.common.MetadataVersion;
import java.util.HashMap;
@ -39,6 +41,7 @@ import java.util.Optional;
public final class ClusterDelta {
private final ClusterImage image;
private final HashMap<Integer, Optional<BrokerRegistration>> changedBrokers = new HashMap<>();
private final HashMap<Integer, Optional<ControllerRegistration>> changedControllers = new HashMap<>();
public ClusterDelta(ClusterImage image) {
this.image = image;
@ -48,6 +51,10 @@ public final class ClusterDelta {
return changedBrokers;
}
public HashMap<Integer, Optional<ControllerRegistration>> changedControllers() {
return changedControllers;
}
public BrokerRegistration broker(int nodeId) {
Optional<BrokerRegistration> result = changedBrokers.get(nodeId);
if (result != null) {
@ -62,6 +69,11 @@ public final class ClusterDelta {
changedBrokers.put(brokerId, Optional.empty());
}
}
for (Integer controllerId : image.controllers().keySet()) {
if (!changedControllers.containsKey(controllerId)) {
changedControllers.put(controllerId, Optional.empty());
}
}
}
public void handleMetadataVersionChange(MetadataVersion newVersion) {
@ -77,6 +89,11 @@ public final class ClusterDelta {
changedBrokers.put(record.brokerId(), Optional.empty());
}
public void replay(RegisterControllerRecord record) {
ControllerRegistration controller = ControllerRegistration.fromRecord(record);
changedControllers.put(controller.id(), Optional.of(controller));
}
private BrokerRegistration getBrokerOrThrow(int brokerId, long epoch, String action) {
BrokerRegistration broker = broker(brokerId);
if (broker == null) {
@ -147,13 +164,33 @@ public final class ClusterDelta {
}
}
}
return new ClusterImage(newBrokers);
Map<Integer, ControllerRegistration> newControllers = new HashMap<>(image.controllers().size());
for (Entry<Integer, ControllerRegistration> entry : image.controllers().entrySet()) {
int nodeId = entry.getKey();
Optional<ControllerRegistration> change = changedControllers.get(nodeId);
if (change == null) {
newControllers.put(nodeId, entry.getValue());
} else if (change.isPresent()) {
newControllers.put(nodeId, change.get());
}
}
for (Entry<Integer, Optional<ControllerRegistration>> entry : changedControllers.entrySet()) {
int nodeId = entry.getKey();
Optional<ControllerRegistration> controllerRegistration = entry.getValue();
if (!newControllers.containsKey(nodeId)) {
if (controllerRegistration.isPresent()) {
newControllers.put(nodeId, controllerRegistration.get());
}
}
}
return new ClusterImage(newBrokers, newControllers);
}
@Override
public String toString() {
return "ClusterDelta(" +
"changedBrokers=" + changedBrokers +
", changedControllers=" + changedControllers +
')';
}
}

View File

@ -21,6 +21,7 @@ import org.apache.kafka.image.node.ClusterImageNode;
import org.apache.kafka.image.writer.ImageWriter;
import org.apache.kafka.image.writer.ImageWriterOptions;
import org.apache.kafka.metadata.BrokerRegistration;
import org.apache.kafka.metadata.ControllerRegistration;
import java.util.Collections;
import java.util.Map;
@ -32,12 +33,20 @@ import java.util.Map;
* This class is thread-safe.
*/
public final class ClusterImage {
public static final ClusterImage EMPTY = new ClusterImage(Collections.emptyMap());
public static final ClusterImage EMPTY = new ClusterImage(
Collections.emptyMap(),
Collections.emptyMap());
private final Map<Integer, BrokerRegistration> brokers;
public ClusterImage(Map<Integer, BrokerRegistration> brokers) {
private final Map<Integer, ControllerRegistration> controllers;
public ClusterImage(
Map<Integer, BrokerRegistration> brokers,
Map<Integer, ControllerRegistration> controllers
) {
this.brokers = Collections.unmodifiableMap(brokers);
this.controllers = Collections.unmodifiableMap(controllers);
}
public boolean isEmpty() {
@ -52,6 +61,10 @@ public final class ClusterImage {
return brokers.get(nodeId);
}
public Map<Integer, ControllerRegistration> controllers() {
return controllers;
}
public boolean containsBroker(int brokerId) {
return brokers.containsKey(brokerId);
}

View File

@ -28,6 +28,7 @@ import org.apache.kafka.common.metadata.PartitionChangeRecord;
import org.apache.kafka.common.metadata.PartitionRecord;
import org.apache.kafka.common.metadata.ProducerIdsRecord;
import org.apache.kafka.common.metadata.RegisterBrokerRecord;
import org.apache.kafka.common.metadata.RegisterControllerRecord;
import org.apache.kafka.common.metadata.RemoveAccessControlEntryRecord;
import org.apache.kafka.common.metadata.RemoveTopicRecord;
import org.apache.kafka.common.metadata.RemoveUserScramCredentialRecord;
@ -229,6 +230,9 @@ public final class MetadataDelta {
case ZK_MIGRATION_STATE_RECORD:
replay((ZkMigrationStateRecord) record);
break;
case REGISTER_CONTROLLER_RECORD:
replay((RegisterControllerRecord) record);
break;
default:
throw new RuntimeException("Unknown metadata record type " + type);
}
@ -317,6 +321,10 @@ public final class MetadataDelta {
getOrCreateFeaturesDelta().replay(record);
}
public void replay(RegisterControllerRecord record) {
getOrCreateClusterDelta().replay(record);
}
/**
* Create removal deltas for anything which was in the base image, but which was not
* referenced in the snapshot records we just applied.

View File

@ -0,0 +1,62 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.image.node;
import org.apache.kafka.image.ClusterImage;
import org.apache.kafka.metadata.BrokerRegistration;
import java.util.ArrayList;
import java.util.Collection;
public class ClusterImageBrokersNode implements MetadataNode {
/**
* The name of this node.
*/
public final static String NAME = "brokers";
/**
* The cluster image.
*/
private final ClusterImage image;
public ClusterImageBrokersNode(ClusterImage image) {
this.image = image;
}
@Override
public Collection<String> childNames() {
ArrayList<String> childNames = new ArrayList<>();
for (Integer brokerId : image.brokers().keySet()) {
childNames.add(brokerId.toString());
}
return childNames;
}
@Override
public MetadataNode child(String name) {
try {
Integer brokerId = Integer.valueOf(name);
BrokerRegistration registration = image.brokers().get(brokerId);
if (registration == null) return null;
return new MetadataLeafNode(registration.toString());
} catch (NumberFormatException e) {
return null;
}
}
}

View File

@ -0,0 +1,62 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.image.node;
import org.apache.kafka.image.ClusterImage;
import org.apache.kafka.metadata.ControllerRegistration;
import java.util.ArrayList;
import java.util.Collection;
public class ClusterImageControllersNode implements MetadataNode {
/**
* The name of this node.
*/
public final static String NAME = "controllers";
/**
* The cluster image.
*/
private final ClusterImage image;
public ClusterImageControllersNode(ClusterImage image) {
this.image = image;
}
@Override
public Collection<String> childNames() {
ArrayList<String> childNames = new ArrayList<>();
for (Integer brokerId : image.controllers().keySet()) {
childNames.add(brokerId.toString());
}
return childNames;
}
@Override
public MetadataNode child(String name) {
try {
Integer brokerId = Integer.valueOf(name);
ControllerRegistration registration = image.controllers().get(brokerId);
if (registration == null) return null;
return new MetadataLeafNode(registration.toString());
} catch (NumberFormatException e) {
return null;
}
}
}

View File

@ -18,9 +18,8 @@
package org.apache.kafka.image.node;
import org.apache.kafka.image.ClusterImage;
import org.apache.kafka.metadata.BrokerRegistration;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
@ -41,21 +40,16 @@ public class ClusterImageNode implements MetadataNode {
@Override
public Collection<String> childNames() {
ArrayList<String> childNames = new ArrayList<>();
for (Integer brokerId : image.brokers().keySet()) {
childNames.add(brokerId.toString());
}
return childNames;
return Arrays.asList(ClusterImageBrokersNode.NAME, ClusterImageControllersNode.NAME);
}
@Override
public MetadataNode child(String name) {
try {
Integer brokerId = Integer.valueOf(name);
BrokerRegistration registration = image.brokers().get(brokerId);
if (registration == null) return null;
return new MetadataLeafNode(registration.toString());
} catch (NumberFormatException e) {
if (name.equals(ClusterImageBrokersNode.NAME)) {
return new ClusterImageBrokersNode(image);
} else if (name.equals(ClusterImageControllersNode.NAME)) {
return new ClusterImageControllersNode(image);
} else {
return null;
}
}

View File

@ -0,0 +1,83 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.image.publisher;
import org.apache.kafka.common.Endpoint;
import org.apache.kafka.common.message.DescribeClusterResponseData.DescribeClusterBroker;
import org.apache.kafka.common.message.DescribeClusterResponseData.DescribeClusterBrokerCollection;
import org.apache.kafka.image.MetadataDelta;
import org.apache.kafka.image.MetadataImage;
import org.apache.kafka.image.loader.LoaderManifest;
import org.apache.kafka.image.loader.LoaderManifestType;
import org.apache.kafka.metadata.ControllerRegistration;
import java.util.Collections;
import java.util.Map;
/**
* A publisher to track controller registrations.
*/
public class ControllerRegistrationsPublisher implements MetadataPublisher {
private volatile Map<Integer, ControllerRegistration> controllers;
public ControllerRegistrationsPublisher() {
this.controllers = Collections.emptyMap();
}
@Override
public String name() {
return "ControllerRegistrationsPublisher";
}
@Override
public void onMetadataUpdate(
MetadataDelta delta,
MetadataImage newImage,
LoaderManifest manifest
) {
if (manifest.type() == LoaderManifestType.LOG_DELTA || delta.clusterDelta() != null) {
controllers = newImage.cluster().controllers();
}
}
public DescribeClusterBrokerCollection describeClusterControllers(
String endpointName
) {
DescribeClusterBrokerCollection results = new DescribeClusterBrokerCollection();
for (ControllerRegistration registration : controllers.values()) {
Endpoint endpoint = registration.listeners().get(endpointName);
if (endpoint != null) {
results.add(new DescribeClusterBroker().
setBrokerId(registration.id()).
setHost(endpoint.host()).
setPort(endpoint.port()).
setRack(null));
}
}
return results;
}
public Map<Integer, ControllerRegistration> controllers() {
return controllers;
}
@Override
public void close() {
}
}

View File

@ -0,0 +1,253 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.metadata;
import org.apache.kafka.common.Endpoint;
import org.apache.kafka.common.Node;
import org.apache.kafka.common.Uuid;
import org.apache.kafka.common.metadata.RegisterControllerRecord;
import org.apache.kafka.common.metadata.RegisterControllerRecord.ControllerEndpoint;
import org.apache.kafka.common.metadata.RegisterControllerRecord.ControllerFeature;
import org.apache.kafka.common.security.auth.SecurityProtocol;
import org.apache.kafka.image.writer.ImageWriterOptions;
import org.apache.kafka.server.common.ApiMessageAndVersion;
import org.apache.kafka.server.common.MetadataVersion;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Objects;
import java.util.Optional;
import java.util.stream.Collectors;
/**
* An immutable class which represents controller registrations.
*/
public class ControllerRegistration {
public static class Builder {
private int id = 0;
private Uuid incarnationId = null;
private boolean zkMigrationReady = false;
private Map<String, Endpoint> listeners = null;
private Map<String, VersionRange> supportedFeatures = null;
public Builder() {
this.id = 0;
this.incarnationId = null;
this.zkMigrationReady = false;
this.listeners = null;
this.supportedFeatures = null;
}
public Builder(RegisterControllerRecord record) {
this.id = record.controllerId();
this.incarnationId = record.incarnationId();
this.zkMigrationReady = record.zkMigrationReady();
Map<String, Endpoint> newListeners = new HashMap<>();
record.endPoints().forEach(endPoint -> {
listeners.put(endPoint.name(), new Endpoint(endPoint.name(),
SecurityProtocol.forId(endPoint.securityProtocol()),
endPoint.host(),
endPoint.port()));
});
this.listeners = Collections.unmodifiableMap(newListeners);
Map<String, VersionRange> newSupportedFeatures = new HashMap<>();
record.features().forEach(feature -> {
newSupportedFeatures.put(feature.name(), VersionRange.of(
feature.minSupportedVersion(), feature.maxSupportedVersion()));
});
this.supportedFeatures = Collections.unmodifiableMap(newSupportedFeatures);
}
public Builder setId(int id) {
this.id = id;
return this;
}
public Builder setIncarnationId(Uuid incarnationId) {
this.incarnationId = incarnationId;
return this;
}
public Builder setZkMigrationReady(boolean zkMigrationReady) {
this.zkMigrationReady = zkMigrationReady;
return this;
}
public Builder setListeners(Map<String, Endpoint> listeners) {
this.listeners = listeners;
return this;
}
public Builder setSupportedFeatures(Map<String, VersionRange> supportedFeatures) {
this.supportedFeatures = supportedFeatures;
return this;
}
public ControllerRegistration build() {
if (incarnationId == null) throw new RuntimeException("You must set incarnationId.");
if (listeners == null) throw new RuntimeException("You must set listeners.");
if (supportedFeatures == null) {
supportedFeatures = new HashMap<>();
supportedFeatures.put(MetadataVersion.FEATURE_NAME, VersionRange.of(
MetadataVersion.MINIMUM_KRAFT_VERSION.featureLevel(),
MetadataVersion.latest().featureLevel()));
}
return new ControllerRegistration(id,
incarnationId,
zkMigrationReady,
listeners,
supportedFeatures);
}
}
public static ControllerRegistration fromRecord(RegisterControllerRecord record) {
Map<String, Endpoint> listeners = new HashMap<>();
record.endPoints().forEach(endpoint -> {
SecurityProtocol protocol = SecurityProtocol.forId(endpoint.securityProtocol());
if (protocol == null) {
throw new RuntimeException("Unknown security protocol " +
(int) endpoint.securityProtocol());
}
listeners.put(endpoint.name(), new Endpoint(endpoint.name(),
protocol,
endpoint.host(),
endpoint.port()));
});
Map<String, VersionRange> supportedFeatures = new HashMap<>();
record.features().forEach(feature -> {
supportedFeatures.put(feature.name(),
VersionRange.of(feature.minSupportedVersion(), feature.maxSupportedVersion()));
});
return new ControllerRegistration(record.controllerId(),
record.incarnationId(),
record.zkMigrationReady(),
listeners,
supportedFeatures);
}
private final int id;
private final Uuid incarnationId;
private final boolean zkMigrationReady;
private final Map<String, Endpoint> listeners;
private final Map<String, VersionRange> supportedFeatures;
private ControllerRegistration(int id,
Uuid incarnationId,
boolean zkMigrationReady,
Map<String, Endpoint> listeners,
Map<String, VersionRange> supportedFeatures
) {
this.id = id;
this.incarnationId = incarnationId;
this.zkMigrationReady = zkMigrationReady;
this.listeners = listeners;
this.supportedFeatures = supportedFeatures;
}
public int id() {
return id;
}
public Uuid incarnationId() {
return incarnationId;
}
public boolean zkMigrationReady() {
return zkMigrationReady;
}
public Map<String, Endpoint> listeners() {
return listeners;
}
public Optional<Node> node(String listenerName) {
Endpoint endpoint = listeners().get(listenerName);
if (endpoint == null) {
return Optional.empty();
}
return Optional.of(new Node(id, endpoint.host(), endpoint.port(), null));
}
public Map<String, VersionRange> supportedFeatures() {
return supportedFeatures;
}
public ApiMessageAndVersion toRecord(ImageWriterOptions options) {
RegisterControllerRecord registrationRecord = new RegisterControllerRecord().
setControllerId(id).
setIncarnationId(incarnationId).
setZkMigrationReady(zkMigrationReady);
for (Entry<String, Endpoint> entry : listeners.entrySet()) {
Endpoint endpoint = entry.getValue();
registrationRecord.endPoints().add(new ControllerEndpoint().
setName(entry.getKey()).
setHost(endpoint.host()).
setPort(endpoint.port()).
setSecurityProtocol(endpoint.securityProtocol().id));
}
for (Entry<String, VersionRange> entry : supportedFeatures.entrySet()) {
registrationRecord.features().add(new ControllerFeature().
setName(entry.getKey()).
setMinSupportedVersion(entry.getValue().min()).
setMaxSupportedVersion(entry.getValue().max()));
}
return new ApiMessageAndVersion(registrationRecord,
options.metadataVersion().registerBrokerRecordVersion());
}
@Override
public int hashCode() {
return Objects.hash(id,
incarnationId,
zkMigrationReady,
listeners,
supportedFeatures);
}
@Override
public boolean equals(Object o) {
if (!(o instanceof ControllerRegistration)) return false;
ControllerRegistration other = (ControllerRegistration) o;
return other.id == id &&
other.incarnationId.equals(incarnationId) &&
other.zkMigrationReady == zkMigrationReady &&
other.listeners.equals(listeners) &&
other.supportedFeatures.equals(supportedFeatures);
}
@Override
public String toString() {
StringBuilder bld = new StringBuilder();
bld.append("ControllerRegistration(id=").append(id);
bld.append(", incarnationId=").append(incarnationId);
bld.append(", zkMigrationReady=").append(zkMigrationReady);
bld.append(", listeners=[").append(
listeners.keySet().stream().sorted().
map(n -> listeners.get(n).toString()).
collect(Collectors.joining(", ")));
bld.append("], supportedFeatures={").append(
supportedFeatures.keySet().stream().sorted().
map(k -> k + ": " + supportedFeatures.get(k)).
collect(Collectors.joining(", ")));
bld.append("}");
bld.append(")");
return bld.toString();
}
}

View File

@ -174,7 +174,8 @@ public class KRaftMigrationDriver implements MetadataPublisher {
}
private boolean isControllerQuorumReadyForMigration() {
Optional<String> notReadyMsg = this.quorumFeatures.reasonAllControllersZkMigrationNotReady();
Optional<String> notReadyMsg = this.quorumFeatures.reasonAllControllersZkMigrationNotReady(
image.features().metadataVersion(), image.cluster().controllers());
if (notReadyMsg.isPresent()) {
log.warn("Still waiting for all controller nodes ready to begin the migration. Not ready due to:" + notReadyMsg.get());
return false;

View File

@ -17,18 +17,27 @@
package org.apache.kafka.metadata.publisher;
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.image.MetadataDelta;
import org.apache.kafka.image.MetadataImage;
import org.apache.kafka.image.loader.LoaderManifest;
import org.apache.kafka.image.publisher.MetadataPublisher;
import org.apache.kafka.server.common.Features;
import org.slf4j.Logger;
import static org.apache.kafka.server.common.MetadataVersion.MINIMUM_KRAFT_VERSION;
public class FeaturesPublisher implements MetadataPublisher {
private final Logger log;
private volatile Features features = Features.fromKRaftVersion(MINIMUM_KRAFT_VERSION);
public FeaturesPublisher(
LogContext logContext
) {
log = logContext.logger(FeaturesPublisher.class);
}
public Features features() {
return features;
}
@ -45,10 +54,14 @@ public class FeaturesPublisher implements MetadataPublisher {
LoaderManifest manifest
) {
if (delta.featuresDelta() != null) {
features = new Features(newImage.features().metadataVersion(),
Features newFeatures = new Features(newImage.features().metadataVersion(),
newImage.features().finalizedVersions(),
newImage.provenance().lastContainedOffset(),
true);
if (!newFeatures.equals(features)) {
log.info("Loaded new metadata {}.", newFeatures);
features = newFeatures;
}
}
}
}

View File

@ -0,0 +1,50 @@
// Licensed to the Apache Software Foundation (ASF) under one or more
// contributor license agreements. See the NOTICE file distributed with
// this work for additional information regarding copyright ownership.
// The ASF licenses this file to You under the Apache License, Version 2.0
// (the "License"); you may not use this file except in compliance with
// the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
{
"apiKey": 26,
"type": "metadata",
"name": "RegisterControllerRecord",
"validVersions": "0",
"flexibleVersions": "0+",
"fields": [
{ "name": "ControllerId", "type": "int32", "versions": "0+",
"about": "The controller id." },
{ "name": "IncarnationId", "type": "uuid", "versions": "0+",
"about": "The incarnation ID of the controller process" },
{ "name": "ZkMigrationReady", "type": "bool", "versions": "0+",
"about": "Set if the required configurations for ZK migration are present." },
{ "name": "EndPoints", "type": "[]ControllerEndpoint", "versions": "0+",
"about": "The endpoints that can be used to communicate with this controller.", "fields": [
{ "name": "Name", "type": "string", "versions": "0+", "mapKey": true,
"about": "The name of the endpoint." },
{ "name": "Host", "type": "string", "versions": "0+",
"about": "The hostname." },
{ "name": "Port", "type": "uint16", "versions": "0+",
"about": "The port." },
{ "name": "SecurityProtocol", "type": "int16", "versions": "0+",
"about": "The security protocol." }
]},
{ "name": "Features", "type": "[]ControllerFeature",
"about": "The features on this controller", "versions": "0+", "fields": [
{ "name": "Name", "type": "string", "versions": "0+", "mapKey": true,
"about": "The feature name." },
{ "name": "MinSupportedVersion", "type": "int16", "versions": "0+",
"about": "The minimum supported feature level." },
{ "name": "MaxSupportedVersion", "type": "int16", "versions": "0+",
"about": "The maximum supported feature level." }
]}
]
}

View File

@ -17,7 +17,6 @@
package org.apache.kafka.controller;
import org.apache.kafka.clients.ApiVersions;
import org.apache.kafka.common.Endpoint;
import org.apache.kafka.common.Uuid;
import org.apache.kafka.common.errors.InconsistentClusterIdException;
@ -75,7 +74,7 @@ public class ClusterControlManagerTest {
SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
FeatureControlManager featureControl = new FeatureControlManager.Builder().
setSnapshotRegistry(snapshotRegistry).
setQuorumFeatures(new QuorumFeatures(0, new ApiVersions(),
setQuorumFeatures(new QuorumFeatures(0,
QuorumFeatures.defaultFeatureMap(),
Collections.singletonList(0))).
setMetadataVersion(MetadataVersion.latest()).
@ -136,7 +135,7 @@ public class ClusterControlManagerTest {
SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
FeatureControlManager featureControl = new FeatureControlManager.Builder().
setSnapshotRegistry(snapshotRegistry).
setQuorumFeatures(new QuorumFeatures(0, new ApiVersions(),
setQuorumFeatures(new QuorumFeatures(0,
QuorumFeatures.defaultFeatureMap(),
Collections.singletonList(0))).
setMetadataVersion(MetadataVersion.latest()).
@ -189,7 +188,7 @@ public class ClusterControlManagerTest {
SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
FeatureControlManager featureControl = new FeatureControlManager.Builder().
setSnapshotRegistry(snapshotRegistry).
setQuorumFeatures(new QuorumFeatures(0, new ApiVersions(),
setQuorumFeatures(new QuorumFeatures(0,
QuorumFeatures.defaultFeatureMap(),
Collections.singletonList(0))).
setMetadataVersion(MetadataVersion.latest()).
@ -244,7 +243,7 @@ public class ClusterControlManagerTest {
SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
FeatureControlManager featureControl = new FeatureControlManager.Builder().
setSnapshotRegistry(snapshotRegistry).
setQuorumFeatures(new QuorumFeatures(0, new ApiVersions(),
setQuorumFeatures(new QuorumFeatures(0,
QuorumFeatures.defaultFeatureMap(),
Collections.singletonList(0))).
setMetadataVersion(MetadataVersion.latest()).
@ -273,7 +272,7 @@ public class ClusterControlManagerTest {
SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
FeatureControlManager featureControl = new FeatureControlManager.Builder().
setSnapshotRegistry(snapshotRegistry).
setQuorumFeatures(new QuorumFeatures(0, new ApiVersions(),
setQuorumFeatures(new QuorumFeatures(0,
QuorumFeatures.defaultFeatureMap(),
Collections.singletonList(0))).
setMetadataVersion(metadataVersion).
@ -329,7 +328,7 @@ public class ClusterControlManagerTest {
SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
FeatureControlManager featureControl = new FeatureControlManager.Builder().
setSnapshotRegistry(snapshotRegistry).
setQuorumFeatures(new QuorumFeatures(0, new ApiVersions(),
setQuorumFeatures(new QuorumFeatures(0,
QuorumFeatures.defaultFeatureMap(),
Collections.singletonList(0))).
setMetadataVersion(MetadataVersion.latest()).
@ -363,7 +362,7 @@ public class ClusterControlManagerTest {
SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
FeatureControlManager featureControl = new FeatureControlManager.Builder().
setSnapshotRegistry(snapshotRegistry).
setQuorumFeatures(new QuorumFeatures(0, new ApiVersions(),
setQuorumFeatures(new QuorumFeatures(0,
QuorumFeatures.defaultFeatureMap(),
Collections.singletonList(0))).
setMetadataVersion(MetadataVersion.latest()).
@ -416,7 +415,7 @@ public class ClusterControlManagerTest {
SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
FeatureControlManager featureControl = new FeatureControlManager.Builder().
setSnapshotRegistry(snapshotRegistry).
setQuorumFeatures(new QuorumFeatures(0, new ApiVersions(),
setQuorumFeatures(new QuorumFeatures(0,
QuorumFeatures.defaultFeatureMap(),
Collections.singletonList(0))).
setMetadataVersion(metadataVersion).
@ -492,7 +491,7 @@ public class ClusterControlManagerTest {
SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
FeatureControlManager featureControl = new FeatureControlManager.Builder().
setSnapshotRegistry(snapshotRegistry).
setQuorumFeatures(new QuorumFeatures(0, new ApiVersions(),
setQuorumFeatures(new QuorumFeatures(0,
Collections.singletonMap(MetadataVersion.FEATURE_NAME, VersionRange.of(
MetadataVersion.IBP_3_1_IV0.featureLevel(),
MetadataVersion.IBP_3_3_IV0.featureLevel())),

View File

@ -26,7 +26,6 @@ import java.util.List;
import java.util.Map;
import java.util.Optional;
import org.apache.kafka.clients.ApiVersions;
import org.apache.kafka.clients.admin.FeatureUpdate;
import org.apache.kafka.common.metadata.FeatureLevelRecord;
import org.apache.kafka.common.protocol.Errors;
@ -78,7 +77,7 @@ public class FeatureControlManagerTest {
public static QuorumFeatures features(Object... args) {
Map<String, VersionRange> features = QuorumFeatures.defaultFeatureMap();
features.putAll(rangeMap(args));
return new QuorumFeatures(0, new ApiVersions(), features, emptyList());
return new QuorumFeatures(0, features, emptyList());
}
private static Map<String, Short> updateMap(Object... args) {
@ -392,7 +391,7 @@ public class FeatureControlManagerTest {
MetadataVersion.IBP_3_0_IV1.featureLevel(), MetadataVersion.latest().featureLevel()));
localSupportedFeatures.put("foo", VersionRange.of(0, 2));
FeatureControlManager manager = new FeatureControlManager.Builder().
setQuorumFeatures(new QuorumFeatures(0, new ApiVersions(), localSupportedFeatures, emptyList())).
setQuorumFeatures(new QuorumFeatures(0, localSupportedFeatures, emptyList())).
build();
ControllerResult<Map<String, ApiError>> result = manager.updateFeatures(
Collections.singletonMap("foo", (short) 1),

View File

@ -18,7 +18,6 @@
package org.apache.kafka.controller;
import java.util.Collections;
import org.apache.kafka.clients.ApiVersions;
import org.apache.kafka.common.errors.StaleBrokerEpochException;
import org.apache.kafka.common.errors.UnknownServerException;
import org.apache.kafka.common.metadata.ProducerIdsRecord;
@ -49,7 +48,7 @@ public class ProducerIdControlManagerTest {
snapshotRegistry = new SnapshotRegistry(new LogContext());
featureControl = new FeatureControlManager.Builder().
setSnapshotRegistry(snapshotRegistry).
setQuorumFeatures(new QuorumFeatures(0, new ApiVersions(),
setQuorumFeatures(new QuorumFeatures(0,
QuorumFeatures.defaultFeatureMap(),
Collections.singletonList(0))).
setMetadataVersion(MetadataVersion.latest()).

View File

@ -158,7 +158,7 @@ public class QuorumControllerTest {
) {
controlEnv.activeController().registerBroker(ANONYMOUS_CONTEXT,
new BrokerRegistrationRequestData().
setFeatures(brokerFeatures(MetadataVersion.IBP_3_0_IV1, MetadataVersion.IBP_3_6_IV1)).
setFeatures(brokerFeatures(MetadataVersion.IBP_3_0_IV1, MetadataVersion.latest())).
setBrokerId(0).
setClusterId(logEnv.clusterId())).get();
testConfigurationOperations(controlEnv.activeController());
@ -199,7 +199,7 @@ public class QuorumControllerTest {
) {
controlEnv.activeController().registerBroker(ANONYMOUS_CONTEXT,
new BrokerRegistrationRequestData().
setFeatures(brokerFeatures(MetadataVersion.IBP_3_0_IV1, MetadataVersion.IBP_3_6_IV1)).
setFeatures(brokerFeatures(MetadataVersion.IBP_3_0_IV1, MetadataVersion.latest())).
setBrokerId(0).
setClusterId(logEnv.clusterId())).get();
testDelayedConfigurationOperations(logEnv, controlEnv.activeController());
@ -536,7 +536,7 @@ public class QuorumControllerTest {
setBrokerId(0).
setClusterId(active.clusterId()).
setIncarnationId(Uuid.fromString("kxAT73dKQsitIedpiPtwBA")).
setFeatures(brokerFeatures(MetadataVersion.IBP_3_0_IV1, MetadataVersion.IBP_3_6_IV1)).
setFeatures(brokerFeatures(MetadataVersion.IBP_3_0_IV1, MetadataVersion.latest())).
setListeners(listeners));
assertEquals(3L, reply.get().epoch());
CreateTopicsRequestData createTopicsRequestData =

View File

@ -105,7 +105,7 @@ public class QuorumControllerTestEnv implements AutoCloseable {
builder.setRaftClient(logEnv.logManagers().get(nodeId));
builder.setBootstrapMetadata(bootstrapMetadata);
builder.setLeaderImbalanceCheckIntervalNs(leaderImbalanceCheckIntervalNs);
builder.setQuorumFeatures(new QuorumFeatures(nodeId, apiVersions, QuorumFeatures.defaultFeatureMap(), nodeIds));
builder.setQuorumFeatures(new QuorumFeatures(nodeId, QuorumFeatures.defaultFeatureMap(), nodeIds));
sessionTimeoutMillis.ifPresent(timeout -> {
builder.setSessionTimeoutNs(NANOSECONDS.convert(timeout, TimeUnit.MILLISECONDS));
});

View File

@ -17,23 +17,17 @@
package org.apache.kafka.controller;
import org.apache.kafka.clients.ApiVersions;
import org.apache.kafka.clients.NodeApiVersions;
import org.apache.kafka.common.message.ApiVersionsResponseData.SupportedFeatureKey;
import org.apache.kafka.metadata.ControllerRegistration;
import org.apache.kafka.metadata.VersionRange;
import org.apache.kafka.server.common.MetadataVersion;
import org.junit.jupiter.api.Test;
import java.util.AbstractMap.SimpleImmutableEntry;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Optional;
import static java.util.Collections.emptyMap;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertTrue;
@ -41,92 +35,82 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
public class QuorumFeaturesTest {
private final static Map<String, VersionRange> LOCAL;
private final static QuorumFeatures QUORUM_FEATURES;
static {
Map<String, VersionRange> local = new HashMap<>();
local.put("foo", VersionRange.of(0, 3));
local.put("bar", VersionRange.of(0, 4));
local.put("baz", VersionRange.of(2, 2));
LOCAL = Collections.unmodifiableMap(local);
}
@Test
public void testDefaultSupportedLevels() {
QuorumFeatures quorumFeatures = new QuorumFeatures(0, new ApiVersions(), emptyMap(), Arrays.asList(0, 1, 2));
assertEquals(Optional.empty(), quorumFeatures.reasonNotSupported("foo", (short) 0));
assertEquals(Optional.of("Local controller 0 does not support this feature."),
quorumFeatures.reasonNotSupported("foo", (short) 1));
QUORUM_FEATURES = new QuorumFeatures(0, LOCAL, Arrays.asList(0, 1, 2));
}
@Test
public void testLocalSupportedFeature() {
QuorumFeatures quorumFeatures = new QuorumFeatures(0, new ApiVersions(), LOCAL, Arrays.asList(0, 1, 2));
assertEquals(VersionRange.of(0, 3), quorumFeatures.localSupportedFeature("foo"));
assertEquals(VersionRange.of(0, 4), quorumFeatures.localSupportedFeature("bar"));
assertEquals(VersionRange.of(2, 2), quorumFeatures.localSupportedFeature("baz"));
assertEquals(VersionRange.of(0, 0), quorumFeatures.localSupportedFeature("quux"));
assertEquals(VersionRange.of(0, 3), QUORUM_FEATURES.localSupportedFeature("foo"));
assertEquals(VersionRange.of(0, 4), QUORUM_FEATURES.localSupportedFeature("bar"));
assertEquals(VersionRange.of(2, 2), QUORUM_FEATURES.localSupportedFeature("baz"));
assertEquals(VersionRange.of(0, 0), QUORUM_FEATURES.localSupportedFeature("quux"));
}
@Test
public void testReasonNotSupported() {
ApiVersions apiVersions = new ApiVersions();
QuorumFeatures quorumFeatures = new QuorumFeatures(0, apiVersions, LOCAL, Arrays.asList(0, 1, 2));
assertEquals(Optional.of("Local controller 0 only supports versions 0-3"),
quorumFeatures.reasonNotSupported("foo", (short) 10));
apiVersions.update("1", nodeApiVersions(Arrays.asList(
new SimpleImmutableEntry<>("foo", VersionRange.of(1, 3)),
new SimpleImmutableEntry<>("bar", VersionRange.of(1, 3)),
new SimpleImmutableEntry<>("baz", VersionRange.of(1, 2)))));
assertEquals(Optional.empty(), quorumFeatures.reasonNotSupported("bar", (short) 3));
assertEquals(Optional.of("Controller 1 only supports versions 1-3"),
quorumFeatures.reasonNotSupported("bar", (short) 4));
}
private static NodeApiVersions nodeApiVersions(List<Entry<String, VersionRange>> entries) {
List<SupportedFeatureKey> features = new ArrayList<>();
entries.forEach(entry -> {
features.add(new SupportedFeatureKey().
setName(entry.getKey()).
setMinVersion(entry.getValue().min()).
setMaxVersion(entry.getValue().max()));
});
return new NodeApiVersions(Collections.emptyList(), features, false);
QuorumFeatures.reasonNotSupported((short) 10,
"Local controller 0", VersionRange.of(0, 3)));
assertEquals(Optional.empty(),
QuorumFeatures.reasonNotSupported((short) 3,
"Local controller 0", VersionRange.of(0, 3)));
}
@Test
public void testIsControllerId() {
QuorumFeatures quorumFeatures = new QuorumFeatures(0, new ApiVersions(), LOCAL, Arrays.asList(0, 1, 2));
assertTrue(quorumFeatures.isControllerId(0));
assertTrue(quorumFeatures.isControllerId(1));
assertTrue(quorumFeatures.isControllerId(2));
assertFalse(quorumFeatures.isControllerId(3));
assertTrue(QUORUM_FEATURES.isControllerId(0));
assertTrue(QUORUM_FEATURES.isControllerId(1));
assertTrue(QUORUM_FEATURES.isControllerId(2));
assertFalse(QUORUM_FEATURES.isControllerId(3));
}
@Test
public void testZkMigrationReady() {
ApiVersions apiVersions = new ApiVersions();
QuorumFeatures quorumFeatures = new QuorumFeatures(0, apiVersions, LOCAL, Arrays.asList(0, 1, 2));
public void testZkMigrationNotReadyIfMetadataVersionTooLow() {
assertEquals(Optional.of("Metadata version too low at 3.0-IV1"),
QUORUM_FEATURES.reasonAllControllersZkMigrationNotReady(
MetadataVersion.IBP_3_0_IV1, Collections.emptyMap()));
}
// create apiVersion with zkMigrationEnabled flag set for node 0, the other 2 nodes have no apiVersions info
apiVersions.update("0", new NodeApiVersions(Collections.emptyList(), Collections.emptyList(), true));
assertTrue(quorumFeatures.reasonAllControllersZkMigrationNotReady().isPresent());
assertTrue(quorumFeatures.reasonAllControllersZkMigrationNotReady().get().contains("Missing apiVersion from nodes: [1, 2]"));
@Test
public void testZkMigrationReadyIfControllerRegistrationNotSupported() {
assertEquals(Optional.empty(),
QUORUM_FEATURES.reasonAllControllersZkMigrationNotReady(
MetadataVersion.IBP_3_4_IV0, Collections.emptyMap()));
}
// create apiVersion with zkMigrationEnabled flag set for node 1, the other 1 node have no apiVersions info
apiVersions.update("1", new NodeApiVersions(Collections.emptyList(), Collections.emptyList(), true));
assertTrue(quorumFeatures.reasonAllControllersZkMigrationNotReady().isPresent());
assertTrue(quorumFeatures.reasonAllControllersZkMigrationNotReady().get().contains("Missing apiVersion from nodes: [2]"));
@Test
public void testZkMigrationNotReadyIfNotAllControllersRegistered() {
assertEquals(Optional.of("No registration found for controller 0"),
QUORUM_FEATURES.reasonAllControllersZkMigrationNotReady(
MetadataVersion.IBP_3_6_IV2, Collections.emptyMap()));
}
// create apiVersion with zkMigrationEnabled flag disabled for node 2, should still be not ready
apiVersions.update("2", NodeApiVersions.create());
assertTrue(quorumFeatures.reasonAllControllersZkMigrationNotReady().isPresent());
assertTrue(quorumFeatures.reasonAllControllersZkMigrationNotReady().get().contains("Nodes don't enable `zookeeper.metadata.migration.enable`: [2]"));
@Test
public void testZkMigrationNotReadyIfControllerNotReady() {
assertEquals(Optional.of("Controller 0 has not enabled zookeeper.metadata.migration.enable"),
QUORUM_FEATURES.reasonAllControllersZkMigrationNotReady(
MetadataVersion.IBP_3_6_IV2, Collections.singletonMap(0,
new ControllerRegistration.Builder().
setId(0).setZkMigrationReady(false).build())));
}
// update zkMigrationEnabled flag to enabled for node 2, should be ready now
apiVersions.update("2", new NodeApiVersions(Collections.emptyList(), Collections.emptyList(), true));
assertFalse(quorumFeatures.reasonAllControllersZkMigrationNotReady().isPresent());
// create apiVersion with zkMigrationEnabled flag disabled for a non-controller, and expect we fill filter it out
apiVersions.update("3", NodeApiVersions.create());
assertFalse(quorumFeatures.reasonAllControllersZkMigrationNotReady().isPresent());
@Test
public void testZkMigrationReadyIfAllControllersReady() {
Map<Integer, ControllerRegistration> controllers = new HashMap<>();
QUORUM_FEATURES.quorumNodeIds().forEach(id -> {
controllers.put(id, new ControllerRegistration.Builder().
setId(id).setZkMigrationReady(true).build());
});
assertEquals(Optional.of("Controller 0 has not enabled zookeeper.metadata.migration.enable"),
QUORUM_FEATURES.reasonAllControllersZkMigrationNotReady(
MetadataVersion.IBP_3_6_IV2, controllers));
}
}

View File

@ -17,7 +17,6 @@
package org.apache.kafka.controller;
import org.apache.kafka.clients.ApiVersions;
import org.apache.kafka.common.ElectionType;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.Uuid;
@ -207,7 +206,7 @@ public class ReplicationControlManagerTest {
this.time = time;
this.featureControl = new FeatureControlManager.Builder().
setSnapshotRegistry(snapshotRegistry).
setQuorumFeatures(new QuorumFeatures(0, new ApiVersions(),
setQuorumFeatures(new QuorumFeatures(0,
QuorumFeatures.defaultFeatureMap(),
Collections.singletonList(0))).
setMetadataVersion(metadataVersion).

View File

@ -21,6 +21,9 @@ import org.apache.kafka.common.Endpoint;
import org.apache.kafka.common.Uuid;
import org.apache.kafka.common.metadata.BrokerRegistrationChangeRecord;
import org.apache.kafka.common.metadata.FenceBrokerRecord;
import org.apache.kafka.common.metadata.RegisterControllerRecord;
import org.apache.kafka.common.metadata.RegisterControllerRecord.ControllerEndpoint;
import org.apache.kafka.common.metadata.RegisterControllerRecord.ControllerEndpointCollection;
import org.apache.kafka.common.metadata.UnfenceBrokerRecord;
import org.apache.kafka.common.metadata.UnregisterBrokerRecord;
import org.apache.kafka.common.security.auth.SecurityProtocol;
@ -28,6 +31,7 @@ import org.apache.kafka.image.writer.ImageWriterOptions;
import org.apache.kafka.image.writer.RecordListWriter;
import org.apache.kafka.metadata.BrokerRegistration;
import org.apache.kafka.metadata.BrokerRegistrationInControlledShutdownChange;
import org.apache.kafka.metadata.ControllerRegistration;
import org.apache.kafka.metadata.RecordTestUtils;
import org.apache.kafka.metadata.VersionRange;
import org.apache.kafka.server.common.ApiMessageAndVersion;
@ -46,7 +50,6 @@ import java.util.Optional;
import static org.apache.kafka.common.metadata.MetadataRecordType.FENCE_BROKER_RECORD;
import static org.apache.kafka.common.metadata.MetadataRecordType.UNFENCE_BROKER_RECORD;
import static org.apache.kafka.common.metadata.MetadataRecordType.UNREGISTER_BROKER_RECORD;
import static org.junit.jupiter.api.Assertions.assertEquals;
@ -88,7 +91,15 @@ public class ClusterImageTest {
Optional.of("arack"),
false,
false));
IMAGE1 = new ClusterImage(map1);
Map<Integer, ControllerRegistration> cmap1 = new HashMap<>();
cmap1.put(1000, new ControllerRegistration.Builder().
setId(1000).
setIncarnationId(Uuid.fromString("9ABu6HEgRuS-hjHLgC4cHw")).
setZkMigrationReady(false).
setListeners(Collections.singletonMap("PLAINTEXT",
new Endpoint("PLAINTEXT", SecurityProtocol.PLAINTEXT, "localhost", 19092))).
setSupportedFeatures(Collections.emptyMap()).build());
IMAGE1 = new ClusterImage(map1, cmap1);
DELTA1_RECORDS = new ArrayList<>();
DELTA1_RECORDS.add(new ApiMessageAndVersion(new UnfenceBrokerRecord().
@ -98,10 +109,23 @@ public class ClusterImageTest {
DELTA1_RECORDS.add(new ApiMessageAndVersion(new BrokerRegistrationChangeRecord().
setBrokerId(0).setBrokerEpoch(1000).setInControlledShutdown(
BrokerRegistrationInControlledShutdownChange.IN_CONTROLLED_SHUTDOWN.value()),
FENCE_BROKER_RECORD.highestSupportedVersion()));
(short) 0));
DELTA1_RECORDS.add(new ApiMessageAndVersion(new UnregisterBrokerRecord().
setBrokerId(2).setBrokerEpoch(123),
UNREGISTER_BROKER_RECORD.highestSupportedVersion()));
(short) 0));
ControllerEndpointCollection endpointsFor1001 = new ControllerEndpointCollection();
new ControllerEndpointCollection().add(new ControllerEndpoint().
setHost("localhost").
setName("PLAINTEXT").
setPort(19093).
setSecurityProtocol(SecurityProtocol.PLAINTEXT.id));
DELTA1_RECORDS.add(new ApiMessageAndVersion(new RegisterControllerRecord().
setControllerId(1001).
setIncarnationId(Uuid.fromString("FdEHF-IqScKfYyjZ1CjfNQ")).
setZkMigrationReady(true).
setEndPoints(endpointsFor1001),
(short) 0));
DELTA1 = new ClusterDelta(IMAGE1);
RecordTestUtils.replayAll(DELTA1, DELTA1_RECORDS);
@ -123,7 +147,15 @@ public class ClusterImageTest {
Optional.empty(),
true,
false));
IMAGE2 = new ClusterImage(map2);
Map<Integer, ControllerRegistration> cmap2 = new HashMap<>(cmap1);
cmap2.put(1001, new ControllerRegistration.Builder().
setId(1001).
setIncarnationId(Uuid.fromString("FdEHF-IqScKfYyjZ1CjfNQ")).
setZkMigrationReady(true).
setListeners(Collections.singletonMap("PLAINTEXT",
new Endpoint("PLAINTEXT", SecurityProtocol.PLAINTEXT, "localhost", 19093))).
setSupportedFeatures(Collections.emptyMap()).build());
IMAGE2 = new ClusterImage(map2, cmap2);
}
@Test

View File

@ -0,0 +1,75 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.image.node;
import org.apache.kafka.common.Uuid;
import org.apache.kafka.image.ClusterImage;
import org.apache.kafka.metadata.BrokerRegistration;
import org.apache.kafka.metadata.VersionRange;
import org.apache.kafka.server.common.MetadataVersion;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.Timeout;
import java.util.Arrays;
import java.util.Collections;
import java.util.Optional;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertNull;
@Timeout(value = 40)
public class ClusterImageBrokersNodeTest {
private static final ClusterImage TEST_IMAGE = new ClusterImage(
Collections.singletonMap(1, new BrokerRegistration(1,
1001,
Uuid.fromString("MJkaH0j0RwuC3W2GHQHtWA"),
Collections.emptyList(),
Collections.singletonMap(MetadataVersion.FEATURE_NAME, VersionRange.of(1, 4)),
Optional.empty(),
false,
false)),
Collections.emptyMap());
private final static ClusterImageBrokersNode NODE = new ClusterImageBrokersNode(TEST_IMAGE);
@Test
public void testChildNames() {
assertEquals(Arrays.asList("1"), NODE.childNames());
}
@Test
public void testNode1Child() {
MetadataNode child = NODE.child("1");
assertNotNull(child);
assertEquals("BrokerRegistration(id=1, epoch=1001, " +
"incarnationId=MJkaH0j0RwuC3W2GHQHtWA, " +
"listeners=[], " +
"supportedFeatures={metadata.version: 1-4}, " +
"rack=Optional.empty, " +
"fenced=false, " +
"inControlledShutdown=false, " +
"isMigratingZkBroker=false)", child.stringify());
}
@Test
public void testUnknownChild() {
assertNull(NODE.child("2"));
}
}

View File

@ -0,0 +1,72 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.image.node;
import org.apache.kafka.common.Uuid;
import org.apache.kafka.image.ClusterImage;
import org.apache.kafka.metadata.ControllerRegistration;
import org.apache.kafka.metadata.VersionRange;
import org.apache.kafka.server.common.MetadataVersion;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.Timeout;
import java.util.Arrays;
import java.util.Collections;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertNull;
@Timeout(value = 40)
public class ClusterImageControllersNodeTest {
private static final ClusterImage TEST_IMAGE = new ClusterImage(
Collections.emptyMap(),
Collections.singletonMap(2, new ControllerRegistration.Builder().
setId(2).
setIncarnationId(Uuid.fromString("adGo6sTPS0uJshjvdTUmqQ")).
setZkMigrationReady(false).
setSupportedFeatures(Collections.singletonMap(
MetadataVersion.FEATURE_NAME, VersionRange.of(1, 4))).
setListeners(Collections.emptyMap()).
build()));
private final static ClusterImageControllersNode NODE = new ClusterImageControllersNode(TEST_IMAGE);
@Test
public void testChildNames() {
assertEquals(Arrays.asList("2"), NODE.childNames());
}
@Test
public void testNode1Child() {
MetadataNode child = NODE.child("2");
assertNotNull(child);
assertEquals("ControllerRegistration(id=2, " +
"incarnationId=adGo6sTPS0uJshjvdTUmqQ, " +
"zkMigrationReady=false, " +
"listeners=[], " +
"supportedFeatures={metadata.version: 1-4})",
child.stringify());
}
@Test
public void testUnknownChild() {
assertNull(NODE.child("1"));
}
}

View File

@ -0,0 +1,58 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.image.node;
import org.apache.kafka.image.ClusterImage;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.Timeout;
import java.util.Arrays;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertNull;
@Timeout(value = 40)
public class ClusterImageNodeTest {
private final static ClusterImageNode NODE = new ClusterImageNode(ClusterImage.EMPTY);
@Test
public void testChildNames() {
assertEquals(Arrays.asList("brokers", "controllers"), NODE.childNames());
}
@Test
public void testBrokersChild() {
MetadataNode child = NODE.child("brokers");
assertNotNull(child);
assertEquals(ClusterImageBrokersNode.class, child.getClass());
}
@Test
public void testControllersChild() {
MetadataNode child = NODE.child("controllers");
assertNotNull(child);
assertEquals(ClusterImageControllersNode.class, child.getClass());
}
@Test
public void testUnknownChild() {
assertNull(NODE.child("unknown"));
}
}

View File

@ -0,0 +1,149 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.image.publisher;
import org.junit.jupiter.api.Timeout;
@Timeout(value = 40)
public class ControllerRegistrationsPublisherTest {
/*
@Test
public void testInitialControllers() {
ControllerRegistrationsPublisher publisher = new ControllerRegistrationsPublisher();
assertEquals(Collections.emptyMap(), publisher.controllers());
}
@Test
public void testName() {
ControllerRegistrationsPublisher publisher = new ControllerRegistrationsPublisher();
assertEquals("ControllerRegistrationsPublisher", publisher.name());
}
private static final MetadataDelta TEST_DELTA;
private static final MetadataImage TEST_IMAGE;
static {
List<ApiMessageAndVersion> records = Arrays.asList(
new ApiMessageAndVersion(
new RegisterControllerRecord().
setControllerId(0).
setIncarnationId(Uuid.fromString("1IAc4mS9RgqR00apcA2UTQ")).
setZkMigrationReady(false).
setEndPoints(
new ControllerEndpointCollection(Arrays.asList(
new ControllerEndpoint().
setName("CONTROLLER").
setHost("example.com").
setPort(8080).
setSecurityProtocol(SecurityProtocol.SASL_PLAINTEXT.id),
new ControllerEndpoint().
setName("CONTROLLER_SSL").
setHost("example.com").
setPort(8090).
setSecurityProtocol(SecurityProtocol.SASL_SSL.id)
).iterator())).
setFeatures(
new ControllerFeatureCollection(Arrays.asList(
new ControllerFeature().
setName(MetadataVersion.FEATURE_NAME).
setMinSupportedVersion((short) 1).
setMaxSupportedVersion((short) 13)
).iterator())
),
(short) 0),
new ApiMessageAndVersion(
new RegisterControllerRecord().
setControllerId(1).
setIncarnationId(Uuid.fromString("yOVziEQLQO6HQK0J76EeFw")).
setZkMigrationReady(false).
setEndPoints(
new ControllerEndpointCollection(Arrays.asList(
new ControllerEndpoint().
setName("CONTROLLER").
setHost("example.com").
setPort(8081).
setSecurityProtocol(SecurityProtocol.SASL_PLAINTEXT.id),
new ControllerEndpoint().
setName("CONTROLLER_SSL").
setHost("example.com").
setPort(8091).
setSecurityProtocol(SecurityProtocol.SASL_SSL.id)
).iterator())).
setFeatures(
new ControllerFeatureCollection(Arrays.asList(
new ControllerFeature().
setName(MetadataVersion.FEATURE_NAME).
setMinSupportedVersion((short) 1).
setMaxSupportedVersion((short) 13)
).iterator())
),
(short) 0),
new ApiMessageAndVersion(
new RegisterControllerRecord().
setControllerId(2).
setIncarnationId(Uuid.fromString("4JXjhEtARYO85g-o3I4Ieg")).
setZkMigrationReady(false).
setEndPoints(
new ControllerEndpointCollection(Arrays.asList(
new ControllerEndpoint().
setName("CONTROLLER").
setHost("example.com").
setPort(8082).
setSecurityProtocol(SecurityProtocol.SASL_PLAINTEXT.id),
new ControllerEndpoint().
setName("CONTROLLER_SSL").
setHost("example.com").
setPort(8092).
setSecurityProtocol(SecurityProtocol.SASL_SSL.id)
).iterator())).
setFeatures(
new ControllerFeatureCollection(Arrays.asList(
new ControllerFeature().
setName(MetadataVersion.FEATURE_NAME).
setMinSupportedVersion((short) 1).
setMaxSupportedVersion((short) 13)
).iterator())
),
(short) 0)
);
TEST_DELTA = new MetadataDelta.Builder().build();
RecordTestUtils.replayAll(TEST_DELTA, records);
TEST_IMAGE = TEST_DELTA.image();
}
@ParameterizedTest
@ValueSource(booleans = {false, true})
public void testOnMetadataUpdate(boolean fromSnapshot) {
ControllerRegistrationsPublisher publisher = new ControllerRegistrationsPublisher();
if (fromSnapshot) {
publisher.onMetadataUpdate(TEST_DELTA, TEST_IMAGE,
new SnapshotManifest(new MetadataProvenance(100L, 10, 2000L), 100L));
} else {
publisher.onMetadataUpdate(TEST_DELTA, TEST_IMAGE,
new LogDeltaManifest(new MetadataProvenance(100L, 10, 2000L),
new LeaderAndEpoch(OptionalInt.of(1), 200),
3,
1000L,
234));
}
assertEquals(new HashSet<>(Arrays.asList(0, 1, 2)), publisher.controllers().keySet());
}
*/
}

View File

@ -0,0 +1,137 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.metadata;
import org.apache.kafka.common.Endpoint;
import org.apache.kafka.common.Node;
import org.apache.kafka.common.Uuid;
import org.apache.kafka.common.metadata.RegisterControllerRecord;
import org.apache.kafka.common.security.auth.SecurityProtocol;
import org.apache.kafka.image.writer.ImageWriterOptions;
import org.apache.kafka.server.common.ApiMessageAndVersion;
import org.apache.kafka.server.common.MetadataVersion;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.Timeout;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNotEquals;
@Timeout(value = 40)
public class ControllerRegistrationTest {
static <K, V> Map<K, V> doubleMap(K k1, V v1, K k2, V v2) {
HashMap<K, V> map = new HashMap<>();
map.put(k1, v1);
map.put(k2, v2);
return Collections.unmodifiableMap(map);
}
private static final List<ControllerRegistration> REGISTRATIONS = Arrays.asList(
new ControllerRegistration.Builder().
setId(0).
setIncarnationId(Uuid.fromString("ycRmGrOFQru7HXf6fOybZQ")).
setZkMigrationReady(true).
setListeners(doubleMap(
"PLAINTEXT", new Endpoint("PLAINTEXT", SecurityProtocol.PLAINTEXT, "localhost", 9107),
"SSL", new Endpoint("SSL", SecurityProtocol.SSL, "localhost", 9207))).
setSupportedFeatures(Collections.singletonMap(MetadataVersion.FEATURE_NAME, VersionRange.of(1, 10))).
build(),
new ControllerRegistration.Builder().
setId(1).
setIncarnationId(Uuid.fromString("ubT_wuD6R3uopZ_lV76dQg")).
setZkMigrationReady(true).
setListeners(doubleMap(
"PLAINTEXT", new Endpoint("PLAINTEXT", SecurityProtocol.PLAINTEXT, "localhost", 9108),
"SSL", new Endpoint("SSL", SecurityProtocol.SSL, "localhost", 9208))).
setSupportedFeatures(Collections.singletonMap(MetadataVersion.FEATURE_NAME, VersionRange.of(1, 10))).
build(),
new ControllerRegistration.Builder().
setId(2).
setIncarnationId(Uuid.fromString("muQS341gRIeNh9Ps7reDSw")).
setZkMigrationReady(false).
setListeners(doubleMap(
"PLAINTEXT", new Endpoint("PLAINTEXT", SecurityProtocol.PLAINTEXT, "localhost", 9109),
"SSL", new Endpoint("SSL", SecurityProtocol.SSL, "localhost", 9209))).
setSupportedFeatures(Collections.singletonMap(MetadataVersion.FEATURE_NAME, VersionRange.of(1, 10))).
build()
);
@Test
public void testValues() {
assertEquals(0, REGISTRATIONS.get(0).id());
assertEquals(1, REGISTRATIONS.get(1).id());
assertEquals(2, REGISTRATIONS.get(2).id());
}
@Test
public void testEquals() {
assertNotEquals(REGISTRATIONS.get(0), REGISTRATIONS.get(1));
assertNotEquals(REGISTRATIONS.get(1), REGISTRATIONS.get(0));
assertNotEquals(REGISTRATIONS.get(0), REGISTRATIONS.get(2));
assertNotEquals(REGISTRATIONS.get(2), REGISTRATIONS.get(0));
assertEquals(REGISTRATIONS.get(0), REGISTRATIONS.get(0));
assertEquals(REGISTRATIONS.get(1), REGISTRATIONS.get(1));
assertEquals(REGISTRATIONS.get(2), REGISTRATIONS.get(2));
}
@Test
public void testToString() {
assertEquals("ControllerRegistration(id=1, " +
"incarnationId=ubT_wuD6R3uopZ_lV76dQg, " +
"zkMigrationReady=true, " +
"listeners=[" +
"Endpoint(listenerName='PLAINTEXT', securityProtocol=PLAINTEXT, host='localhost', port=9108), " +
"Endpoint(listenerName='SSL', securityProtocol=SSL, host='localhost', port=9208)]" +
", supportedFeatures={metadata.version: 1-10})",
REGISTRATIONS.get(1).toString());
}
@Test
public void testFromRecordAndToRecord() {
testRoundTrip(REGISTRATIONS.get(0));
testRoundTrip(REGISTRATIONS.get(1));
testRoundTrip(REGISTRATIONS.get(2));
}
private void testRoundTrip(ControllerRegistration registration) {
ApiMessageAndVersion messageAndVersion = registration.
toRecord(new ImageWriterOptions.Builder().build());
ControllerRegistration registration2 = ControllerRegistration.fromRecord(
(RegisterControllerRecord) messageAndVersion.message());
assertEquals(registration, registration2);
ApiMessageAndVersion messageAndVersion2 = registration2.
toRecord(new ImageWriterOptions.Builder().build());
assertEquals(messageAndVersion, messageAndVersion2);
}
@Test
public void testToNode() {
assertEquals(Optional.empty(), REGISTRATIONS.get(0).node("NONEXISTENT"));
assertEquals(Optional.of(new Node(0, "localhost", 9107, null)),
REGISTRATIONS.get(0).node("PLAINTEXT"));
assertEquals(Optional.of(new Node(0, "localhost", 9207, null)),
REGISTRATIONS.get(0).node("SSL"));
}
}

View File

@ -18,21 +18,29 @@
package org.apache.kafka.metadata;
import org.apache.kafka.common.Uuid;
import org.apache.kafka.common.metadata.FeatureLevelRecord;
import org.apache.kafka.common.metadata.RegisterControllerRecord;
import org.apache.kafka.common.metadata.TopicRecord;
import org.apache.kafka.common.protocol.ApiMessage;
import org.apache.kafka.common.protocol.Message;
import org.apache.kafka.common.protocol.ObjectSerializationCache;
import org.apache.kafka.common.security.auth.SecurityProtocol;
import org.apache.kafka.common.utils.ImplicitLinkedHashCollection;
import org.apache.kafka.image.MetadataDelta;
import org.apache.kafka.image.MetadataImage;
import org.apache.kafka.metadata.migration.ZkMigrationState;
import org.apache.kafka.raft.Batch;
import org.apache.kafka.raft.BatchReader;
import org.apache.kafka.raft.internals.MemoryBatchReader;
import org.apache.kafka.server.common.ApiMessageAndVersion;
import org.apache.kafka.server.common.MetadataVersion;
import org.apache.kafka.server.util.MockRandom;
import java.lang.reflect.Field;
import java.lang.reflect.InvocationTargetException;
import java.lang.reflect.Method;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashSet;
@ -289,4 +297,36 @@ public class RecordTestUtils {
new TopicRecord().setName("test" + index).
setTopicId(new Uuid(random.nextLong(), random.nextLong())), (short) 0);
}
public static RegisterControllerRecord createTestControllerRegistration(
int id,
boolean zkMigrationReady
) {
return new RegisterControllerRecord().
setControllerId(id).
setIncarnationId(new Uuid(3465346L, id)).
setZkMigrationReady(zkMigrationReady).
setEndPoints(new RegisterControllerRecord.ControllerEndpointCollection(
Arrays.asList(
new RegisterControllerRecord.ControllerEndpoint().
setName("CONTROLLER").
setHost("localhost").
setPort(8000 + id).
setSecurityProtocol(SecurityProtocol.PLAINTEXT.id),
new RegisterControllerRecord.ControllerEndpoint().
setName("CONTROLLER_SSL").
setHost("localhost").
setPort(9000 + id).
setSecurityProtocol(SecurityProtocol.SSL.id)
).iterator()
)).
setFeatures(new RegisterControllerRecord.ControllerFeatureCollection(
Arrays.asList(
new RegisterControllerRecord.ControllerFeature().
setName(MetadataVersion.FEATURE_NAME).
setMinSupportedVersion(MetadataVersion.MINIMUM_KRAFT_VERSION.featureLevel()).
setMaxSupportedVersion(MetadataVersion.IBP_3_6_IV1.featureLevel())
).iterator()
));
}
}

View File

@ -16,15 +16,15 @@
*/
package org.apache.kafka.metadata.migration;
import org.apache.kafka.clients.ApiVersions;
import org.apache.kafka.clients.NodeApiVersions;
import org.apache.kafka.common.Node;
import org.apache.kafka.common.TopicIdPartition;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.Uuid;
import org.apache.kafka.common.config.ConfigResource;
import org.apache.kafka.common.metadata.BrokerRegistrationChangeRecord;
import org.apache.kafka.common.metadata.ConfigRecord;
import org.apache.kafka.common.metadata.FeatureLevelRecord;
import org.apache.kafka.common.metadata.RegisterBrokerRecord;
import org.apache.kafka.common.metadata.RegisterControllerRecord;
import org.apache.kafka.common.utils.MockTime;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.controller.QuorumFeatures;
@ -48,10 +48,10 @@ import org.apache.kafka.metadata.RecordTestUtils;
import org.apache.kafka.raft.LeaderAndEpoch;
import org.apache.kafka.raft.OffsetAndEpoch;
import org.apache.kafka.server.common.ApiMessageAndVersion;
import org.apache.kafka.server.common.MetadataVersion;
import org.apache.kafka.server.fault.MockFaultHandler;
import org.apache.kafka.test.TestUtils;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.ValueSource;
@ -82,16 +82,9 @@ import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertTrue;
public class KRaftMigrationDriverTest {
List<Node> controllerNodes = Arrays.asList(
new Node(4, "host4", 0),
new Node(5, "host5", 0),
new Node(6, "host6", 0)
);
ApiVersions apiVersions = new ApiVersions();
QuorumFeatures quorumFeatures = QuorumFeatures.create(4,
apiVersions,
QuorumFeatures quorumFeatures = new QuorumFeatures(4,
QuorumFeatures.defaultFeatureMap(),
controllerNodes);
Arrays.asList(4, 5, 6));
static class MockControllerMetrics extends QuorumControllerMetrics {
final AtomicBoolean closed = new AtomicBoolean(false);
@ -131,13 +124,6 @@ public class KRaftMigrationDriverTest {
.setTime(mockTime);
}
@BeforeEach
public void setup() {
apiVersions.update("4", new NodeApiVersions(Collections.emptyList(), Collections.emptyList(), true));
apiVersions.update("5", new NodeApiVersions(Collections.emptyList(), Collections.emptyList(), true));
apiVersions.update("6", new NodeApiVersions(Collections.emptyList(), Collections.emptyList(), true));
}
static class NoOpRecordConsumer implements ZkRecordConsumer {
@Override
public void beginMigration() {
@ -356,11 +342,30 @@ public class KRaftMigrationDriverTest {
}
}
@Test
public void testShouldNotMoveToNextStateIfControllerNodesAreNotReadyToMigrate() throws Exception {
private void setupDeltaWithControllerRegistrations(
MetadataDelta delta,
List<Integer> notReadyIds,
List<Integer> readyIds
) {
delta.replay(new FeatureLevelRecord().
setName(MetadataVersion.FEATURE_NAME).
setFeatureLevel(MetadataVersion.IBP_3_6_IV2.featureLevel()));
delta.replay(ZkMigrationState.PRE_MIGRATION.toRecord().message());
for (int id : notReadyIds) {
delta.replay(RecordTestUtils.createTestControllerRegistration(id, false));
}
for (int id : readyIds) {
delta.replay(RecordTestUtils.createTestControllerRegistration(id, true));
}
}
@ParameterizedTest
@ValueSource(booleans = {false, true})
public void testShouldNotMoveToNextStateIfControllerNodesAreNotReadyToMigrate(
boolean allNodePresent
) throws Exception {
CountingMetadataPropagator metadataPropagator = new CountingMetadataPropagator();
CapturingMigrationClient migrationClient = CapturingMigrationClient.newBuilder().setBrokersInZk(1).build();
apiVersions.remove("6");
KRaftMigrationDriver.Builder builder = defaultTestBuilder()
.setZkMigrationClient(migrationClient)
@ -370,7 +375,11 @@ public class KRaftMigrationDriverTest {
MetadataDelta delta = new MetadataDelta(image);
driver.start();
delta.replay(ZkMigrationState.PRE_MIGRATION.toRecord().message());
if (allNodePresent) {
setupDeltaWithControllerRegistrations(delta, Arrays.asList(4, 5, 6), Arrays.asList());
} else {
setupDeltaWithControllerRegistrations(delta, Arrays.asList(), Arrays.asList(4, 5));
}
delta.replay(zkBrokerRecord(1));
MetadataProvenance provenance = new MetadataProvenance(100, 1, 1);
image = delta.apply(provenance);
@ -380,16 +389,18 @@ public class KRaftMigrationDriverTest {
driver.onControllerChange(newLeader);
driver.onMetadataUpdate(delta, image, new LogDeltaManifest(provenance, newLeader, 1, 100, 42));
// Current apiVersions are missing the controller node 6, should stay at WAIT_FOR_CONTROLLER_QUORUM state
// Not all controller nodes are ready. So we should stay at WAIT_FOR_CONTROLLER_QUORUM state.
TestUtils.waitForCondition(() -> driver.migrationState().get(1, TimeUnit.MINUTES).equals(MigrationDriverState.WAIT_FOR_CONTROLLER_QUORUM),
"Waiting for KRaftMigrationDriver to enter WAIT_FOR_CONTROLLER_QUORUM state");
// Current apiVersions of node 6 has no zkMigrationReady set, should still stay at WAIT_FOR_CONTROLLER_QUORUM state
apiVersions.update("6", NodeApiVersions.create());
// Controller nodes don't have zkMigrationReady set. Should still stay at WAIT_FOR_CONTROLLER_QUORUM state.
assertEquals(MigrationDriverState.WAIT_FOR_CONTROLLER_QUORUM, driver.migrationState().get(1, TimeUnit.MINUTES));
// all controller nodes are zkMigrationReady, should be able to move to next state
apiVersions.update("6", new NodeApiVersions(Collections.emptyList(), Collections.emptyList(), true));
// Update so that all controller nodes are zkMigrationReady. Now we should be able to move to the next state.
delta = new MetadataDelta(image);
setupDeltaWithControllerRegistrations(delta, Arrays.asList(), Arrays.asList(4, 5, 6));
image = delta.apply(new MetadataProvenance(200, 1, 2));
driver.onMetadataUpdate(delta, image, new LogDeltaManifest(image.provenance(), newLeader, 1, 100, 42));
TestUtils.waitForCondition(() -> driver.migrationState().get(1, TimeUnit.MINUTES).equals(MigrationDriverState.DUAL_WRITE),
"Waiting for KRaftMigrationDriver to enter DUAL_WRITE state");
}

View File

@ -180,7 +180,10 @@ public enum MetadataVersion {
IBP_3_6_IV0(12, "3.6", "IV0", false),
// Add metadata transactions
IBP_3_6_IV1(13, "3.6", "IV1", true);
IBP_3_6_IV1(13, "3.6", "IV1", true),
// Implement KIP-919 controller registration.
IBP_3_6_IV2(14, "3.6", "IV2", true);
// NOTE: update the default version in @ClusterTest annotation to point to the latest version
public static final String FEATURE_NAME = "metadata.version";
@ -311,6 +314,19 @@ public enum MetadataVersion {
}
}
public short registerControllerRecordVersion() {
if (isAtLeast(MetadataVersion.IBP_3_6_IV2)) {
return (short) 0;
} else {
throw new RuntimeException("Controller registration is not supported in " +
"MetadataVersion " + this);
}
}
public boolean isControllerRegistrationSupported() {
return this.isAtLeast(MetadataVersion.IBP_3_6_IV2);
}
public short fetchRequestVersion() {
if (this.isAtLeast(IBP_3_5_IV1)) {
return 15;

View File

@ -125,7 +125,7 @@ public class FeatureCommandTest {
"disable", "--feature", "metadata.version"))
);
assertEquals("Could not disable metadata.version. Invalid update version 0 for feature " +
"metadata.version. Local controller 3000 only supports versions 1-13", commandOutput);
"metadata.version. Local controller 3000 only supports versions 1-14", commandOutput);
commandOutput = ToolsTestUtils.captureStandardOut(() ->
assertEquals(1, FeatureCommand.mainNoExit("--bootstrap-server", cluster.bootstrapServers(),