From fb4f297207ef62f71e4a6d2d0dac75752933043d Mon Sep 17 00:00:00 2001 From: Kowshik Prakasam Date: Wed, 7 Oct 2020 10:23:16 -0700 Subject: [PATCH] KAFKA-10028: Implement write path for feature versioning system (KIP-584) (#9001) Summary: In this PR, I have implemented the write path of the feature versioning system (KIP-584). Here is a summary of what's in this PR: New APIs in org.apache.kafka.clients.admin.Admin interface, and their client and server implementations. These APIs can be used to describe features and update finalized features. These APIs are: Admin#describeFeatures and Admin#updateFeatures. The write path is provided by the Admin#updateFeatures API. The corresponding server-side implementation is provided in KafkaApis and KafkaController classes. This can be a good place to start the code review. The write path is supplemented by Admin#describeFeatures client API. This does not translate 1:1 to a server-side API. Instead, under the hood the API makes an explicit ApiVersionsRequest to the Broker to fetch the supported and finalized features. Implemented a suite of integration tests in UpdateFeaturesTest.scala that thoroughly exercises the various cases in the write path. Other changes: The data type of the FinalizedFeaturesEpoch field in ApiVersionsResponse has been modified from int32 to int64. This change is to conform with the latest changes to the KIP explained in the voting thread. Along the way, the class SupportedFeatures has been renamed to be called BrokerFeatures, and, it now holds both supported features as well as default minimum version levels. For the purpose of testing, both the BrokerFeatures and FinalizedFeatureCache classes have been changed to be no longer singleton in implementation. Instead, these are now instantiated once and maintained in KafkaServer. The singleton instances are passed around to various classes, as needed. Reviewers: Boyang Chen , Jun Rao --- .../org/apache/kafka/clients/admin/Admin.java | 68 ++ .../admin/DescribeFeaturesOptions.java | 48 ++ .../clients/admin/DescribeFeaturesResult.java | 37 ++ .../kafka/clients/admin/FeatureMetadata.java | 111 ++++ .../kafka/clients/admin/FeatureUpdate.java | 78 +++ .../clients/admin/FinalizedVersionRange.java | 84 +++ .../kafka/clients/admin/KafkaAdminClient.java | 152 +++++ .../clients/admin/SupportedVersionRange.java | 82 +++ .../clients/admin/UpdateFeaturesOptions.java | 29 + .../clients/admin/UpdateFeaturesResult.java | 48 ++ .../errors/FeatureUpdateFailedException.java | 29 + .../common/feature/BaseVersionRange.java | 21 +- .../common/feature/FinalizedVersionRange.java | 4 +- .../common/feature/SupportedVersionRange.java | 10 +- .../apache/kafka/common/protocol/ApiKeys.java | 6 +- .../apache/kafka/common/protocol/Errors.java | 4 +- .../common/requests/AbstractRequest.java | 2 + .../common/requests/AbstractResponse.java | 2 + .../common/requests/ApiVersionsResponse.java | 58 +- .../requests/UpdateFeaturesRequest.java | 95 +++ .../requests/UpdateFeaturesResponse.java | 109 ++++ .../common/message/ApiVersionsResponse.json | 4 +- .../common/message/UpdateFeaturesRequest.json | 35 ++ .../message/UpdateFeaturesResponse.json | 39 ++ .../clients/admin/KafkaAdminClientTest.java | 236 +++++++ .../kafka/clients/admin/MockAdminClient.java | 10 + .../kafka/controller/ControllerState.scala | 7 +- .../kafka/controller/KafkaController.scala | 434 ++++++++++++- .../scala/kafka/server/BrokerFeatures.scala | 116 ++++ .../kafka/server/FinalizedFeatureCache.scala | 91 ++- .../FinalizedFeatureChangeListener.scala | 16 +- .../main/scala/kafka/server/KafkaApis.scala | 56 +- .../main/scala/kafka/server/KafkaConfig.scala | 2 +- .../main/scala/kafka/server/KafkaServer.scala | 18 +- .../kafka/server/SupportedFeatures.scala | 93 --- .../main/scala/kafka/zk/KafkaZkClient.scala | 3 +- core/src/main/scala/kafka/zk/ZkData.scala | 41 +- .../ControllerIntegrationTest.scala | 104 +++- .../kafka/server/BrokerFeaturesTest.scala | 106 ++++ .../server/FinalizedFeatureCacheTest.scala | 78 +-- .../FinalizedFeatureChangeListenerTest.scala | 144 +++-- .../unit/kafka/server/KafkaApisTest.scala | 7 +- .../unit/kafka/server/RequestQuotaTest.scala | 3 + .../kafka/server/SupportedFeaturesTest.scala | 56 -- .../kafka/server/UpdateFeaturesTest.scala | 581 ++++++++++++++++++ .../metadata/MetadataRequestBenchmark.java | 7 +- 46 files changed, 3005 insertions(+), 359 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/clients/admin/DescribeFeaturesOptions.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/admin/DescribeFeaturesResult.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/admin/FeatureMetadata.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/admin/FeatureUpdate.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/admin/FinalizedVersionRange.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/admin/SupportedVersionRange.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/admin/UpdateFeaturesOptions.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/admin/UpdateFeaturesResult.java create mode 100644 clients/src/main/java/org/apache/kafka/common/errors/FeatureUpdateFailedException.java create mode 100644 clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesRequest.java create mode 100644 clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesResponse.java create mode 100644 clients/src/main/resources/common/message/UpdateFeaturesRequest.json create mode 100644 clients/src/main/resources/common/message/UpdateFeaturesResponse.json create mode 100644 core/src/main/scala/kafka/server/BrokerFeatures.scala delete mode 100644 core/src/main/scala/kafka/server/SupportedFeatures.scala create mode 100644 core/src/test/scala/unit/kafka/server/BrokerFeaturesTest.scala delete mode 100644 core/src/test/scala/unit/kafka/server/SupportedFeaturesTest.scala create mode 100644 core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java b/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java index 53ed92afd8888..96620df17fba3 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java @@ -37,6 +37,7 @@ import org.apache.kafka.common.acl.AclBindingFilter; import org.apache.kafka.common.annotation.InterfaceStability; import org.apache.kafka.common.config.ConfigResource; +import org.apache.kafka.common.errors.FeatureUpdateFailedException; import org.apache.kafka.common.quota.ClientQuotaAlteration; import org.apache.kafka.common.quota.ClientQuotaFilter; import org.apache.kafka.common.requests.LeaveGroupResponse; @@ -1306,6 +1307,73 @@ default AlterUserScramCredentialsResult alterUserScramCredentials(List alterations, AlterUserScramCredentialsOptions options); + /** + * Describes finalized as well as supported features. By default, the request is issued to any + * broker. It can be optionally directed only to the controller via DescribeFeaturesOptions + * parameter. This is particularly useful if the user requires strongly consistent reads of + * finalized features. + *

+ * The following exceptions can be anticipated when calling {@code get()} on the future from the + * returned {@link DescribeFeaturesResult}: + *

    + *
  • {@link org.apache.kafka.common.errors.TimeoutException} + * If the request timed out before the describe operation could finish.
  • + *
+ *

+ * @param options the options to use + * + * @return the {@link DescribeFeaturesResult} containing the result + */ + DescribeFeaturesResult describeFeatures(DescribeFeaturesOptions options); + + /** + * Applies specified updates to finalized features. This operation is not transactional so some + * updates may succeed while the rest may fail. + *

+ * The API takes in a map of finalized feature names to {@link FeatureUpdate} that needs to be + * applied. Each entry in the map specifies the finalized feature to be added or updated or + * deleted, along with the new max feature version level value. This request is issued only to + * the controller since the API is only served by the controller. The return value contains an + * error code for each supplied {@link FeatureUpdate}, and the code indicates if the update + * succeeded or failed in the controller. + *

    + *
  • Downgrade of feature version level is not a regular operation/intent. It is only allowed + * in the controller if the {@link FeatureUpdate} has the allowDowngrade flag set. Setting this + * flag conveys user intent to attempt downgrade of a feature max version level. Note that + * despite the allowDowngrade flag being set, certain downgrades may be rejected by the + * controller if it is deemed impossible.
  • + *
  • Deletion of a finalized feature version is not a regular operation/intent. It could be + * done by setting the allowDowngrade flag to true in the {@link FeatureUpdate}, and, setting + * the max version level to a value less than 1.
  • + *
+ *

+ * The following exceptions can be anticipated when calling {@code get()} on the futures + * obtained from the returned {@link UpdateFeaturesResult}: + *

    + *
  • {@link org.apache.kafka.common.errors.ClusterAuthorizationException} + * If the authenticated user didn't have alter access to the cluster.
  • + *
  • {@link org.apache.kafka.common.errors.InvalidRequestException} + * If the request details are invalid. e.g., a non-existing finalized feature is attempted + * to be deleted or downgraded.
  • + *
  • {@link org.apache.kafka.common.errors.TimeoutException} + * If the request timed out before the updates could finish. It cannot be guaranteed whether + * the updates succeeded or not.
  • + *
  • {@link FeatureUpdateFailedException} + * This means there was an unexpected error encountered when the update was applied on + * the controller. There is no guarantee on whether the update succeeded or failed. The best + * way to find out is to issue a {@link Admin#describeFeatures(DescribeFeaturesOptions)} + * request to the controller to get the latest features.
  • + *
+ *

+ * This operation is supported by brokers with version 2.7.0 or higher. + + * @param featureUpdates the map of finalized feature name to {@link FeatureUpdate} + * @param options the options to use + * + * @return the {@link UpdateFeaturesResult} containing the result + */ + UpdateFeaturesResult updateFeatures(Map featureUpdates, UpdateFeaturesOptions options); + /** * Get the metrics kept by the adminClient */ diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/DescribeFeaturesOptions.java b/clients/src/main/java/org/apache/kafka/clients/admin/DescribeFeaturesOptions.java new file mode 100644 index 0000000000000..4a37956aab98c --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/admin/DescribeFeaturesOptions.java @@ -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.apache.kafka.common.annotation.InterfaceStability; + +/** + * Options for {@link AdminClient#describeFeatures(DescribeFeaturesOptions)}. + * + * The API of this class is evolving. See {@link Admin} for details. + */ +@InterfaceStability.Evolving +public class DescribeFeaturesOptions extends AbstractOptions { + + /** + * - True means the {@link Admin#describeFeatures(DescribeFeaturesOptions)} request must be + * issued only to the controller. + * - False means the {@link Admin#describeFeatures(DescribeFeaturesOptions)} request can be + * issued to any random broker. + */ + private boolean sendRequestToController = false; + + /** + * Sets a flag indicating that the describe features request must be issued only to the controller. + */ + public DescribeFeaturesOptions sendRequestToController(boolean sendRequestToController) { + this.sendRequestToController = sendRequestToController; + return this; + } + + public boolean sendRequestToController() { + return sendRequestToController; + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/DescribeFeaturesResult.java b/clients/src/main/java/org/apache/kafka/clients/admin/DescribeFeaturesResult.java new file mode 100644 index 0000000000000..c48dc19143077 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/admin/DescribeFeaturesResult.java @@ -0,0 +1,37 @@ +/* + * 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.apache.kafka.common.KafkaFuture; + +/** + * The result of the {@link Admin#describeFeatures(DescribeFeaturesOptions)} call. + * + * The API of this class is evolving, see {@link Admin} for details. + */ +public class DescribeFeaturesResult { + + private final KafkaFuture future; + + DescribeFeaturesResult(KafkaFuture future) { + this.future = future; + } + + public KafkaFuture featureMetadata() { + return future; + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/FeatureMetadata.java b/clients/src/main/java/org/apache/kafka/clients/admin/FeatureMetadata.java new file mode 100644 index 0000000000000..815f9e3b97ca6 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/admin/FeatureMetadata.java @@ -0,0 +1,111 @@ +/* + * 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 static java.util.stream.Collectors.joining; + +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; + +/** + * Encapsulates details about finalized as well as supported features. This is particularly useful + * to hold the result returned by the {@link Admin#describeFeatures(DescribeFeaturesOptions)} API. + */ +public class FeatureMetadata { + + private final Map finalizedFeatures; + + private final Optional finalizedFeaturesEpoch; + + private final Map supportedFeatures; + + FeatureMetadata(final Map finalizedFeatures, + final Optional finalizedFeaturesEpoch, + final Map supportedFeatures) { + this.finalizedFeatures = new HashMap<>(finalizedFeatures); + this.finalizedFeaturesEpoch = finalizedFeaturesEpoch; + this.supportedFeatures = new HashMap<>(supportedFeatures); + } + + /** + * Returns a map of finalized feature versions. Each entry in the map contains a key being a + * feature name and the value being a range of version levels supported by every broker in the + * cluster. + */ + public Map finalizedFeatures() { + return new HashMap<>(finalizedFeatures); + } + + /** + * The epoch for the finalized features. + * If the returned value is empty, it means the finalized features are absent/unavailable. + */ + public Optional finalizedFeaturesEpoch() { + return finalizedFeaturesEpoch; + } + + /** + * Returns a map of supported feature versions. Each entry in the map contains a key being a + * feature name and the value being a range of versions supported by a particular broker in the + * cluster. + */ + public Map supportedFeatures() { + return new HashMap<>(supportedFeatures); + } + + @Override + public boolean equals(Object other) { + if (this == other) { + return true; + } + if (!(other instanceof FeatureMetadata)) { + return false; + } + + final FeatureMetadata that = (FeatureMetadata) other; + return Objects.equals(this.finalizedFeatures, that.finalizedFeatures) && + Objects.equals(this.finalizedFeaturesEpoch, that.finalizedFeaturesEpoch) && + Objects.equals(this.supportedFeatures, that.supportedFeatures); + } + + @Override + public int hashCode() { + return Objects.hash(finalizedFeatures, finalizedFeaturesEpoch, supportedFeatures); + } + + private static String mapToString(final Map featureVersionsMap) { + return String.format( + "{%s}", + featureVersionsMap + .entrySet() + .stream() + .map(entry -> String.format("(%s -> %s)", entry.getKey(), entry.getValue())) + .collect(joining(", ")) + ); + } + + @Override + public String toString() { + return String.format( + "FeatureMetadata{finalizedFeatures:%s, finalizedFeaturesEpoch:%s, supportedFeatures:%s}", + mapToString(finalizedFeatures), + finalizedFeaturesEpoch.map(Object::toString).orElse(""), + mapToString(supportedFeatures)); + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/FeatureUpdate.java b/clients/src/main/java/org/apache/kafka/clients/admin/FeatureUpdate.java new file mode 100644 index 0000000000000..f43d63ed35234 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/admin/FeatureUpdate.java @@ -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 org.apache.kafka.clients.admin; + +import java.util.Objects; + +/** + * Encapsulates details about an update to a finalized feature. + */ +public class FeatureUpdate { + private final short maxVersionLevel; + private final boolean allowDowngrade; + + /** + * @param maxVersionLevel the new maximum version level for the finalized feature. + * a value < 1 is special and indicates that the update is intended to + * delete the finalized feature, and should be accompanied by setting + * the allowDowngrade flag to true. + * @param allowDowngrade - true, if this feature update was meant to downgrade the existing + * maximum version level of the finalized feature. + * - false, otherwise. + */ + public FeatureUpdate(final short maxVersionLevel, final boolean allowDowngrade) { + if (maxVersionLevel < 1 && !allowDowngrade) { + throw new IllegalArgumentException(String.format( + "The allowDowngrade flag should be set when the provided maxVersionLevel:%d is < 1.", + maxVersionLevel)); + } + this.maxVersionLevel = maxVersionLevel; + this.allowDowngrade = allowDowngrade; + } + + public short maxVersionLevel() { + return maxVersionLevel; + } + + public boolean allowDowngrade() { + return allowDowngrade; + } + + @Override + public boolean equals(Object other) { + if (this == other) { + return true; + } + + if (!(other instanceof FeatureUpdate)) { + return false; + } + + final FeatureUpdate that = (FeatureUpdate) other; + return this.maxVersionLevel == that.maxVersionLevel && this.allowDowngrade == that.allowDowngrade; + } + + @Override + public int hashCode() { + return Objects.hash(maxVersionLevel, allowDowngrade); + } + + @Override + public String toString() { + return String.format("FeatureUpdate{maxVersionLevel:%d, allowDowngrade:%s}", maxVersionLevel, allowDowngrade); + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/FinalizedVersionRange.java b/clients/src/main/java/org/apache/kafka/clients/admin/FinalizedVersionRange.java new file mode 100644 index 0000000000000..aa0401a8a86eb --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/admin/FinalizedVersionRange.java @@ -0,0 +1,84 @@ +/* + * 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 java.util.Objects; + +/** + * Represents a range of version levels supported by every broker in a cluster for some feature. + */ +public class FinalizedVersionRange { + private final short minVersionLevel; + + private final short maxVersionLevel; + + /** + * Raises an exception unless the following condition is met: + * minVersionLevel >= 1 and maxVersionLevel >= 1 and maxVersionLevel >= minVersionLevel. + * + * @param minVersionLevel The minimum version level value. + * @param maxVersionLevel The maximum version level value. + * + * @throws IllegalArgumentException Raised when the condition described above is not met. + */ + FinalizedVersionRange(final short minVersionLevel, final short maxVersionLevel) { + if (minVersionLevel < 1 || maxVersionLevel < 1 || maxVersionLevel < minVersionLevel) { + throw new IllegalArgumentException( + String.format( + "Expected minVersionLevel >= 1, maxVersionLevel >= 1 and" + + " maxVersionLevel >= minVersionLevel, but received" + + " minVersionLevel: %d, maxVersionLevel: %d", minVersionLevel, maxVersionLevel)); + } + this.minVersionLevel = minVersionLevel; + this.maxVersionLevel = maxVersionLevel; + } + + public short minVersionLevel() { + return minVersionLevel; + } + + public short maxVersionLevel() { + return maxVersionLevel; + } + + @Override + public boolean equals(Object other) { + if (this == other) { + return true; + } + if (!(other instanceof FinalizedVersionRange)) { + return false; + } + + final FinalizedVersionRange that = (FinalizedVersionRange) other; + return this.minVersionLevel == that.minVersionLevel && + this.maxVersionLevel == that.maxVersionLevel; + } + + @Override + public int hashCode() { + return Objects.hash(minVersionLevel, maxVersionLevel); + } + + @Override + public String toString() { + return String.format( + "FinalizedVersionRange[min_version_level:%d, max_version_level:%d]", + minVersionLevel, + maxVersionLevel); + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java index 00ad9c5f573df..43dc197157073 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java @@ -79,6 +79,8 @@ import org.apache.kafka.common.message.AlterReplicaLogDirsResponseData.AlterReplicaLogDirPartitionResult; import org.apache.kafka.common.message.AlterReplicaLogDirsResponseData.AlterReplicaLogDirTopicResult; import org.apache.kafka.common.message.AlterUserScramCredentialsRequestData; +import org.apache.kafka.common.message.ApiVersionsResponseData.FinalizedFeatureKey; +import org.apache.kafka.common.message.ApiVersionsResponseData.SupportedFeatureKey; import org.apache.kafka.common.message.CreateAclsRequestData; import org.apache.kafka.common.message.CreateAclsRequestData.AclCreation; import org.apache.kafka.common.message.CreateAclsResponseData.AclCreationResult; @@ -143,6 +145,8 @@ import org.apache.kafka.common.message.OffsetDeleteRequestData.OffsetDeleteRequestTopic; import org.apache.kafka.common.message.OffsetDeleteRequestData.OffsetDeleteRequestTopicCollection; import org.apache.kafka.common.message.RenewDelegationTokenRequestData; +import org.apache.kafka.common.message.UpdateFeaturesRequestData; +import org.apache.kafka.common.message.UpdateFeaturesResponseData.UpdatableFeatureResult; import org.apache.kafka.common.metrics.JmxReporter; import org.apache.kafka.common.metrics.KafkaMetricsContext; import org.apache.kafka.common.metrics.MetricConfig; @@ -169,6 +173,8 @@ import org.apache.kafka.common.requests.AlterUserScramCredentialsRequest; import org.apache.kafka.common.requests.AlterUserScramCredentialsResponse; import org.apache.kafka.common.requests.ApiError; +import org.apache.kafka.common.requests.ApiVersionsRequest; +import org.apache.kafka.common.requests.ApiVersionsResponse; import org.apache.kafka.common.requests.CreateAclsRequest; import org.apache.kafka.common.requests.CreateAclsResponse; import org.apache.kafka.common.requests.CreateDelegationTokenRequest; @@ -226,6 +232,8 @@ import org.apache.kafka.common.requests.OffsetFetchResponse; import org.apache.kafka.common.requests.RenewDelegationTokenRequest; import org.apache.kafka.common.requests.RenewDelegationTokenResponse; +import org.apache.kafka.common.requests.UpdateFeaturesRequest; +import org.apache.kafka.common.requests.UpdateFeaturesResponse; import org.apache.kafka.common.security.auth.KafkaPrincipal; import org.apache.kafka.common.security.scram.internals.ScramFormatter; import org.apache.kafka.common.security.token.delegation.DelegationToken; @@ -4335,6 +4343,150 @@ private static byte[] getSaltedPasword(ScramMechanism publicScramMechanism, byte .hi(password, salt, iterations); } + public DescribeFeaturesResult describeFeatures(final DescribeFeaturesOptions options) { + final KafkaFutureImpl future = new KafkaFutureImpl<>(); + final long now = time.milliseconds(); + final NodeProvider provider = + options.sendRequestToController() ? new ControllerNodeProvider() : new LeastLoadedNodeProvider(); + + final Call call = new Call( + "describeFeatures", calcDeadlineMs(now, options.timeoutMs()), provider) { + + private FeatureMetadata createFeatureMetadata(final ApiVersionsResponse response) { + final Map finalizedFeatures = new HashMap<>(); + for (final FinalizedFeatureKey key : response.data().finalizedFeatures().valuesSet()) { + finalizedFeatures.put(key.name(), new FinalizedVersionRange(key.minVersionLevel(), key.maxVersionLevel())); + } + + Optional finalizedFeaturesEpoch; + if (response.data().finalizedFeaturesEpoch() >= 0L) { + finalizedFeaturesEpoch = Optional.of(response.data().finalizedFeaturesEpoch()); + } else { + finalizedFeaturesEpoch = Optional.empty(); + } + + final Map supportedFeatures = new HashMap<>(); + for (final SupportedFeatureKey key : response.data().supportedFeatures().valuesSet()) { + supportedFeatures.put(key.name(), new SupportedVersionRange(key.minVersion(), key.maxVersion())); + } + + return new FeatureMetadata(finalizedFeatures, finalizedFeaturesEpoch, supportedFeatures); + } + + @Override + ApiVersionsRequest.Builder createRequest(int timeoutMs) { + return new ApiVersionsRequest.Builder(); + } + + @Override + void handleResponse(AbstractResponse response) { + final ApiVersionsResponse apiVersionsResponse = (ApiVersionsResponse) response; + if (apiVersionsResponse.data.errorCode() == Errors.NONE.code()) { + future.complete(createFeatureMetadata(apiVersionsResponse)); + } else if (options.sendRequestToController() && + apiVersionsResponse.data.errorCode() == Errors.NOT_CONTROLLER.code()) { + handleNotControllerError(Errors.NOT_CONTROLLER); + } else { + future.completeExceptionally(Errors.forCode(apiVersionsResponse.data.errorCode()).exception()); + } + } + + @Override + void handleFailure(Throwable throwable) { + completeAllExceptionally(Collections.singletonList(future), throwable); + } + }; + + runnable.call(call, now); + return new DescribeFeaturesResult(future); + } + + @Override + public UpdateFeaturesResult updateFeatures(final Map featureUpdates, + final UpdateFeaturesOptions options) { + if (featureUpdates.isEmpty()) { + throw new IllegalArgumentException("Feature updates can not be null or empty."); + } + + final Map> updateFutures = new HashMap<>(); + for (final Map.Entry entry : featureUpdates.entrySet()) { + updateFutures.put(entry.getKey(), new KafkaFutureImpl<>()); + } + + final long now = time.milliseconds(); + final Call call = new Call("updateFeatures", calcDeadlineMs(now, options.timeoutMs()), + new ControllerNodeProvider()) { + + @Override + UpdateFeaturesRequest.Builder createRequest(int timeoutMs) { + final UpdateFeaturesRequestData.FeatureUpdateKeyCollection featureUpdatesRequestData + = new UpdateFeaturesRequestData.FeatureUpdateKeyCollection(); + for (Map.Entry entry : featureUpdates.entrySet()) { + final String feature = entry.getKey(); + final FeatureUpdate update = entry.getValue(); + if (feature.trim().isEmpty()) { + throw new IllegalArgumentException("Provided feature can not be null or empty."); + } + + final UpdateFeaturesRequestData.FeatureUpdateKey requestItem = + new UpdateFeaturesRequestData.FeatureUpdateKey(); + requestItem.setFeature(feature); + requestItem.setMaxVersionLevel(update.maxVersionLevel()); + requestItem.setAllowDowngrade(update.allowDowngrade()); + featureUpdatesRequestData.add(requestItem); + } + return new UpdateFeaturesRequest.Builder( + new UpdateFeaturesRequestData() + .setTimeoutMs(timeoutMs) + .setFeatureUpdates(featureUpdatesRequestData)); + } + + @Override + void handleResponse(AbstractResponse abstractResponse) { + final UpdateFeaturesResponse response = + (UpdateFeaturesResponse) abstractResponse; + + Errors topLevelError = Errors.forCode(response.data().errorCode()); + switch (topLevelError) { + case NONE: + for (final UpdatableFeatureResult result : response.data().results()) { + final KafkaFutureImpl future = updateFutures.get(result.feature()); + if (future == null) { + log.warn("Server response mentioned unknown feature {}", result.feature()); + } else { + final Errors error = Errors.forCode(result.errorCode()); + if (error == Errors.NONE) { + future.complete(null); + } else { + future.completeExceptionally(error.exception(result.errorMessage())); + } + } + } + // The server should send back a response for every feature, but we do a sanity check anyway. + completeUnrealizedFutures(updateFutures.entrySet().stream(), + feature -> "The controller response did not contain a result for feature " + feature); + break; + case NOT_CONTROLLER: + handleNotControllerError(topLevelError); + break; + default: + for (final Map.Entry> entry : updateFutures.entrySet()) { + entry.getValue().completeExceptionally(topLevelError.exception()); + } + break; + } + } + + @Override + void handleFailure(Throwable throwable) { + completeAllExceptionally(updateFutures.values(), throwable); + } + }; + + runnable.call(call, now); + return new UpdateFeaturesResult(new HashMap<>(updateFutures)); + } + /** * Get a sub level error when the request is in batch. If given key was not found, * return an {@link IllegalArgumentException}. diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/SupportedVersionRange.java b/clients/src/main/java/org/apache/kafka/clients/admin/SupportedVersionRange.java new file mode 100644 index 0000000000000..d71da31fb8200 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/admin/SupportedVersionRange.java @@ -0,0 +1,82 @@ +/* + * 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 java.util.Objects; + +/** + * Represents a range of versions that a particular broker supports for some feature. + */ +public class SupportedVersionRange { + private final short minVersion; + + private final short maxVersion; + + /** + * Raises an exception unless the following conditions are met: + * 1 <= minVersion <= maxVersion. + * + * @param minVersion The minimum version value. + * @param maxVersion The maximum version value. + * + * @throws IllegalArgumentException Raised when the condition described above is not met. + */ + SupportedVersionRange(final short minVersion, final short maxVersion) { + if (minVersion < 1 || maxVersion < 1 || maxVersion < minVersion) { + throw new IllegalArgumentException( + String.format( + "Expected 1 <= minVersion <= maxVersion but received minVersion:%d, maxVersion:%d.", + minVersion, + maxVersion)); + } + this.minVersion = minVersion; + this.maxVersion = maxVersion; + } + + public short minVersion() { + return minVersion; + } + + public short maxVersion() { + return maxVersion; + } + + @Override + public boolean equals(Object other) { + if (this == other) { + return true; + } + + if (other == null || getClass() != other.getClass()) { + return false; + } + + final SupportedVersionRange that = (SupportedVersionRange) other; + return this.minVersion == that.minVersion && this.maxVersion == that.maxVersion; + } + + @Override + public int hashCode() { + return Objects.hash(minVersion, maxVersion); + } + + @Override + public String toString() { + return String.format("SupportedVersionRange[min_version:%d, max_version:%d]", minVersion, maxVersion); + } +} + diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/UpdateFeaturesOptions.java b/clients/src/main/java/org/apache/kafka/clients/admin/UpdateFeaturesOptions.java new file mode 100644 index 0000000000000..7a9f2141b2ab1 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/admin/UpdateFeaturesOptions.java @@ -0,0 +1,29 @@ +/* + * 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 java.util.Map; +import org.apache.kafka.common.annotation.InterfaceStability; + +/** + * Options for {@link AdminClient#updateFeatures(Map, UpdateFeaturesOptions)}. + * + * The API of this class is evolving. See {@link Admin} for details. + */ +@InterfaceStability.Evolving +public class UpdateFeaturesOptions extends AbstractOptions { +} diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/UpdateFeaturesResult.java b/clients/src/main/java/org/apache/kafka/clients/admin/UpdateFeaturesResult.java new file mode 100644 index 0000000000000..6c484dc24d95b --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/admin/UpdateFeaturesResult.java @@ -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 java.util.Map; +import org.apache.kafka.common.KafkaFuture; + +/** + * The result of the {@link Admin#updateFeatures(Map, UpdateFeaturesOptions)} call. + * + * The API of this class is evolving, see {@link Admin} for details. + */ +public class UpdateFeaturesResult { + private final Map> futures; + + /** + * @param futures a map from feature name to future, which can be used to check the status of + * individual feature updates. + */ + UpdateFeaturesResult(final Map> futures) { + this.futures = futures; + } + + public Map> values() { + return futures; + } + + /** + * Return a future which succeeds if all the feature updates succeed. + */ + public KafkaFuture all() { + return KafkaFuture.allOf(futures.values().toArray(new KafkaFuture[0])); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/errors/FeatureUpdateFailedException.java b/clients/src/main/java/org/apache/kafka/common/errors/FeatureUpdateFailedException.java new file mode 100644 index 0000000000000..9f5e23d3104da --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/errors/FeatureUpdateFailedException.java @@ -0,0 +1,29 @@ +/* + * 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 FeatureUpdateFailedException extends ApiException { + private static final long serialVersionUID = 1L; + + public FeatureUpdateFailedException(final String message) { + super(message); + } + + public FeatureUpdateFailedException(final String message, final Throwable cause) { + super(message, cause); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/feature/BaseVersionRange.java b/clients/src/main/java/org/apache/kafka/common/feature/BaseVersionRange.java index 6fdc9b1f82b77..2d6ce702e253f 100644 --- a/clients/src/main/java/org/apache/kafka/common/feature/BaseVersionRange.java +++ b/clients/src/main/java/org/apache/kafka/common/feature/BaseVersionRange.java @@ -16,6 +16,8 @@ */ package org.apache.kafka.common.feature; +import static java.util.stream.Collectors.joining; + import java.util.Map; import java.util.Objects; @@ -85,20 +87,31 @@ public short max() { } public String toString() { - return String.format("%s[%s:%d, %s:%d]", - this.getClass().getSimpleName(), this.minKeyLabel, min(), this.maxKeyLabel, max()); + return String.format( + "%s[%s]", + this.getClass().getSimpleName(), + mapToString(toMap())); } public Map toMap() { return Utils.mkMap(Utils.mkEntry(minKeyLabel, min()), Utils.mkEntry(maxKeyLabel, max())); } + private static String mapToString(final Map map) { + return map + .entrySet() + .stream() + .map(entry -> String.format("%s:%d", entry.getKey(), entry.getValue())) + .collect(joining(", ")); + } + @Override public boolean equals(Object other) { if (this == other) { return true; } - if (!(other instanceof BaseVersionRange)) { + + if (other == null || getClass() != other.getClass()) { return false; } @@ -117,7 +130,7 @@ public int hashCode() { public static short valueOrThrow(String key, Map versionRangeMap) { final Short value = versionRangeMap.get(key); if (value == null) { - throw new IllegalArgumentException(key + " absent in " + versionRangeMap); + throw new IllegalArgumentException(String.format("%s absent in [%s]", key, mapToString(versionRangeMap))); } return value; } diff --git a/clients/src/main/java/org/apache/kafka/common/feature/FinalizedVersionRange.java b/clients/src/main/java/org/apache/kafka/common/feature/FinalizedVersionRange.java index 811712df4fbc4..27e6440478644 100644 --- a/clients/src/main/java/org/apache/kafka/common/feature/FinalizedVersionRange.java +++ b/clients/src/main/java/org/apache/kafka/common/feature/FinalizedVersionRange.java @@ -19,7 +19,7 @@ import java.util.Map; /** - * An extended {@link BaseVersionRange} representing the min/max versions for finalized features. + * An extended {@link BaseVersionRange} representing the min/max versions for a finalized feature. */ public class FinalizedVersionRange extends BaseVersionRange { // Label for the min version key, that's used only to convert to/from a map. @@ -40,7 +40,7 @@ public static FinalizedVersionRange fromMap(Map versionRangeMap) /** * Checks if the [min, max] version level range of this object does *NOT* fall within the - * [min, max] version range of the provided SupportedVersionRange parameter. + * [min, max] range of the provided SupportedVersionRange parameter. * * @param supportedVersionRange the SupportedVersionRange to be checked * diff --git a/clients/src/main/java/org/apache/kafka/common/feature/SupportedVersionRange.java b/clients/src/main/java/org/apache/kafka/common/feature/SupportedVersionRange.java index cc4e8517184b9..8993014a74b2e 100644 --- a/clients/src/main/java/org/apache/kafka/common/feature/SupportedVersionRange.java +++ b/clients/src/main/java/org/apache/kafka/common/feature/SupportedVersionRange.java @@ -19,7 +19,7 @@ import java.util.Map; /** - * An extended {@link BaseVersionRange} representing the min/max versions for supported features. + * An extended {@link BaseVersionRange} representing the min/max versions for a supported feature. */ public class SupportedVersionRange extends BaseVersionRange { // Label for the min version key, that's used only to convert to/from a map. @@ -28,8 +28,12 @@ public class SupportedVersionRange extends BaseVersionRange { // Label for the max version key, that's used only to convert to/from a map. private static final String MAX_VERSION_KEY_LABEL = "max_version"; - public SupportedVersionRange(short minVersionLevel, short maxVersionLevel) { - super(MIN_VERSION_KEY_LABEL, minVersionLevel, MAX_VERSION_KEY_LABEL, maxVersionLevel); + public SupportedVersionRange(short minVersion, short maxVersion) { + super(MIN_VERSION_KEY_LABEL, minVersion, MAX_VERSION_KEY_LABEL, maxVersion); + } + + public SupportedVersionRange(short maxVersion) { + this((short) 1, maxVersion); } public static SupportedVersionRange fromMap(Map versionRangeMap) { diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java index 5d785f56c8ed8..46b22964b523b 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java @@ -121,6 +121,8 @@ import org.apache.kafka.common.message.SyncGroupResponseData; import org.apache.kafka.common.message.TxnOffsetCommitRequestData; import org.apache.kafka.common.message.TxnOffsetCommitResponseData; +import org.apache.kafka.common.message.UpdateFeaturesRequestData; +import org.apache.kafka.common.message.UpdateFeaturesResponseData; import org.apache.kafka.common.message.UpdateMetadataRequestData; import org.apache.kafka.common.message.UpdateMetadataResponseData; import org.apache.kafka.common.message.VoteRequestData; @@ -247,7 +249,9 @@ public Struct parseResponse(short version, ByteBuffer buffer) { EndQuorumEpochRequestData.SCHEMAS, EndQuorumEpochResponseData.SCHEMAS), DESCRIBE_QUORUM(55, "DescribeQuorum", true, false, DescribeQuorumRequestData.SCHEMAS, DescribeQuorumResponseData.SCHEMAS), - ALTER_ISR(56, "AlterIsr", AlterIsrRequestData.SCHEMAS, AlterIsrResponseData.SCHEMAS); + ALTER_ISR(56, "AlterIsr", AlterIsrRequestData.SCHEMAS, AlterIsrResponseData.SCHEMAS), + UPDATE_FEATURES(57, "UpdateFeatures", + UpdateFeaturesRequestData.SCHEMAS, UpdateFeaturesResponseData.SCHEMAS); private static final ApiKeys[] ID_TO_TYPE; private static final int MIN_API_KEY = 0; diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java index 041a1feb98e7a..3c3b8003ec1ad 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java @@ -35,6 +35,7 @@ import org.apache.kafka.common.errors.ElectionNotNeededException; import org.apache.kafka.common.errors.EligibleLeadersNotAvailableException; import org.apache.kafka.common.errors.FencedInstanceIdException; +import org.apache.kafka.common.errors.FeatureUpdateFailedException; import org.apache.kafka.common.errors.FencedLeaderEpochException; import org.apache.kafka.common.errors.InvalidUpdateVersionException; import org.apache.kafka.common.errors.FetchSessionIdNotFoundException; @@ -336,7 +337,8 @@ public enum Errors { UNACCEPTABLE_CREDENTIAL(93, "Requested credential would not meet criteria for acceptability.", UnacceptableCredentialException::new), INCONSISTENT_VOTER_SET(94, "Indicates that the either the sender or recipient of a " + "voter-only request is not one of the expected voters", InconsistentVoterSetException::new), - INVALID_UPDATE_VERSION(95, "The given update version was invalid.", InvalidUpdateVersionException::new); + INVALID_UPDATE_VERSION(95, "The given update version was invalid.", InvalidUpdateVersionException::new), + FEATURE_UPDATE_FAILED(96, "Unable to update finalized features due to an unexpected server error.", FeatureUpdateFailedException::new); private static final Logger log = LoggerFactory.getLogger(Errors.class); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java index 8b7646008972d..e96f748445955 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java @@ -257,6 +257,8 @@ public static AbstractRequest parseRequest(ApiKeys apiKey, short apiVersion, Str return new EndQuorumEpochRequest(struct, apiVersion); case ALTER_ISR: return new AlterIsrRequest(new AlterIsrRequestData(struct, apiVersion), apiVersion); + case UPDATE_FEATURES: + return new UpdateFeaturesRequest(struct, apiVersion); default: throw new AssertionError(String.format("ApiKey %s is not currently handled in `parseRequest`, the " + "code should be updated to do so.", apiKey)); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java index e64716b490b65..b3ded3231ba7e 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java @@ -202,6 +202,8 @@ public static AbstractResponse parseResponse(ApiKeys apiKey, Struct struct, shor return new DescribeQuorumResponse(struct, version); case ALTER_ISR: return new AlterIsrResponse(new AlterIsrResponseData(struct, version)); + case UPDATE_FEATURES: + return new UpdateFeaturesResponse(struct, version); default: throw new AssertionError(String.format("ApiKey %s is not currently handled in `parseResponse`, the " + "code should be updated to do so.", apiKey)); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java index 1734472e590c5..9f10d3347d2b7 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java @@ -43,7 +43,7 @@ */ public class ApiVersionsResponse extends AbstractResponse { - public static final int UNKNOWN_FINALIZED_FEATURES_EPOCH = -1; + public static final long UNKNOWN_FINALIZED_FEATURES_EPOCH = -1L; public static final ApiVersionsResponse DEFAULT_API_VERSIONS_RESPONSE = createApiVersionsResponse( @@ -67,6 +67,10 @@ public ApiVersionsResponse(Struct struct, short version) { this(new ApiVersionsResponseData(struct, version)); } + public ApiVersionsResponseData data() { + return data; + } + @Override protected Struct toStruct(short version) { return this.data.toStruct(version); @@ -140,9 +144,15 @@ public static ApiVersionsResponse apiVersionsResponse( byte maxMagic, Features latestSupportedFeatures, Features finalizedFeatures, - int finalizedFeaturesEpoch) { + long finalizedFeaturesEpoch) { if (maxMagic == RecordBatch.CURRENT_MAGIC_VALUE && throttleTimeMs == DEFAULT_THROTTLE_TIME) { - return DEFAULT_API_VERSIONS_RESPONSE; + return new ApiVersionsResponse(createApiVersionsResponseData( + DEFAULT_API_VERSIONS_RESPONSE.throttleTimeMs(), + Errors.forCode(DEFAULT_API_VERSIONS_RESPONSE.data().errorCode()), + DEFAULT_API_VERSIONS_RESPONSE.data().apiKeys(), + latestSupportedFeatures, + finalizedFeatures, + finalizedFeaturesEpoch)); } return createApiVersionsResponse( throttleTimeMs, maxMagic, latestSupportedFeatures, finalizedFeatures, finalizedFeaturesEpoch); @@ -159,12 +169,12 @@ public static ApiVersionsResponse createApiVersionsResponse( UNKNOWN_FINALIZED_FEATURES_EPOCH); } - public static ApiVersionsResponse createApiVersionsResponse( + private static ApiVersionsResponse createApiVersionsResponse( final int throttleTimeMs, final byte minMagic, final Features latestSupportedFeatures, final Features finalizedFeatures, - final int finalizedFeaturesEpoch + final long finalizedFeaturesEpoch ) { ApiVersionsResponseKeyCollection apiKeys = new ApiVersionsResponseKeyCollection(); for (ApiKeys apiKey : ApiKeys.enabledApis()) { @@ -176,25 +186,44 @@ public static ApiVersionsResponse createApiVersionsResponse( } } - ApiVersionsResponseData data = new ApiVersionsResponseData(); + return new ApiVersionsResponse( + createApiVersionsResponseData( + throttleTimeMs, + Errors.NONE, + apiKeys, + latestSupportedFeatures, + finalizedFeatures, + finalizedFeaturesEpoch)); + } + + public static ApiVersionsResponseData createApiVersionsResponseData( + final int throttleTimeMs, + final Errors error, + final ApiVersionsResponseKeyCollection apiKeys, + final Features latestSupportedFeatures, + final Features finalizedFeatures, + final long finalizedFeaturesEpoch + ) { + final ApiVersionsResponseData data = new ApiVersionsResponseData(); data.setThrottleTimeMs(throttleTimeMs); - data.setErrorCode(Errors.NONE.code()); + data.setErrorCode(error.code()); data.setApiKeys(apiKeys); data.setSupportedFeatures(createSupportedFeatureKeys(latestSupportedFeatures)); data.setFinalizedFeatures(createFinalizedFeatureKeys(finalizedFeatures)); data.setFinalizedFeaturesEpoch(finalizedFeaturesEpoch); - return new ApiVersionsResponse(data); + return data; } private static SupportedFeatureKeyCollection createSupportedFeatureKeys( Features latestSupportedFeatures) { SupportedFeatureKeyCollection converted = new SupportedFeatureKeyCollection(); for (Map.Entry feature : latestSupportedFeatures.features().entrySet()) { - SupportedFeatureKey key = new SupportedFeatureKey(); + final SupportedFeatureKey key = new SupportedFeatureKey(); + final SupportedVersionRange versionRange = feature.getValue(); key.setName(feature.getKey()); - key.setMinVersion(feature.getValue().min()); - key.setMaxVersion(feature.getValue().max()); + key.setMinVersion(versionRange.min()); + key.setMaxVersion(versionRange.max()); converted.add(key); } @@ -205,10 +234,11 @@ private static FinalizedFeatureKeyCollection createFinalizedFeatureKeys( Features finalizedFeatures) { FinalizedFeatureKeyCollection converted = new FinalizedFeatureKeyCollection(); for (Map.Entry feature : finalizedFeatures.features().entrySet()) { - FinalizedFeatureKey key = new FinalizedFeatureKey(); + final FinalizedFeatureKey key = new FinalizedFeatureKey(); + final FinalizedVersionRange versionLevelRange = feature.getValue(); key.setName(feature.getKey()); - key.setMinVersionLevel(feature.getValue().min()); - key.setMaxVersionLevel(feature.getValue().max()); + key.setMinVersionLevel(versionLevelRange.min()); + key.setMaxVersionLevel(versionLevelRange.max()); converted.add(key); } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesRequest.java new file mode 100644 index 0000000000000..3276c0abbe438 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesRequest.java @@ -0,0 +1,95 @@ +/* + * 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 java.nio.ByteBuffer; +import org.apache.kafka.common.message.UpdateFeaturesRequestData.FeatureUpdateKey; +import org.apache.kafka.common.message.UpdateFeaturesResponseData; +import org.apache.kafka.common.message.UpdateFeaturesRequestData; +import org.apache.kafka.common.message.UpdateFeaturesResponseData.UpdatableFeatureResult; +import org.apache.kafka.common.message.UpdateFeaturesResponseData.UpdatableFeatureResultCollection; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.types.Struct; + +public class UpdateFeaturesRequest extends AbstractRequest { + + public static class Builder extends AbstractRequest.Builder { + + private final UpdateFeaturesRequestData data; + + public Builder(UpdateFeaturesRequestData data) { + super(ApiKeys.UPDATE_FEATURES); + this.data = data; + } + + @Override + public UpdateFeaturesRequest build(short version) { + return new UpdateFeaturesRequest(data, version); + } + + @Override + public String toString() { + return data.toString(); + } + } + + private final UpdateFeaturesRequestData data; + + public UpdateFeaturesRequest(UpdateFeaturesRequestData data, short version) { + super(ApiKeys.UPDATE_FEATURES, version); + this.data = data; + } + + public UpdateFeaturesRequest(Struct struct, short version) { + super(ApiKeys.UPDATE_FEATURES, version); + this.data = new UpdateFeaturesRequestData(struct, version); + } + + @Override + public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) { + final ApiError apiError = ApiError.fromThrowable(e); + final UpdatableFeatureResultCollection results = new UpdatableFeatureResultCollection(); + for (FeatureUpdateKey update : this.data.featureUpdates().valuesSet()) { + final UpdatableFeatureResult result = new UpdatableFeatureResult() + .setFeature(update.feature()) + .setErrorCode(apiError.error().code()) + .setErrorMessage(apiError.message()); + results.add(result); + } + final UpdateFeaturesResponseData responseData = new UpdateFeaturesResponseData() + .setThrottleTimeMs(throttleTimeMs) + .setResults(results); + return new UpdateFeaturesResponse(responseData); } + + @Override + protected Struct toStruct() { + return data.toStruct(version()); + } + + public UpdateFeaturesRequestData data() { + return data; + } + + public static UpdateFeaturesRequest parse(ByteBuffer buffer, short version) { + return new UpdateFeaturesRequest( + ApiKeys.UPDATE_FEATURES.parseRequest(version, buffer), version); + } + + public static boolean isDeleteRequest(UpdateFeaturesRequestData.FeatureUpdateKey update) { + return update.maxVersionLevel() < 1 && update.allowDowngrade(); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesResponse.java new file mode 100644 index 0000000000000..5754f13595c8b --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesResponse.java @@ -0,0 +1,109 @@ +/* + * 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 java.nio.ByteBuffer; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.kafka.common.message.UpdateFeaturesResponseData; +import org.apache.kafka.common.message.UpdateFeaturesResponseData.UpdatableFeatureResult; +import org.apache.kafka.common.message.UpdateFeaturesResponseData.UpdatableFeatureResultCollection; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.types.Struct; + + +/** + * Possible error codes: + * + * - {@link Errors#CLUSTER_AUTHORIZATION_FAILED} + * - {@link Errors#NOT_CONTROLLER} + * - {@link Errors#INVALID_REQUEST} + * - {@link Errors#FEATURE_UPDATE_FAILED} + */ +public class UpdateFeaturesResponse extends AbstractResponse { + + private final UpdateFeaturesResponseData data; + + public UpdateFeaturesResponse(UpdateFeaturesResponseData data) { + this.data = data; + } + + public UpdateFeaturesResponse(Struct struct) { + final short latestVersion = (short) (UpdateFeaturesResponseData.SCHEMAS.length - 1); + this.data = new UpdateFeaturesResponseData(struct, latestVersion); + } + + public UpdateFeaturesResponse(Struct struct, short version) { + this.data = new UpdateFeaturesResponseData(struct, version); + } + + public Map errors() { + return data.results().valuesSet().stream().collect( + Collectors.toMap( + result -> result.feature(), + result -> new ApiError(Errors.forCode(result.errorCode()), result.errorMessage()))); + } + + @Override + public Map errorCounts() { + return apiErrorCounts(errors()); + } + + @Override + public int throttleTimeMs() { + return data.throttleTimeMs(); + } + + @Override + protected Struct toStruct(short version) { + return data.toStruct(version); + } + + @Override + public String toString() { + return data.toString(); + } + + public UpdateFeaturesResponseData data() { + return data; + } + + public static UpdateFeaturesResponse parse(ByteBuffer buffer, short version) { + return new UpdateFeaturesResponse(ApiKeys.UPDATE_FEATURES.parseResponse(version, buffer), version); + } + + public static UpdateFeaturesResponse createWithErrors(ApiError topLevelError, Map updateErrors, int throttleTimeMs) { + final UpdatableFeatureResultCollection results = new UpdatableFeatureResultCollection(); + for (final Map.Entry updateError : updateErrors.entrySet()) { + final String feature = updateError.getKey(); + final ApiError error = updateError.getValue(); + final UpdatableFeatureResult result = new UpdatableFeatureResult(); + result.setFeature(feature) + .setErrorCode(error.error().code()) + .setErrorMessage(error.message()); + results.add(result); + } + final UpdateFeaturesResponseData responseData = new UpdateFeaturesResponseData() + .setThrottleTimeMs(throttleTimeMs) + .setErrorCode(topLevelError.error().code()) + .setErrorMessage(topLevelError.message()) + .setResults(results) + .setThrottleTimeMs(throttleTimeMs); + return new UpdateFeaturesResponse(responseData); + } +} diff --git a/clients/src/main/resources/common/message/ApiVersionsResponse.json b/clients/src/main/resources/common/message/ApiVersionsResponse.json index 834564c146e26..ba6f01cb9434a 100644 --- a/clients/src/main/resources/common/message/ApiVersionsResponse.json +++ b/clients/src/main/resources/common/message/ApiVersionsResponse.json @@ -55,8 +55,8 @@ "about": "The maximum supported version for the feature." } ] }, - {"name": "FinalizedFeaturesEpoch", "type": "int32", "versions": "3+", - "tag": 1, "taggedVersions": "3+", "default": "-1", + { "name": "FinalizedFeaturesEpoch", "type": "int64", "versions": "3+", + "tag": 1, "taggedVersions": "3+", "default": "-1", "ignorable": true, "about": "The monotonically increasing epoch for the finalized features information. Valid values are >= 0. A value of -1 is special and represents unknown epoch."}, { "name": "FinalizedFeatures", "type": "[]FinalizedFeatureKey", "versions": "3+", "tag": 2, "taggedVersions": "3+", diff --git a/clients/src/main/resources/common/message/UpdateFeaturesRequest.json b/clients/src/main/resources/common/message/UpdateFeaturesRequest.json new file mode 100644 index 0000000000000..ab882dff1c754 --- /dev/null +++ b/clients/src/main/resources/common/message/UpdateFeaturesRequest.json @@ -0,0 +1,35 @@ +// 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": 57, + "type": "request", + "name": "UpdateFeaturesRequest", + "validVersions": "0", + "flexibleVersions": "0+", + "fields": [ + { "name": "timeoutMs", "type": "int32", "versions": "0+", "default": "60000", + "about": "How long to wait in milliseconds before timing out the request." }, + { "name": "FeatureUpdates", "type": "[]FeatureUpdateKey", "versions": "0+", + "about": "The list of updates to finalized features.", "fields": [ + {"name": "Feature", "type": "string", "versions": "0+", "mapKey": true, + "about": "The name of the finalized feature to be updated."}, + {"name": "MaxVersionLevel", "type": "int16", "versions": "0+", + "about": "The new maximum version level for the finalized feature. A value >= 1 is valid. A value < 1, is special, and can be used to request the deletion of the finalized feature."}, + {"name": "AllowDowngrade", "type": "bool", "versions": "0+", + "about": "When set to true, the finalized feature version level is allowed to be downgraded/deleted. The downgrade request will fail if the new maximum version level is a value that's not lower than the existing maximum finalized version level."} + ]} + ] +} diff --git a/clients/src/main/resources/common/message/UpdateFeaturesResponse.json b/clients/src/main/resources/common/message/UpdateFeaturesResponse.json new file mode 100644 index 0000000000000..615f6177cfbee --- /dev/null +++ b/clients/src/main/resources/common/message/UpdateFeaturesResponse.json @@ -0,0 +1,39 @@ +// 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": 57, + "type": "response", + "name": "UpdateFeaturesResponse", + "validVersions": "0", + "flexibleVersions": "0+", + "fields": [ + { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", + "about": "The 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 top-level error code, or `0` if there was no top-level error." }, + { "name": "ErrorMessage", "type": "string", "versions": "0+", "nullableVersions": "0+", + "about": "The top-level error message, or `null` if there was no top-level error." }, + { "name": "Results", "type": "[]UpdatableFeatureResult", "versions": "0+", + "about": "Results for each feature update.", "fields": [ + {"name": "Feature", "type": "string", "versions": "0+", "mapKey": true, + "about": "The name of the finalized feature."}, + { "name": "ErrorCode", "type": "int16", "versions": "0+", + "about": "The feature update error code or `0` if the feature update succeeded." }, + { "name": "ErrorMessage", "type": "string", "versions": "0+", "nullableVersions": "0+", + "about": "The feature update error, or `null` if the feature update succeeded." } + ]} + ] +} diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java index bf273d6fdb6b1..e91e0d5e59b12 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java @@ -66,11 +66,13 @@ import org.apache.kafka.common.errors.UnknownServerException; import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; import org.apache.kafka.common.errors.UnsupportedVersionException; +import org.apache.kafka.common.feature.Features; import org.apache.kafka.common.message.AlterPartitionReassignmentsResponseData; import org.apache.kafka.common.message.AlterReplicaLogDirsResponseData; import org.apache.kafka.common.message.AlterReplicaLogDirsResponseData.AlterReplicaLogDirPartitionResult; import org.apache.kafka.common.message.AlterReplicaLogDirsResponseData.AlterReplicaLogDirTopicResult; import org.apache.kafka.common.message.AlterUserScramCredentialsResponseData; +import org.apache.kafka.common.message.ApiVersionsResponseData; import org.apache.kafka.common.message.CreatePartitionsResponseData; import org.apache.kafka.common.message.CreatePartitionsResponseData.CreatePartitionsTopicResult; import org.apache.kafka.common.message.CreateAclsResponseData; @@ -122,6 +124,8 @@ import org.apache.kafka.common.requests.AlterReplicaLogDirsResponse; import org.apache.kafka.common.requests.AlterUserScramCredentialsResponse; import org.apache.kafka.common.requests.ApiError; +import org.apache.kafka.common.requests.ApiVersionsRequest; +import org.apache.kafka.common.requests.ApiVersionsResponse; import org.apache.kafka.common.requests.CreateAclsResponse; import org.apache.kafka.common.requests.CreatePartitionsRequest; import org.apache.kafka.common.requests.CreatePartitionsResponse; @@ -152,6 +156,8 @@ import org.apache.kafka.common.requests.OffsetCommitResponse; import org.apache.kafka.common.requests.OffsetDeleteResponse; import org.apache.kafka.common.requests.OffsetFetchResponse; +import org.apache.kafka.common.requests.UpdateFeaturesRequest; +import org.apache.kafka.common.requests.UpdateFeaturesResponse; import org.apache.kafka.common.resource.PatternType; import org.apache.kafka.common.resource.ResourcePattern; import org.apache.kafka.common.resource.ResourcePatternFilter; @@ -474,6 +480,56 @@ private static DescribeGroupsResponseData prepareDescribeGroupsResponseData(Stri Collections.emptySet())); return data; } + + private static FeatureMetadata defaultFeatureMetadata() { + return new FeatureMetadata( + Utils.mkMap(Utils.mkEntry("test_feature_1", new FinalizedVersionRange((short) 2, (short) 3))), + Optional.of(1L), + Utils.mkMap(Utils.mkEntry("test_feature_1", new SupportedVersionRange((short) 1, (short) 5)))); + } + + private static Features convertSupportedFeaturesMap(Map features) { + final Map featuresMap = new HashMap<>(); + for (final Map.Entry entry : features.entrySet()) { + final SupportedVersionRange versionRange = entry.getValue(); + featuresMap.put( + entry.getKey(), + new org.apache.kafka.common.feature.SupportedVersionRange(versionRange.minVersion(), + versionRange.maxVersion())); + } + + return Features.supportedFeatures(featuresMap); + } + + private static Features convertFinalizedFeaturesMap(Map features) { + final Map featuresMap = new HashMap<>(); + for (final Map.Entry entry : features.entrySet()) { + final FinalizedVersionRange versionRange = entry.getValue(); + featuresMap.put( + entry.getKey(), + new org.apache.kafka.common.feature.FinalizedVersionRange( + versionRange.minVersionLevel(), versionRange.maxVersionLevel())); + } + + return Features.finalizedFeatures(featuresMap); + } + + private static ApiVersionsResponse prepareApiVersionsResponseForDescribeFeatures(Errors error) { + if (error == Errors.NONE) { + return new ApiVersionsResponse(ApiVersionsResponse.createApiVersionsResponseData( + ApiVersionsResponse.DEFAULT_API_VERSIONS_RESPONSE.throttleTimeMs(), + error, + ApiVersionsResponse.DEFAULT_API_VERSIONS_RESPONSE.data().apiKeys(), + convertSupportedFeaturesMap(defaultFeatureMetadata().supportedFeatures()), + convertFinalizedFeaturesMap(defaultFeatureMetadata().finalizedFeatures()), + defaultFeatureMetadata().finalizedFeaturesEpoch().get())); + } + return new ApiVersionsResponse( + new ApiVersionsResponseData() + .setThrottleTimeMs(ApiVersionsResponse.DEFAULT_API_VERSIONS_RESPONSE.throttleTimeMs()) + .setErrorCode(error.code())); + } + /** * Test that the client properly times out when we don't receive any metadata. */ @@ -3884,6 +3940,186 @@ public void testListOffsetsNonRetriableErrors() throws Exception { } } + private Map makeTestFeatureUpdates() { + return Utils.mkMap( + Utils.mkEntry("test_feature_1", new FeatureUpdate((short) 2, false)), + Utils.mkEntry("test_feature_2", new FeatureUpdate((short) 3, true))); + } + + private Map makeTestFeatureUpdateErrors(final Map updates, final Errors error) { + final Map errors = new HashMap<>(); + for (Map.Entry entry : updates.entrySet()) { + errors.put(entry.getKey(), new ApiError(error)); + } + return errors; + } + + private void testUpdateFeatures(Map featureUpdates, + ApiError topLevelError, + Map featureUpdateErrors) throws Exception { + try (final AdminClientUnitTestEnv env = mockClientEnv()) { + env.kafkaClient().prepareResponse( + body -> body instanceof UpdateFeaturesRequest, + UpdateFeaturesResponse.createWithErrors(topLevelError, featureUpdateErrors, 0)); + final Map> futures = env.adminClient().updateFeatures( + featureUpdates, + new UpdateFeaturesOptions().timeoutMs(10000)).values(); + for (final Map.Entry> entry : futures.entrySet()) { + final KafkaFuture future = entry.getValue(); + final ApiError error = featureUpdateErrors.get(entry.getKey()); + if (topLevelError.error() == Errors.NONE) { + assertNotNull(error); + if (error.error() == Errors.NONE) { + future.get(); + } else { + final ExecutionException e = assertThrows(ExecutionException.class, + () -> future.get()); + assertEquals(e.getCause().getClass(), error.exception().getClass()); + } + } else { + final ExecutionException e = assertThrows(ExecutionException.class, + () -> future.get()); + assertEquals(e.getCause().getClass(), topLevelError.exception().getClass()); + } + } + } + } + + @Test + public void testUpdateFeaturesDuringSuccess() throws Exception { + final Map updates = makeTestFeatureUpdates(); + testUpdateFeatures(updates, ApiError.NONE, makeTestFeatureUpdateErrors(updates, Errors.NONE)); + } + + @Test + public void testUpdateFeaturesTopLevelError() throws Exception { + final Map updates = makeTestFeatureUpdates(); + testUpdateFeatures(updates, new ApiError(Errors.INVALID_REQUEST), new HashMap<>()); + } + + @Test + public void testUpdateFeaturesInvalidRequestError() throws Exception { + final Map updates = makeTestFeatureUpdates(); + testUpdateFeatures(updates, ApiError.NONE, makeTestFeatureUpdateErrors(updates, Errors.INVALID_REQUEST)); + } + + @Test + public void testUpdateFeaturesUpdateFailedError() throws Exception { + final Map updates = makeTestFeatureUpdates(); + testUpdateFeatures(updates, ApiError.NONE, makeTestFeatureUpdateErrors(updates, Errors.FEATURE_UPDATE_FAILED)); + } + + @Test + public void testUpdateFeaturesPartialSuccess() throws Exception { + final Map errors = makeTestFeatureUpdateErrors(makeTestFeatureUpdates(), Errors.NONE); + errors.put("test_feature_2", new ApiError(Errors.INVALID_REQUEST)); + testUpdateFeatures(makeTestFeatureUpdates(), ApiError.NONE, errors); + } + + @Test + public void testUpdateFeaturesHandleNotControllerException() throws Exception { + try (final AdminClientUnitTestEnv env = mockClientEnv()) { + env.kafkaClient().prepareResponseFrom( + request -> request instanceof UpdateFeaturesRequest, + UpdateFeaturesResponse.createWithErrors( + new ApiError(Errors.NOT_CONTROLLER), + Utils.mkMap(), + 0), + env.cluster().nodeById(0)); + final int controllerId = 1; + env.kafkaClient().prepareResponse(MetadataResponse.prepareResponse(env.cluster().nodes(), + env.cluster().clusterResource().clusterId(), + controllerId, + Collections.emptyList())); + env.kafkaClient().prepareResponseFrom( + request -> request instanceof UpdateFeaturesRequest, + UpdateFeaturesResponse.createWithErrors( + ApiError.NONE, + Utils.mkMap(Utils.mkEntry("test_feature_1", ApiError.NONE), + Utils.mkEntry("test_feature_2", ApiError.NONE)), + 0), + env.cluster().nodeById(controllerId)); + final KafkaFuture future = env.adminClient().updateFeatures( + Utils.mkMap( + Utils.mkEntry("test_feature_1", new FeatureUpdate((short) 2, false)), + Utils.mkEntry("test_feature_2", new FeatureUpdate((short) 3, true))), + new UpdateFeaturesOptions().timeoutMs(10000) + ).all(); + future.get(); + } + } + + @Test + public void testUpdateFeaturesShouldFailRequestForEmptyUpdates() { + try (final AdminClientUnitTestEnv env = mockClientEnv()) { + assertThrows( + IllegalArgumentException.class, + () -> env.adminClient().updateFeatures( + new HashMap<>(), new UpdateFeaturesOptions())); + } + } + + @Test + public void testUpdateFeaturesShouldFailRequestForInvalidFeatureName() { + try (final AdminClientUnitTestEnv env = mockClientEnv()) { + final UpdateFeaturesResult result = env.adminClient().updateFeatures( + Utils.mkMap(Utils.mkEntry("", new FeatureUpdate((short) 2, false))), + new UpdateFeaturesOptions()); + + final Map> futures = result.values(); + for (Map.Entry> entry : futures.entrySet()) { + final Throwable cause = assertThrows(ExecutionException.class, () -> entry.getValue().get()); + assertEquals(KafkaException.class, cause.getCause().getClass()); + } + + final KafkaFuture future = result.all(); + final Throwable cause = assertThrows(ExecutionException.class, () -> future.get()); + assertEquals(KafkaException.class, cause.getCause().getClass()); + } + } + + @Test + public void testUpdateFeaturesShouldFailRequestInClientWhenDowngradeFlagIsNotSetDuringDeletion() { + assertThrows( + IllegalArgumentException.class, + () -> new FeatureUpdate((short) 0, false)); + } + + @Test + public void testDescribeFeaturesSuccess() throws Exception { + try (final AdminClientUnitTestEnv env = mockClientEnv()) { + env.kafkaClient().prepareResponse( + body -> body instanceof ApiVersionsRequest, + prepareApiVersionsResponseForDescribeFeatures(Errors.NONE)); + final KafkaFuture future = env.adminClient().describeFeatures( + new DescribeFeaturesOptions().timeoutMs(10000)).featureMetadata(); + final FeatureMetadata metadata = future.get(); + assertEquals(defaultFeatureMetadata(), metadata); + } + } + + @Test + public void testDescribeFeaturesHandleNotControllerException() throws Exception { + try (final AdminClientUnitTestEnv env = mockClientEnv()) { + env.kafkaClient().prepareResponseFrom( + prepareApiVersionsResponseForDescribeFeatures(Errors.NOT_CONTROLLER), + env.cluster().nodeById(0)); + env.kafkaClient().prepareResponse(MetadataResponse.prepareResponse(env.cluster().nodes(), + env.cluster().clusterResource().clusterId(), + 1, + Collections.emptyList())); + env.kafkaClient().prepareResponseFrom( + prepareApiVersionsResponseForDescribeFeatures(Errors.NONE), + env.cluster().nodeById(1)); + final DescribeFeaturesOptions options = new DescribeFeaturesOptions(); + options.sendRequestToController(true); + options.timeoutMs(10000); + final KafkaFuture future + = env.adminClient().describeFeatures(options).featureMetadata(); + future.get(); + } + } + @Test public void testListOffsetsMetadataRetriableErrors() throws Exception { diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java b/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java index 03a50c8e9ffc8..3e9f605923bb7 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java @@ -836,6 +836,16 @@ public AlterUserScramCredentialsResult alterUserScramCredentials(List featureUpdates, UpdateFeaturesOptions options) { + throw new UnsupportedOperationException("Not implemented yet"); + } + @Override synchronized public void close(Duration timeout) {} diff --git a/core/src/main/scala/kafka/controller/ControllerState.scala b/core/src/main/scala/kafka/controller/ControllerState.scala index 4e3213e68b5ee..f84240536dd85 100644 --- a/core/src/main/scala/kafka/controller/ControllerState.scala +++ b/core/src/main/scala/kafka/controller/ControllerState.scala @@ -110,8 +110,13 @@ object ControllerState { override protected def hasRateAndTimeMetric: Boolean = false } + case object UpdateFeatures extends ControllerState { + def value = 17 + } + val values: Seq[ControllerState] = Seq(Idle, ControllerChange, BrokerChange, TopicChange, TopicDeletion, AlterPartitionReassignment, AutoLeaderBalance, ManualLeaderBalance, ControlledShutdown, IsrChange, LeaderAndIsrResponseReceived, LogDirChange, ControllerShutdown, UncleanLeaderElectionEnable, - TopicUncleanLeaderElectionEnable, ListPartitionReassignment, UpdateMetadataResponseReceived) + TopicUncleanLeaderElectionEnable, ListPartitionReassignment, UpdateMetadataResponseReceived, + UpdateFeatures) } diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 583d5368fcd68..37b5c4d77399c 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -22,23 +22,27 @@ import java.util.concurrent.TimeUnit import kafka.admin.AdminOperationException import kafka.api._ import kafka.common._ -import kafka.controller.KafkaController.{AlterIsrCallback, AlterReassignmentsCallback, ElectLeadersCallback, ListReassignmentsCallback} +import kafka.controller.KafkaController.AlterIsrCallback +import kafka.cluster.Broker +import kafka.controller.KafkaController.{AlterReassignmentsCallback, ElectLeadersCallback, ListReassignmentsCallback, UpdateFeaturesCallback} import kafka.metrics.{KafkaMetricsGroup, KafkaTimer} import kafka.server._ import kafka.utils._ import kafka.utils.Implicits._ import kafka.zk.KafkaZkClient.UpdateLeaderAndIsrResult -import kafka.zk._ +import kafka.zk.{FeatureZNodeStatus, _} import kafka.zookeeper.{StateChangeHandler, ZNodeChangeHandler, ZNodeChildChangeHandler} import org.apache.kafka.common.ElectionType import org.apache.kafka.common.KafkaException import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.errors.{BrokerNotAvailableException, ControllerMovedException, StaleBrokerEpochException} import org.apache.kafka.common.message.{AlterIsrRequestData, AlterIsrResponseData} +import org.apache.kafka.common.feature.{Features, FinalizedVersionRange} +import org.apache.kafka.common.message.UpdateFeaturesRequestData import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.protocol.Errors -import org.apache.kafka.common.requests.{AbstractControlRequest, ApiError, LeaderAndIsrResponse, UpdateMetadataResponse} -import org.apache.kafka.common.utils.Time +import org.apache.kafka.common.requests.{AbstractControlRequest, ApiError, LeaderAndIsrResponse, UpdateFeaturesRequest, UpdateMetadataResponse} +import org.apache.kafka.common.utils.{Time, Utils} import org.apache.zookeeper.KeeperException import org.apache.zookeeper.KeeperException.Code @@ -60,6 +64,7 @@ object KafkaController extends Logging { type ListReassignmentsCallback = Either[Map[TopicPartition, ReplicaAssignment], ApiError] => Unit type AlterReassignmentsCallback = Either[Map[TopicPartition, ApiError], ApiError] => Unit type AlterIsrCallback = Either[Map[TopicPartition, Either[Errors, LeaderAndIsr]], Errors] => Unit + type UpdateFeaturesCallback = Either[ApiError, Map[String, ApiError]] => Unit } class KafkaController(val config: KafkaConfig, @@ -69,6 +74,8 @@ class KafkaController(val config: KafkaConfig, initialBrokerInfo: BrokerInfo, initialBrokerEpoch: Long, tokenManager: DelegationTokenManager, + brokerFeatures: BrokerFeatures, + featureCache: FinalizedFeatureCache, threadNamePrefix: Option[String] = None) extends ControllerEventProcessor with Logging with KafkaMetricsGroup { @@ -219,6 +226,8 @@ class KafkaController(val config: KafkaConfig, * This ensures another controller election will be triggered and there will always be an actively serving controller */ private def onControllerFailover(): Unit = { + maybeSetupFeatureVersioning() + info("Registering handlers") // before reading source of truth from zookeeper, register the listeners to get broker/topic callbacks @@ -272,6 +281,161 @@ class KafkaController(val config: KafkaConfig, } } + private def createFeatureZNode(newNode: FeatureZNode): Int = { + info(s"Creating FeatureZNode at path: ${FeatureZNode.path} with contents: $newNode") + zkClient.createFeatureZNode(newNode) + val (_, newVersion) = zkClient.getDataAndVersion(FeatureZNode.path) + newVersion + } + + private def updateFeatureZNode(updatedNode: FeatureZNode): Int = { + info(s"Updating FeatureZNode at path: ${FeatureZNode.path} with contents: $updatedNode") + zkClient.updateFeatureZNode(updatedNode) + } + + /** + * This method enables the feature versioning system (KIP-584). + * + * Development in Kafka (from a high level) is organized into features. Each feature is tracked by + * a name and a range of version numbers. A feature can be of two types: + * + * 1. Supported feature: + * A supported feature is represented by a name (string) and a range of versions (defined by a + * SupportedVersionRange). It refers to a feature that a particular broker advertises support for. + * Each broker advertises the version ranges of its own supported features in its own + * BrokerIdZNode. The contents of the advertisement are specific to the particular broker and + * do not represent any guarantee of a cluster-wide availability of the feature for any particular + * range of versions. + * + * 2. Finalized feature: + * A finalized feature is represented by a name (string) and a range of version levels (defined + * by a FinalizedVersionRange). Whenever the feature versioning system (KIP-584) is + * enabled, the finalized features are stored in the cluster-wide common FeatureZNode. + * In comparison to a supported feature, the key difference is that a finalized feature exists + * in ZK only when it is guaranteed to be supported by any random broker in the cluster for a + * specified range of version levels. Also, the controller is the only entity modifying the + * information about finalized features. + * + * This method sets up the FeatureZNode with enabled status, which means that the finalized + * features stored in the FeatureZNode are active. The enabled status should be written by the + * controller to the FeatureZNode only when the broker IBP config is greater than or equal to + * KAFKA_2_7_IV0. + * + * There are multiple cases handled here: + * + * 1. New cluster bootstrap: + * A new Kafka cluster (i.e. it is deployed first time) is almost always started with IBP config + * setting greater than or equal to KAFKA_2_7_IV0. We would like to start the cluster with all + * the possible supported features finalized immediately. Assuming this is the case, the + * controller will start up and notice that the FeatureZNode is absent in the new cluster, + * it will then create a FeatureZNode (with enabled status) containing the entire list of + * supported features as its finalized features. + * + * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0: + * Imagine there was an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the + * broker binary has now been upgraded to a newer version that supports the feature versioning + * system (KIP-584). But the IBP config is still set to lower than KAFKA_2_7_IV0, and may be + * set to a higher value later. In this case, we want to start with no finalized features and + * allow the user to finalize them whenever they are ready i.e. in the future whenever the + * user sets IBP config to be greater than or equal to KAFKA_2_7_IV0, then the user could start + * finalizing the features. This process ensures we do not enable all the possible features + * immediately after an upgrade, which could be harmful to Kafka. + * This is how we handle such a case: + * - Before the IBP config upgrade (i.e. IBP config set to less than KAFKA_2_7_IV0), the + * controller will start up and check if the FeatureZNode is absent. + * - If the node is absent, it will react by creating a FeatureZNode with disabled status + * and empty finalized features. + * - Otherwise, if a node already exists in enabled status then the controller will just + * flip the status to disabled and clear the finalized features. + * - After the IBP config upgrade (i.e. IBP config set to greater than or equal to + * KAFKA_2_7_IV0), when the controller starts up it will check if the FeatureZNode exists + * and whether it is disabled. + * - If the node is in disabled status, the controller won’t upgrade all features immediately. + * Instead it will just switch the FeatureZNode status to enabled status. This lets the + * user finalize the features later. + * - Otherwise, if a node already exists in enabled status then the controller will leave + * the node umodified. + * + * 3. Broker binary upgraded, with existing cluster IBP config >= KAFKA_2_7_IV0: + * Imagine there was an existing Kafka cluster with IBP config >= KAFKA_2_7_IV0, and the broker + * binary has just been upgraded to a newer version (that supports IBP config KAFKA_2_7_IV0 and + * higher). The controller will start up and find that a FeatureZNode is already present with + * enabled status and existing finalized features. In such a case, the controller leaves the node + * unmodified. + * + * 4. Broker downgrade: + * Imagine that a Kafka cluster exists already and the IBP config is greater than or equal to + * KAFKA_2_7_IV0. Then, the user decided to downgrade the cluster by setting IBP config to a + * value less than KAFKA_2_7_IV0. This means the user is also disabling the feature versioning + * system (KIP-584). In this case, when the controller starts up with the lower IBP config, it + * will switch the FeatureZNode status to disabled with empty features. + */ + private def enableFeatureVersioning(): Unit = { + val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path) + if (version == ZkVersion.UnknownVersion) { + val newVersion = createFeatureZNode(new FeatureZNode(FeatureZNodeStatus.Enabled, + brokerFeatures.defaultFinalizedFeatures)) + featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs) + } else { + val existingFeatureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get) + val newFeatures = existingFeatureZNode.status match { + case FeatureZNodeStatus.Enabled => existingFeatureZNode.features + case FeatureZNodeStatus.Disabled => + if (!existingFeatureZNode.features.empty()) { + warn(s"FeatureZNode at path: ${FeatureZNode.path} with disabled status" + + s" contains non-empty features: ${existingFeatureZNode.features}") + } + Features.emptyFinalizedFeatures + } + val newFeatureZNode = new FeatureZNode(FeatureZNodeStatus.Enabled, newFeatures) + if (!newFeatureZNode.equals(existingFeatureZNode)) { + val newVersion = updateFeatureZNode(newFeatureZNode) + featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs) + } + } + } + + /** + * Disables the feature versioning system (KIP-584). + * + * Sets up the FeatureZNode with disabled status. This status means the feature versioning system + * (KIP-584) is disabled, and, the finalized features stored in the FeatureZNode are not relevant. + * This status should be written by the controller to the FeatureZNode only when the broker + * IBP config is less than KAFKA_2_7_IV0. + * + * NOTE: + * 1. When this method returns, existing finalized features (if any) will be cleared from the + * FeatureZNode. + * 2. This method, unlike enableFeatureVersioning() need not wait for the FinalizedFeatureCache + * to be updated, because, such updates to the cache (via FinalizedFeatureChangeListener) + * are disabled when IBP config is < than KAFKA_2_7_IV0. + */ + private def disableFeatureVersioning(): Unit = { + val newNode = FeatureZNode(FeatureZNodeStatus.Disabled, Features.emptyFinalizedFeatures()) + val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path) + if (version == ZkVersion.UnknownVersion) { + createFeatureZNode(newNode) + } else { + val existingFeatureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get) + if (existingFeatureZNode.status == FeatureZNodeStatus.Disabled && + !existingFeatureZNode.features.empty()) { + warn(s"FeatureZNode at path: ${FeatureZNode.path} with disabled status" + + s" contains non-empty features: ${existingFeatureZNode.features}") + } + if (!newNode.equals(existingFeatureZNode)) { + updateFeatureZNode(newNode) + } + } + } + + private def maybeSetupFeatureVersioning(): Unit = { + if (config.isFeatureVersioningSupported) { + enableFeatureVersioning() + } else { + disableFeatureVersioning() + } + } + private def scheduleAutoLeaderRebalanceTask(delay: Long, unit: TimeUnit): Unit = { kafkaScheduler.schedule("auto-leader-rebalance-task", () => eventManager.put(AutoPreferredReplicaLeaderElection), delay = delay, unit = unit) @@ -728,7 +892,12 @@ class KafkaController(val config: KafkaConfig, private def initializeControllerContext(): Unit = { // update controller cache with delete topic information val curBrokerAndEpochs = zkClient.getAllBrokerAndEpochsInCluster - controllerContext.setLiveBrokers(curBrokerAndEpochs) + val (compatibleBrokerAndEpochs, incompatibleBrokerAndEpochs) = partitionOnFeatureCompatibility(curBrokerAndEpochs) + if (!incompatibleBrokerAndEpochs.isEmpty) { + warn("Ignoring registration of new brokers due to incompatibilities with finalized features: " + + incompatibleBrokerAndEpochs.map { case (broker, _) => broker.id }.toSeq.sorted.mkString(",")) + } + controllerContext.setLiveBrokers(compatibleBrokerAndEpochs) info(s"Initialized broker epochs cache: ${controllerContext.liveBrokerIdAndEpochs}") controllerContext.setAllTopics(zkClient.getAllTopicsInCluster(true)) registerPartitionModificationsHandlers(controllerContext.allTopics.toSeq) @@ -1344,7 +1513,6 @@ class KafkaController(val config: KafkaConfig, debug(s"Broker $activeControllerId was elected as controller instead of broker ${config.brokerId}", e) else warn("A controller has been elected but just resigned, this will result in another round of election", e) - case t: Throwable => error(s"Error while electing or becoming controller on broker ${config.brokerId}. " + s"Trigger controller movement immediately", t) @@ -1352,6 +1520,30 @@ class KafkaController(val config: KafkaConfig, } } + /** + * Partitions the provided map of brokers and epochs into 2 new maps: + * - The first map contains only those brokers whose features were found to be compatible with + * the existing finalized features. + * - The second map contains only those brokers whose features were found to be incompatible with + * the existing finalized features. + * + * @param brokersAndEpochs the map to be partitioned + * @return two maps: first contains compatible brokers and second contains + * incompatible brokers as explained above + */ + private def partitionOnFeatureCompatibility(brokersAndEpochs: Map[Broker, Long]): (Map[Broker, Long], Map[Broker, Long]) = { + // There can not be any feature incompatibilities when the feature versioning system is disabled + // or when the finalized feature cache is empty. Otherwise, we check if the non-empty contents + // of the cache are compatible with the supported features of each broker. + brokersAndEpochs.partition { + case (broker, _) => + !config.isFeatureVersioningSupported || + !featureCache.get.exists( + latestFinalizedFeatures => + BrokerFeatures.hasIncompatibleFeatures(broker.features, latestFinalizedFeatures.features)) + } + } + private def processBrokerChange(): Unit = { if (!isActive) return val curBrokerAndEpochs = zkClient.getAllBrokerAndEpochsInCluster @@ -1377,14 +1569,27 @@ class KafkaController(val config: KafkaConfig, bouncedBrokerIds.foreach(controllerChannelManager.removeBroker) bouncedBrokerAndEpochs.keySet.foreach(controllerChannelManager.addBroker) deadBrokerIds.foreach(controllerChannelManager.removeBroker) + if (newBrokerIds.nonEmpty) { - controllerContext.addLiveBrokers(newBrokerAndEpochs) + val (newCompatibleBrokerAndEpochs, newIncompatibleBrokerAndEpochs) = + partitionOnFeatureCompatibility(newBrokerAndEpochs) + if (!newIncompatibleBrokerAndEpochs.isEmpty) { + warn("Ignoring registration of new brokers due to incompatibilities with finalized features: " + + newIncompatibleBrokerAndEpochs.map { case (broker, _) => broker.id }.toSeq.sorted.mkString(",")) + } + controllerContext.addLiveBrokers(newCompatibleBrokerAndEpochs) onBrokerStartup(newBrokerIdsSorted) } if (bouncedBrokerIds.nonEmpty) { controllerContext.removeLiveBrokers(bouncedBrokerIds) onBrokerFailure(bouncedBrokerIdsSorted) - controllerContext.addLiveBrokers(bouncedBrokerAndEpochs) + val (bouncedCompatibleBrokerAndEpochs, bouncedIncompatibleBrokerAndEpochs) = + partitionOnFeatureCompatibility(bouncedBrokerAndEpochs) + if (!bouncedIncompatibleBrokerAndEpochs.isEmpty) { + warn("Ignoring registration of bounced brokers due to incompatibilities with finalized features: " + + bouncedIncompatibleBrokerAndEpochs.map { case (broker, _) => broker.id }.toSeq.sorted.mkString(",")) + } + controllerContext.addLiveBrokers(bouncedCompatibleBrokerAndEpochs) onBrokerStartup(bouncedBrokerIdsSorted) } if (deadBrokerIds.nonEmpty) { @@ -1404,7 +1609,7 @@ class KafkaController(val config: KafkaConfig, if (newMetadataOpt.nonEmpty && oldMetadataOpt.nonEmpty) { val oldMetadata = oldMetadataOpt.get val newMetadata = newMetadataOpt.get - if (newMetadata.endPoints != oldMetadata.endPoints) { + if (newMetadata.endPoints != oldMetadata.endPoints || !oldMetadata.features.equals(newMetadata.features)) { info(s"Updated broker metadata: $oldMetadata -> $newMetadata") controllerContext.updateBrokerMetadata(oldMetadata, newMetadata) onBrokerUpdate(brokerId) @@ -1656,6 +1861,204 @@ class KafkaController(val config: KafkaConfig, } } + /** + * Returns the new FinalizedVersionRange for the feature, if there are no feature + * incompatibilities seen with all known brokers for the provided feature update. + * Otherwise returns an ApiError object containing Errors.INVALID_REQUEST. + * + * @param update the feature update to be processed (this can not be meant to delete the feature) + * + * @return the new FinalizedVersionRange or error, as described above. + */ + private def newFinalizedVersionRangeOrIncompatibilityError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[FinalizedVersionRange, ApiError] = { + if (UpdateFeaturesRequest.isDeleteRequest(update)) { + throw new IllegalArgumentException(s"Provided feature update can not be meant to delete the feature: $update") + } + + val supportedVersionRange = brokerFeatures.supportedFeatures.get(update.feature) + if (supportedVersionRange == null) { + Right(new ApiError(Errors.INVALID_REQUEST, + "Could not apply finalized feature update because the provided feature" + + " is not supported.")) + } else { + var newVersionRange: FinalizedVersionRange = null + try { + newVersionRange = new FinalizedVersionRange(supportedVersionRange.min, update.maxVersionLevel) + } catch { + case _: IllegalArgumentException => { + // This exception means the provided maxVersionLevel is invalid. It is handled below + // outside of this catch clause. + } + } + if (newVersionRange == null) { + Right(new ApiError(Errors.INVALID_REQUEST, + "Could not apply finalized feature update because the provided" + + s" maxVersionLevel:${update.maxVersionLevel} is lower than the" + + s" supported minVersion:${supportedVersionRange.min}.")) + } else { + val newFinalizedFeature = + Features.finalizedFeatures(Utils.mkMap(Utils.mkEntry(update.feature, newVersionRange))) + val numIncompatibleBrokers = controllerContext.liveOrShuttingDownBrokers.count(broker => { + BrokerFeatures.hasIncompatibleFeatures(broker.features, newFinalizedFeature) + }) + if (numIncompatibleBrokers == 0) { + Left(newVersionRange) + } else { + Right(new ApiError(Errors.INVALID_REQUEST, + "Could not apply finalized feature update because" + + " brokers were found to have incompatible versions for the feature.")) + } + } + } + } + + /** + * Validates a feature update on an existing FinalizedVersionRange. + * If the validation succeeds, then, the return value contains: + * 1. the new FinalizedVersionRange for the feature, if the feature update was not meant to delete the feature. + * 2. Option.empty, if the feature update was meant to delete the feature. + * + * If the validation fails, then returned value contains a suitable ApiError. + * + * @param update the feature update to be processed. + * @param existingVersionRange the existing FinalizedVersionRange which can be empty when no + * FinalizedVersionRange exists for the associated feature + * + * @return the new FinalizedVersionRange to be updated into ZK or error + * as described above. + */ + private def validateFeatureUpdate(update: UpdateFeaturesRequestData.FeatureUpdateKey, + existingVersionRange: Option[FinalizedVersionRange]): Either[Option[FinalizedVersionRange], ApiError] = { + def newVersionRangeOrError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[Option[FinalizedVersionRange], ApiError] = { + newFinalizedVersionRangeOrIncompatibilityError(update) + .fold(versionRange => Left(Some(versionRange)), error => Right(error)) + } + + if (update.feature.isEmpty) { + // Check that the feature name is not empty. + Right(new ApiError(Errors.INVALID_REQUEST, "Feature name can not be empty.")) + } else { + // We handle deletion requests separately from non-deletion requests. + if (UpdateFeaturesRequest.isDeleteRequest(update)) { + if (existingVersionRange.isEmpty) { + // Disallow deletion of a non-existing finalized feature. + Right(new ApiError(Errors.INVALID_REQUEST, + "Can not delete non-existing finalized feature.")) + } else { + Left(Option.empty) + } + } else if (update.maxVersionLevel() < 1) { + // Disallow deletion of a finalized feature without allowDowngrade flag set. + Right(new ApiError(Errors.INVALID_REQUEST, + s"Can not provide maxVersionLevel: ${update.maxVersionLevel} less" + + s" than 1 without setting the allowDowngrade flag to true in the request.")) + } else { + existingVersionRange.map(existing => + if (update.maxVersionLevel == existing.max) { + // Disallow a case where target maxVersionLevel matches existing maxVersionLevel. + Right(new ApiError(Errors.INVALID_REQUEST, + s"Can not ${if (update.allowDowngrade) "downgrade" else "upgrade"}" + + s" a finalized feature from existing maxVersionLevel:${existing.max}" + + " to the same value.")) + } else if (update.maxVersionLevel < existing.max && !update.allowDowngrade) { + // Disallow downgrade of a finalized feature without the allowDowngrade flag set. + Right(new ApiError(Errors.INVALID_REQUEST, + s"Can not downgrade finalized feature from existing" + + s" maxVersionLevel:${existing.max} to provided" + + s" maxVersionLevel:${update.maxVersionLevel} without setting the" + + " allowDowngrade flag in the request.")) + } else if (update.allowDowngrade && update.maxVersionLevel > existing.max) { + // Disallow a request that sets allowDowngrade flag without specifying a + // maxVersionLevel that's lower than the existing maxVersionLevel. + Right(new ApiError(Errors.INVALID_REQUEST, + s"When the allowDowngrade flag set in the request, the provided" + + s" maxVersionLevel:${update.maxVersionLevel} can not be greater than" + + s" existing maxVersionLevel:${existing.max}.")) + } else if (update.maxVersionLevel < existing.min) { + // Disallow downgrade of a finalized feature below the existing finalized + // minVersionLevel. + Right(new ApiError(Errors.INVALID_REQUEST, + s"Can not downgrade finalized feature to maxVersionLevel:${update.maxVersionLevel}" + + s" because it's lower than the existing minVersionLevel:${existing.min}.")) + } else { + newVersionRangeOrError(update) + } + ).getOrElse(newVersionRangeOrError(update)) + } + } + } + + private def processFeatureUpdates(request: UpdateFeaturesRequest, + callback: UpdateFeaturesCallback): Unit = { + if (isActive) { + processFeatureUpdatesWithActiveController(request, callback) + } else { + callback(Left(new ApiError(Errors.NOT_CONTROLLER))) + } + } + + private def processFeatureUpdatesWithActiveController(request: UpdateFeaturesRequest, + callback: UpdateFeaturesCallback): Unit = { + val updates = request.data.featureUpdates + val existingFeatures = featureCache.get + .map(featuresAndEpoch => featuresAndEpoch.features.features().asScala) + .getOrElse(Map[String, FinalizedVersionRange]()) + // A map with key being feature name and value being FinalizedVersionRange. + // This contains the target features to be eventually written to FeatureZNode. + val targetFeatures = scala.collection.mutable.Map[String, FinalizedVersionRange]() ++ existingFeatures + // A map with key being feature name and value being error encountered when the FeatureUpdate + // was applied. + val errors = scala.collection.mutable.Map[String, ApiError]() + + // Below we process each FeatureUpdate using the following logic: + // - If a FeatureUpdate is found to be valid, then: + // - The corresponding entry in errors map would be updated to contain Errors.NONE. + // - If the FeatureUpdate is an add or update request, then the targetFeatures map is updated + // to contain the new FinalizedVersionRange for the feature. + // - Otherwise if the FeatureUpdate is a delete request, then the feature is removed from the + // targetFeatures map. + // - Otherwise if a FeatureUpdate is found to be invalid, then: + // - The corresponding entry in errors map would be updated with the appropriate ApiError. + // - The entry in targetFeatures map is left untouched. + updates.asScala.iterator.foreach { update => + validateFeatureUpdate(update, existingFeatures.get(update.feature())) match { + case Left(newVersionRangeOrNone) => + newVersionRangeOrNone match { + case Some(newVersionRange) => targetFeatures += (update.feature() -> newVersionRange) + case None => targetFeatures -= update.feature() + } + errors += (update.feature() -> new ApiError(Errors.NONE)) + case Right(featureUpdateFailureReason) => + errors += (update.feature() -> featureUpdateFailureReason) + } + } + + // If the existing and target features are the same, then, we skip the update to the + // FeatureZNode as no changes to the node are required. Otherwise, we replace the contents + // of the FeatureZNode with the new features. This may result in partial or full modification + // of the existing finalized features in ZK. + try { + if (!existingFeatures.equals(targetFeatures)) { + val newNode = new FeatureZNode(FeatureZNodeStatus.Enabled, Features.finalizedFeatures(targetFeatures.asJava)) + val newVersion = updateFeatureZNode(newNode) + featureCache.waitUntilEpochOrThrow(newVersion, request.data().timeoutMs()) + } + } catch { + // For all features that correspond to valid FeatureUpdate (i.e. error is Errors.NONE), + // we set the error as Errors.FEATURE_UPDATE_FAILED since the FeatureZNode update has failed + // for these. For the rest, the existing error is left untouched. + case e: Exception => + warn(s"Processing of feature updates: $request failed due to error: $e") + errors.foreach { case (feature, apiError) => + if (apiError.error() == Errors.NONE) { + errors(feature) = new ApiError(Errors.FEATURE_UPDATE_FAILED) + } + } + } finally { + callback(Right(errors)) + } + } + private def processIsrChangeNotification(): Unit = { def processUpdateNotifications(partitions: Seq[TopicPartition]): Unit = { val liveBrokers: Seq[Int] = controllerContext.liveOrShuttingDownBrokerIds.toSeq @@ -1690,6 +2093,11 @@ class KafkaController(val config: KafkaConfig, eventManager.put(ListPartitionReassignments(partitions, callback)) } + def updateFeatures(request: UpdateFeaturesRequest, + callback: UpdateFeaturesCallback): Unit = { + eventManager.put(UpdateFeatures(request, callback)) + } + def alterPartitionReassignments(partitions: Map[TopicPartition, Option[Seq[Int]]], callback: AlterReassignmentsCallback): Unit = { eventManager.put(ApiPartitionReassignment(partitions, callback)) @@ -1979,6 +2387,8 @@ class KafkaController(val config: KafkaConfig, processZkPartitionReassignment() case ListPartitionReassignments(partitions, callback) => processListPartitionReassignments(partitions, callback) + case UpdateFeatures(request, callback) => + processFeatureUpdates(request, callback) case PartitionReassignmentIsrChange(partition) => processPartitionReassignmentIsrChange(partition) case IsrChangeNotification => @@ -2272,6 +2682,12 @@ case class ListPartitionReassignments(partitionsOpt: Option[Set[TopicPartition]] override def preempt(): Unit = callback(Right(new ApiError(Errors.NOT_CONTROLLER, null))) } +case class UpdateFeatures(request: UpdateFeaturesRequest, + callback: UpdateFeaturesCallback) extends ControllerEvent { + override def state: ControllerState = ControllerState.UpdateFeatures + override def preempt(): Unit = {} +} + // Used only in test cases abstract class MockEvent(val state: ControllerState) extends ControllerEvent { diff --git a/core/src/main/scala/kafka/server/BrokerFeatures.scala b/core/src/main/scala/kafka/server/BrokerFeatures.scala new file mode 100644 index 0000000000000..dd84f9e73e70f --- /dev/null +++ b/core/src/main/scala/kafka/server/BrokerFeatures.scala @@ -0,0 +1,116 @@ +/** + * 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 kafka.utils.Logging +import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange} +import org.apache.kafka.common.feature.Features._ + +import scala.jdk.CollectionConverters._ + +/** + * A class that encapsulates the latest features supported by the Broker and also provides APIs to + * check for incompatibilities between the features supported by the Broker and finalized features. + * This class is immutable in production. It provides few APIs to mutate state only for the purpose + * of testing. + */ +class BrokerFeatures private (@volatile var supportedFeatures: Features[SupportedVersionRange]) { + // For testing only. + def setSupportedFeatures(newFeatures: Features[SupportedVersionRange]): Unit = { + supportedFeatures = newFeatures + } + + /** + * Returns the default finalized features that a new Kafka cluster with IBP config >= KAFKA_2_7_IV0 + * needs to be bootstrapped with. + */ + def defaultFinalizedFeatures: Features[FinalizedVersionRange] = { + Features.finalizedFeatures( + supportedFeatures.features.asScala.map { + case(name, versionRange) => ( + name, new FinalizedVersionRange(versionRange.min, versionRange.max)) + }.asJava) + } + + /** + * Returns the set of feature names found to be incompatible. + * A feature incompatibility is a version mismatch between the latest feature supported by the + * Broker, and a provided finalized feature. This can happen because a provided finalized + * feature: + * 1) Does not exist in the Broker (i.e. it is unknown to the Broker). + * [OR] + * 2) Exists but the FinalizedVersionRange does not match with the SupportedVersionRange + * of the supported feature. + * + * @param finalized The finalized features against which incompatibilities need to be checked for. + * + * @return The subset of input features which are incompatible. If the returned object + * is empty, it means there were no feature incompatibilities found. + */ + def incompatibleFeatures(finalized: Features[FinalizedVersionRange]): Features[FinalizedVersionRange] = { + BrokerFeatures.incompatibleFeatures(supportedFeatures, finalized, logIncompatibilities = true) + } +} + +object BrokerFeatures extends Logging { + + def createDefault(): BrokerFeatures = { + // The arguments are currently empty, but, in the future as we define features we should + // populate the required values here. + new BrokerFeatures(emptySupportedFeatures) + } + + /** + * Returns true if any of the provided finalized features are incompatible with the provided + * supported features. + * + * @param supportedFeatures The supported features to be compared + * @param finalizedFeatures The finalized features to be compared + * + * @return - True if there are any feature incompatibilities found. + * - False otherwise. + */ + def hasIncompatibleFeatures(supportedFeatures: Features[SupportedVersionRange], + finalizedFeatures: Features[FinalizedVersionRange]): Boolean = { + !incompatibleFeatures(supportedFeatures, finalizedFeatures, logIncompatibilities = false).empty + } + + private def incompatibleFeatures(supportedFeatures: Features[SupportedVersionRange], + finalizedFeatures: Features[FinalizedVersionRange], + logIncompatibilities: Boolean): Features[FinalizedVersionRange] = { + val incompatibleFeaturesInfo = finalizedFeatures.features.asScala.map { + case (feature, versionLevels) => + val supportedVersions = supportedFeatures.get(feature) + if (supportedVersions == null) { + (feature, versionLevels, "{feature=%s, reason='Unsupported feature'}".format(feature)) + } else if (versionLevels.isIncompatibleWith(supportedVersions)) { + (feature, versionLevels, "{feature=%s, reason='%s is incompatible with %s'}".format( + feature, versionLevels, supportedVersions)) + } else { + (feature, versionLevels, null) + } + }.filter{ case(_, _, errorReason) => errorReason != null}.toList + + if (logIncompatibilities && incompatibleFeaturesInfo.nonEmpty) { + warn("Feature incompatibilities seen: " + + incompatibleFeaturesInfo.map { case(_, _, errorReason) => errorReason }.mkString(", ")) + } + Features.finalizedFeatures( + incompatibleFeaturesInfo.map { case(feature, versionLevels, _) => (feature, versionLevels) }.toMap.asJava) + } +} diff --git a/core/src/main/scala/kafka/server/FinalizedFeatureCache.scala b/core/src/main/scala/kafka/server/FinalizedFeatureCache.scala index 6ff7e4c01b690..898657d732e4e 100644 --- a/core/src/main/scala/kafka/server/FinalizedFeatureCache.scala +++ b/core/src/main/scala/kafka/server/FinalizedFeatureCache.scala @@ -20,14 +20,17 @@ package kafka.server import kafka.utils.Logging import org.apache.kafka.common.feature.{Features, FinalizedVersionRange} +import scala.concurrent.TimeoutException +import scala.math.max + // Raised whenever there was an error in updating the FinalizedFeatureCache with features. class FeatureCacheUpdateException(message: String) extends RuntimeException(message) { } // Helper class that represents finalized features along with an epoch value. -case class FinalizedFeaturesAndEpoch(features: Features[FinalizedVersionRange], epoch: Int) { +case class FinalizedFeaturesAndEpoch(features: Features[FinalizedVersionRange], epoch: Long) { override def toString(): String = { - "FinalizedFeaturesAndEpoch(features=%s, epoch=%s)".format(features, epoch) + "FinalizedFeaturesAndEpoch(features=%s, epoch=%d)".format(features, epoch) } } @@ -35,11 +38,13 @@ case class FinalizedFeaturesAndEpoch(features: Features[FinalizedVersionRange], * A common mutable cache containing the latest finalized features and epoch. By default the contents of * the cache are empty. This cache needs to be populated at least once for its contents to become * non-empty. Currently the main reader of this cache is the read path that serves an ApiVersionsRequest, - * returning the features information in the response. + * returning the features information in the response. This cache is typically updated asynchronously + * whenever the finalized features and epoch values are modified in ZK by the KafkaController. + * This cache is thread-safe for reads and writes. * * @see FinalizedFeatureChangeListener */ -object FinalizedFeatureCache extends Logging { +class FinalizedFeatureCache(private val brokerFeatures: BrokerFeatures) extends Logging { @volatile private var featuresAndEpoch: Option[FinalizedFeaturesAndEpoch] = Option.empty /** @@ -53,11 +58,34 @@ object FinalizedFeatureCache extends Logging { featuresAndEpoch.isEmpty } + /** + * Waits no more than timeoutMs for the cache's epoch to reach an epoch >= minExpectedEpoch. + * + * @param minExpectedEpoch the minimum expected epoch to be reached by the cache + * (should be >= 0) + * @param timeoutMs the timeout (in milli seconds) + * + * @throws TimeoutException if the cache's epoch has not reached at least + * minExpectedEpoch within timeoutMs. + */ + def waitUntilEpochOrThrow(minExpectedEpoch: Long, timeoutMs: Long): Unit = { + if(minExpectedEpoch < 0L) { + throw new IllegalArgumentException( + s"Expected minExpectedEpoch >= 0, but $minExpectedEpoch was provided.") + } + waitUntilConditionOrThrow( + () => featuresAndEpoch.isDefined && featuresAndEpoch.get.epoch >= minExpectedEpoch, + timeoutMs) + } + /** * Clears all existing finalized features and epoch from the cache. */ def clear(): Unit = { - featuresAndEpoch = Option.empty + synchronized { + featuresAndEpoch = Option.empty + notifyAll() + } info("Cleared cache") } @@ -74,26 +102,61 @@ object FinalizedFeatureCache extends Logging { * supported features. In such a case, the existing cache contents are * not modified. */ - def updateOrThrow(latestFeatures: Features[FinalizedVersionRange], latestEpoch: Int): Unit = { + def updateOrThrow(latestFeatures: Features[FinalizedVersionRange], latestEpoch: Long): Unit = { val latest = FinalizedFeaturesAndEpoch(latestFeatures, latestEpoch) - val oldFeatureAndEpoch = featuresAndEpoch.map(item => item.toString()).getOrElse("") + val existing = featuresAndEpoch.map(item => item.toString()).getOrElse("") if (featuresAndEpoch.isDefined && featuresAndEpoch.get.epoch > latest.epoch) { - val errorMsg = ("FinalizedFeatureCache update failed due to invalid epoch in new finalized %s." + - " The existing cache contents are %s").format(latest, oldFeatureAndEpoch) + val errorMsg = ("FinalizedFeatureCache update failed due to invalid epoch in new %s." + + " The existing cache contents are %s.").format(latest, existing) throw new FeatureCacheUpdateException(errorMsg) } else { - val incompatibleFeatures = SupportedFeatures.incompatibleFeatures(latest.features) + val incompatibleFeatures = brokerFeatures.incompatibleFeatures(latest.features) if (!incompatibleFeatures.empty) { val errorMsg = ("FinalizedFeatureCache update failed since feature compatibility" + " checks failed! Supported %s has incompatibilities with the latest %s." - ).format(SupportedFeatures.get, latest) + ).format(brokerFeatures.supportedFeatures, latest) throw new FeatureCacheUpdateException(errorMsg) } else { - val logMsg = "Updated cache from existing finalized %s to latest finalized %s".format( - oldFeatureAndEpoch, latest) - featuresAndEpoch = Some(latest) + val logMsg = "Updated cache from existing %s to latest %s.".format(existing, latest) + synchronized { + featuresAndEpoch = Some(latest) + notifyAll() + } info(logMsg) } } } + + /** + * Causes the current thread to wait no more than timeoutMs for the specified condition to be met. + * It is guaranteed that the provided condition will always be invoked only from within a + * synchronized block. + * + * @param waitCondition the condition to be waited upon: + * - if the condition returns true, then, the wait will stop. + * - if the condition returns false, it means the wait must continue until + * timeout. + * + * @param timeoutMs the timeout (in milli seconds) + * + * @throws TimeoutException if the condition is not met within timeoutMs. + */ + private def waitUntilConditionOrThrow(waitCondition: () => Boolean, timeoutMs: Long): Unit = { + if(timeoutMs < 0L) { + throw new IllegalArgumentException(s"Expected timeoutMs >= 0, but $timeoutMs was provided.") + } + val waitEndTimeNanos = System.nanoTime() + (timeoutMs * 1000000) + synchronized { + while (!waitCondition()) { + val nowNanos = System.nanoTime() + if (nowNanos > waitEndTimeNanos) { + throw new TimeoutException( + s"Timed out after waiting for ${timeoutMs}ms for required condition to be met." + + s" Current epoch: ${featuresAndEpoch.map(fe => fe.epoch).getOrElse("")}.") + } + val sleepTimeMs = max(1L, (waitEndTimeNanos - nowNanos) / 1000000) + wait(sleepTimeMs) + } + } + } } diff --git a/core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala b/core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala index 91f7e0dadd547..40bce240d2a32 100644 --- a/core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala +++ b/core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala @@ -32,9 +32,11 @@ import scala.concurrent.TimeoutException * to the latest features read from ZK. The cache updates are serialized through a single * notification processor thread. * - * @param zkClient the Zookeeper client + * @param finalizedFeatureCache the finalized feature cache + * @param zkClient the Zookeeper client */ -class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging { +class FinalizedFeatureChangeListener(private val finalizedFeatureCache: FinalizedFeatureCache, + private val zkClient: KafkaZkClient) extends Logging { /** * Helper class used to update the FinalizedFeatureCache. @@ -85,7 +87,7 @@ class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging { // a case. if (version == ZkVersion.UnknownVersion) { info(s"Feature ZK node at path: $featureZkNodePath does not exist") - FinalizedFeatureCache.clear() + finalizedFeatureCache.clear() } else { var maybeFeatureZNode: Option[FeatureZNode] = Option.empty try { @@ -93,17 +95,17 @@ class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging { } catch { case e: IllegalArgumentException => { error(s"Unable to deserialize feature ZK node at path: $featureZkNodePath", e) - FinalizedFeatureCache.clear() + finalizedFeatureCache.clear() } } - maybeFeatureZNode.map(featureZNode => { + maybeFeatureZNode.foreach(featureZNode => { featureZNode.status match { case FeatureZNodeStatus.Disabled => { info(s"Feature ZK node at path: $featureZkNodePath is in disabled status.") - FinalizedFeatureCache.clear() + finalizedFeatureCache.clear() } case FeatureZNodeStatus.Enabled => { - FinalizedFeatureCache.updateOrThrow(featureZNode.features, version) + finalizedFeatureCache.updateOrThrow(featureZNode.features, version) } case _ => throw new IllegalStateException(s"Unexpected FeatureZNodeStatus found in $featureZNode") } diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 5e10d7df3dbba..5077c3209f948 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -112,8 +112,9 @@ class KafkaApis(val requestChannel: RequestChannel, brokerTopicStats: BrokerTopicStats, val clusterId: String, time: Time, - val tokenManager: DelegationTokenManager) - extends ApiRequestHandler with Logging { + val tokenManager: DelegationTokenManager, + val brokerFeatures: BrokerFeatures, + val finalizedFeatureCache: FinalizedFeatureCache) extends ApiRequestHandler with Logging { type FetchResponseStats = Map[TopicPartition, RecordConversionStats] this.logIdent = "[KafkaApi-%d] ".format(brokerId) @@ -186,7 +187,7 @@ class KafkaApis(val requestChannel: RequestChannel, case ApiKeys.DESCRIBE_USER_SCRAM_CREDENTIALS => handleDescribeUserScramCredentialsRequest(request) case ApiKeys.ALTER_USER_SCRAM_CREDENTIALS => handleAlterUserScramCredentialsRequest(request) case ApiKeys.ALTER_ISR => handleAlterIsrRequest(request) - + case ApiKeys.UPDATE_FEATURES => handleUpdateFeatures(request) // Until we are ready to integrate the Raft layer, these APIs are treated as // unexpected and we just close the connection. case ApiKeys.VOTE => closeConnection(request, util.Collections.emptyMap()) @@ -1733,20 +1734,19 @@ class KafkaApis(val requestChannel: RequestChannel, else if (!apiVersionRequest.isValid) apiVersionRequest.getErrorResponse(requestThrottleMs, Errors.INVALID_REQUEST.exception) else { - val supportedFeatures = SupportedFeatures.get - val finalizedFeatures = FinalizedFeatureCache.get - if (finalizedFeatures.isEmpty) { - ApiVersionsResponse.apiVersionsResponse( + val supportedFeatures = brokerFeatures.supportedFeatures + val finalizedFeaturesOpt = finalizedFeatureCache.get + finalizedFeaturesOpt match { + case Some(finalizedFeatures) => ApiVersionsResponse.apiVersionsResponse( requestThrottleMs, config.interBrokerProtocolVersion.recordVersion.value, - supportedFeatures) - } else { - ApiVersionsResponse.apiVersionsResponse( + supportedFeatures, + finalizedFeatures.features, + finalizedFeatures.epoch) + case None => ApiVersionsResponse.apiVersionsResponse( requestThrottleMs, config.interBrokerProtocolVersion.recordVersion.value, - supportedFeatures, - finalizedFeatures.get.features, - finalizedFeatures.get.epoch) + supportedFeatures) } } } @@ -3109,6 +3109,36 @@ class KafkaApis(val requestChannel: RequestChannel, } } + def handleUpdateFeatures(request: RequestChannel.Request): Unit = { + val updateFeaturesRequest = request.body[UpdateFeaturesRequest] + + def sendResponseCallback(errors: Either[ApiError, Map[String, ApiError]]): Unit = { + def createResponse(throttleTimeMs: Int): UpdateFeaturesResponse = { + errors match { + case Left(topLevelError) => + UpdateFeaturesResponse.createWithErrors( + topLevelError, + Collections.emptyMap(), + throttleTimeMs) + case Right(featureUpdateErrors) => + UpdateFeaturesResponse.createWithErrors( + ApiError.NONE, + featureUpdateErrors.asJava, + throttleTimeMs) + } + } + sendResponseMaybeThrottle(request, requestThrottleMs => createResponse(requestThrottleMs)) + } + + if (!authorize(request.context, ALTER, CLUSTER, CLUSTER_NAME)) { + sendResponseCallback(Left(new ApiError(Errors.CLUSTER_AUTHORIZATION_FAILED))) + } else if (!config.isFeatureVersioningSupported) { + sendResponseCallback(Left(new ApiError(Errors.INVALID_REQUEST, "Feature versioning system is disabled."))) + } else { + controller.updateFeatures(updateFeaturesRequest, sendResponseCallback) + } + } + // private package for testing private[server] def authorize(requestContext: RequestContext, operation: AclOperation, diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 0e31d6e55d2a7..9b6535578945c 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -1554,7 +1554,7 @@ class KafkaConfig(val props: java.util.Map[_, _], doLog: Boolean, dynamicConfigO val controlledShutdownEnable = getBoolean(KafkaConfig.ControlledShutdownEnableProp) /** ********* Feature configuration ***********/ - def isFeatureVersioningEnabled = interBrokerProtocolVersion >= KAFKA_2_7_IV0 + def isFeatureVersioningSupported = interBrokerProtocolVersion >= KAFKA_2_7_IV0 /** ********* Group coordinator configuration ***********/ val groupMinSessionTimeoutMs = getInt(KafkaConfig.GroupMinSessionTimeoutMsProp) diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 8defa4204c800..0f2cf3fad7155 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -187,6 +187,10 @@ class KafkaServer(val config: KafkaConfig, time: Time = Time.SYSTEM, threadNameP private var _featureChangeListener: FinalizedFeatureChangeListener = null + val brokerFeatures: BrokerFeatures = BrokerFeatures.createDefault() + + val featureCache: FinalizedFeatureCache = new FinalizedFeatureCache(brokerFeatures) + def clusterId: String = _clusterId // Visible for testing @@ -229,8 +233,8 @@ class KafkaServer(val config: KafkaConfig, time: Time = Time.SYSTEM, threadNameP initZkClient(time) /* initialize features */ - _featureChangeListener = new FinalizedFeatureChangeListener(_zkClient) - if (config.isFeatureVersioningEnabled) { + _featureChangeListener = new FinalizedFeatureChangeListener(featureCache, _zkClient) + if (config.isFeatureVersioningSupported) { _featureChangeListener.initOrThrow(config.zkConnectionTimeoutMs) } @@ -315,7 +319,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = Time.SYSTEM, threadNameP tokenManager.startup() /* start kafka controller */ - kafkaController = new KafkaController(config, zkClient, time, metrics, brokerInfo, brokerEpoch, tokenManager, threadNamePrefix) + kafkaController = new KafkaController(config, zkClient, time, metrics, brokerInfo, brokerEpoch, tokenManager, brokerFeatures, featureCache, threadNamePrefix) kafkaController.startup() adminManager = new AdminManager(config, metrics, metadataCache, zkClient) @@ -351,7 +355,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = Time.SYSTEM, threadNameP /* start processing requests */ dataPlaneRequestProcessor = new KafkaApis(socketServer.dataPlaneRequestChannel, replicaManager, adminManager, groupCoordinator, transactionCoordinator, kafkaController, zkClient, config.brokerId, config, metadataCache, metrics, authorizer, quotaManagers, - fetchManager, brokerTopicStats, clusterId, time, tokenManager) + fetchManager, brokerTopicStats, clusterId, time, tokenManager, brokerFeatures, featureCache) dataPlaneRequestHandlerPool = new KafkaRequestHandlerPool(config.brokerId, socketServer.dataPlaneRequestChannel, dataPlaneRequestProcessor, time, config.numIoThreads, s"${SocketServer.DataPlaneMetricPrefix}RequestHandlerAvgIdlePercent", SocketServer.DataPlaneThreadPrefix) @@ -359,7 +363,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = Time.SYSTEM, threadNameP socketServer.controlPlaneRequestChannelOpt.foreach { controlPlaneRequestChannel => controlPlaneRequestProcessor = new KafkaApis(controlPlaneRequestChannel, replicaManager, adminManager, groupCoordinator, transactionCoordinator, kafkaController, zkClient, config.brokerId, config, metadataCache, metrics, authorizer, quotaManagers, - fetchManager, brokerTopicStats, clusterId, time, tokenManager) + fetchManager, brokerTopicStats, clusterId, time, tokenManager, brokerFeatures, featureCache) controlPlaneRequestHandlerPool = new KafkaRequestHandlerPool(config.brokerId, socketServer.controlPlaneRequestChannelOpt.get, controlPlaneRequestProcessor, time, 1, s"${SocketServer.ControlPlaneMetricPrefix}RequestHandlerAvgIdlePercent", SocketServer.ControlPlaneThreadPrefix) @@ -467,7 +471,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = Time.SYSTEM, threadNameP zkClient.getClusterId.getOrElse(zkClient.createOrGetClusterId(CoreUtils.generateUuidAsBase64())) } - private[server] def createBrokerInfo: BrokerInfo = { + def createBrokerInfo: BrokerInfo = { val endPoints = config.advertisedListeners.map(e => s"${e.host}:${e.port}") zkClient.getAllBrokersInCluster.filter(_.id != config.brokerId).foreach { broker => val commonEndPoints = broker.endPoints.map(e => s"${e.host}:${e.port}").intersect(endPoints) @@ -491,7 +495,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = Time.SYSTEM, threadNameP val jmxPort = System.getProperty("com.sun.management.jmxremote.port", "-1").toInt BrokerInfo( - Broker(config.brokerId, updatedEndpoints, config.rack, SupportedFeatures.get), + Broker(config.brokerId, updatedEndpoints, config.rack, brokerFeatures.supportedFeatures), config.interBrokerProtocolVersion, jmxPort) } diff --git a/core/src/main/scala/kafka/server/SupportedFeatures.scala b/core/src/main/scala/kafka/server/SupportedFeatures.scala deleted file mode 100644 index 089b8989f0149..0000000000000 --- a/core/src/main/scala/kafka/server/SupportedFeatures.scala +++ /dev/null @@ -1,93 +0,0 @@ -/** - * 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 kafka.utils.Logging -import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange} -import org.apache.kafka.common.feature.Features._ - -import scala.jdk.CollectionConverters._ - -/** - * A common immutable object used in the Broker to define the latest features supported by the - * Broker. Also provides API to check for incompatibilities between the latest features supported - * by the Broker and cluster-wide finalized features. - * - * NOTE: the update() and clear() APIs of this class should be used only for testing purposes. - */ -object SupportedFeatures extends Logging { - - /** - * This is the latest features supported by the Broker. - * This is currently empty, but in the future as we define supported features, this map should be - * populated. - */ - @volatile private var supportedFeatures = emptySupportedFeatures - - /** - * Returns a reference to the latest features supported by the Broker. - */ - def get: Features[SupportedVersionRange] = { - supportedFeatures - } - - // For testing only. - def update(newFeatures: Features[SupportedVersionRange]): Unit = { - supportedFeatures = newFeatures - } - - // For testing only. - def clear(): Unit = { - supportedFeatures = emptySupportedFeatures - } - - /** - * Returns the set of feature names found to be 'incompatible'. - * A feature incompatibility is a version mismatch between the latest feature supported by the - * Broker, and the provided finalized feature. This can happen because a provided finalized - * feature: - * 1) Does not exist in the Broker (i.e. it is unknown to the Broker). - * [OR] - * 2) Exists but the FinalizedVersionRange does not match with the supported feature's SupportedVersionRange. - * - * @param finalized The finalized features against which incompatibilities need to be checked for. - * - * @return The subset of input features which are incompatible. If the returned object - * is empty, it means there were no feature incompatibilities found. - */ - def incompatibleFeatures(finalized: Features[FinalizedVersionRange]): Features[FinalizedVersionRange] = { - val incompatibilities = finalized.features.asScala.map { - case (feature, versionLevels) => { - val supportedVersions = supportedFeatures.get(feature) - if (supportedVersions == null) { - (feature, versionLevels, "{feature=%s, reason='Unsupported feature'}".format(feature)) - } else if (versionLevels.isIncompatibleWith(supportedVersions)) { - (feature, versionLevels, "{feature=%s, reason='%s is incompatible with %s'}".format( - feature, versionLevels, supportedVersions)) - } else { - (feature, versionLevels, null) - } - } - }.filter{ case(_, _, errorReason) => errorReason != null}.toList - - if (incompatibilities.nonEmpty) { - warn("Feature incompatibilities seen: " + incompatibilities.map{ case(_, _, errorReason) => errorReason }) - } - Features.finalizedFeatures(incompatibilities.map{ case(feature, versionLevels, _) => (feature, versionLevels) }.toMap.asJava) - } -} diff --git a/core/src/main/scala/kafka/zk/KafkaZkClient.scala b/core/src/main/scala/kafka/zk/KafkaZkClient.scala index 189568ef0e9cd..2f88f4cb903a8 100644 --- a/core/src/main/scala/kafka/zk/KafkaZkClient.scala +++ b/core/src/main/scala/kafka/zk/KafkaZkClient.scala @@ -1570,13 +1570,14 @@ class KafkaZkClient private[zk] (zooKeeperClient: ZooKeeperClient, isSecure: Boo response.maybeThrow() } - def updateFeatureZNode(nodeContents: FeatureZNode): Unit = { + def updateFeatureZNode(nodeContents: FeatureZNode): Int = { val setRequest = SetDataRequest( FeatureZNode.path, FeatureZNode.encode(nodeContents), ZkVersion.MatchAnyVersion) val response = retryRequestUntilConnected(setRequest) response.maybeThrow() + response.stat.getVersion } def deleteFeatureZNode(): Unit = { diff --git a/core/src/main/scala/kafka/zk/ZkData.scala b/core/src/main/scala/kafka/zk/ZkData.scala index 3ec17b34a0813..67fbef39321f6 100644 --- a/core/src/main/scala/kafka/zk/ZkData.scala +++ b/core/src/main/scala/kafka/zk/ZkData.scala @@ -233,7 +233,7 @@ object BrokerIdZNode { * "timestamp":"2233345666", * "endpoints":["CLIENT://host1:9092", "REPLICATION://host1:9093"], * "rack":"dc1", - * "features": {"feature": {"min_version": 1, "max_version": 5}} + * "features": {"feature": {"min_version":1, "first_active_version":2, "max_version":3}} * } */ def decode(id: Int, jsonBytes: Array[Byte]): BrokerInfo = { @@ -797,44 +797,9 @@ object DelegationTokenInfoZNode { * the finalized features stored in the FeatureZNode is not relevant. This status is * written by the controller to the FeatureZNode only when the broker IBP config * is less than KAFKA_2_7_IV0. - * - * The purpose behind the FeatureZNodeStatus is that it helps differentiates between the following - * cases: - * - * 1. New cluster bootstrap: - * For a new Kafka cluster (i.e. it is deployed first time), we would like to start the cluster - * with all the possible supported features finalized immediately. The new cluster will almost - * never be started with an old IBP config that’s less than KAFKA_2_7_IV0. In such a case, the - * controller will start up and notice that the FeatureZNode is absent in the new cluster. - * To handle the requirement, the controller will create a FeatureZNode (with enabled status) - * containing the entire list of supported features as its finalized features. - * - * 2. Cluster upgrade: - * Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, but - * the Broker binary has been upgraded to a state where it supports the feature versioning - * system (KIP-584). This means the user is upgrading from an earlier version of the Broker - * binary. In this case, we want to start with no finalized features and allow the user to enable - * them whenever they are ready i.e. in the future whenever the user sets IBP config - * to be greater than or equal to KAFKA_2_7_IV0. The reason is that enabling all the possible - * features immediately after an upgrade could be harmful to the cluster. - * In such a case: - * - Before the Broker upgrade (i.e. IBP config set to less than KAFKA_2_7_IV0), the controller - * will start up and check if the FeatureZNode is absent. If true, then it will react by - * creating a FeatureZNode with disabled status and empty features. - * - After the Broker upgrade (i.e. IBP config set to greater than or equal to KAFKA_2_7_IV0), - * when the controller starts up it will check if the FeatureZNode exists and whether it is - * disabled. In such a case, it won’t upgrade all features immediately. Instead it will just - * switch the FeatureZNode status to enabled status. This lets the user finalize the features - * later. - * - * 3. Cluster downgrade: - * Imagine that a Kafka cluster exists already and the IBP config is greater than or equal to - * KAFKA_2_7_IV0. Then, the user decided to downgrade the cluster by setting IBP config to a - * value less than KAFKA_2_7_IV0. This means the user is also disabling the feature versioning - * system (KIP-584). In this case, when the controller starts up with the lower IBP config, it - * will switch the FeatureZNode status to disabled with empty features. */ object FeatureZNodeStatus extends Enumeration { + type FeatureZNodeStatus = Value val Disabled, Enabled = Value def withNameOpt(value: Int): Option[Value] = { @@ -848,7 +813,7 @@ object FeatureZNodeStatus extends Enumeration { * @param status the status of the ZK node * @param features the cluster-wide finalized features */ -case class FeatureZNode(status: FeatureZNodeStatus.Value, features: Features[FinalizedVersionRange]) { +case class FeatureZNode(status: FeatureZNodeStatus.FeatureZNodeStatus, features: Features[FinalizedVersionRange]) { } object FeatureZNode { diff --git a/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala b/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala index 62e8d42ff2558..47c271fca0b29 100644 --- a/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala @@ -21,12 +21,13 @@ import java.util.Properties import java.util.concurrent.{CountDownLatch, LinkedBlockingQueue} import com.yammer.metrics.core.Timer -import kafka.api.LeaderAndIsr +import kafka.api.{ApiVersion, KAFKA_2_6_IV0, KAFKA_2_7_IV0, LeaderAndIsr} import kafka.metrics.KafkaYammerMetrics import kafka.server.{KafkaConfig, KafkaServer} import kafka.utils.{LogCaptureAppender, TestUtils} -import kafka.zk._ +import kafka.zk.{FeatureZNodeStatus, _} import org.apache.kafka.common.errors.{ControllerMovedException, StaleBrokerEpochException} +import org.apache.kafka.common.feature.Features import org.apache.kafka.common.metrics.KafkaMetric import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.{ElectionType, TopicPartition} @@ -595,6 +596,36 @@ class ControllerIntegrationTest extends ZooKeeperTestHarness { testControllerMove(() => zkClient.createPartitionReassignment(reassignment)) } + @Test + def testControllerFeatureZNodeSetupWhenFeatureVersioningIsEnabledWithNonExistingFeatureZNode(): Unit = { + testControllerFeatureZNodeSetup(Option.empty, KAFKA_2_7_IV0) + } + + @Test + def testControllerFeatureZNodeSetupWhenFeatureVersioningIsEnabledWithDisabledExistingFeatureZNode(): Unit = { + testControllerFeatureZNodeSetup(Some(new FeatureZNode(FeatureZNodeStatus.Disabled, Features.emptyFinalizedFeatures())), KAFKA_2_7_IV0) + } + + @Test + def testControllerFeatureZNodeSetupWhenFeatureVersioningIsEnabledWithEnabledExistingFeatureZNode(): Unit = { + testControllerFeatureZNodeSetup(Some(new FeatureZNode(FeatureZNodeStatus.Enabled, Features.emptyFinalizedFeatures())), KAFKA_2_7_IV0) + } + + @Test + def testControllerFeatureZNodeSetupWhenFeatureVersioningIsDisabledWithNonExistingFeatureZNode(): Unit = { + testControllerFeatureZNodeSetup(Option.empty, KAFKA_2_6_IV0) + } + + @Test + def testControllerFeatureZNodeSetupWhenFeatureVersioningIsDisabledWithDisabledExistingFeatureZNode(): Unit = { + testControllerFeatureZNodeSetup(Some(new FeatureZNode(FeatureZNodeStatus.Disabled, Features.emptyFinalizedFeatures())), KAFKA_2_6_IV0) + } + + @Test + def testControllerFeatureZNodeSetupWhenFeatureVersioningIsDisabledWithEnabledExistingFeatureZNode(): Unit = { + testControllerFeatureZNodeSetup(Some(new FeatureZNode(FeatureZNodeStatus.Enabled, Features.emptyFinalizedFeatures())), KAFKA_2_6_IV0) + } + @Test def testControllerDetectsBouncedBrokers(): Unit = { servers = makeServers(2, enableControlledShutdown = false) @@ -715,7 +746,72 @@ class ControllerIntegrationTest extends ZooKeeperTestHarness { doAnswer((_: InvocationOnMock) => { latch.countDown() }).doCallRealMethod().when(spyThread).awaitShutdown() - controller.shutdown() + controller.shutdown() + } + + private def testControllerFeatureZNodeSetup(initialZNode: Option[FeatureZNode], + interBrokerProtocolVersion: ApiVersion): Unit = { + val versionBeforeOpt = initialZNode match { + case Some(node) => + zkClient.createFeatureZNode(node) + Some(zkClient.getDataAndVersion(FeatureZNode.path)._2) + case None => + Option.empty + } + servers = makeServers(1, interBrokerProtocolVersion = Some(interBrokerProtocolVersion)) + TestUtils.waitUntilControllerElected(zkClient) + // Below we wait on a dummy event to finish processing in the controller event thread. + // We schedule this dummy event only after the controller is elected, which is a sign that the + // controller has already started processing the Startup event. Waiting on the dummy event is + // used to make sure that the event thread has completed processing Startup event, that triggers + // the setup of FeatureZNode. + val controller = getController().kafkaController + val latch = new CountDownLatch(1) + controller.eventManager.put(new MockEvent(ControllerState.TopicChange) { + override def process(): Unit = { + latch.countDown() + } + override def preempt(): Unit = {} + }) + latch.await() + + val (mayBeFeatureZNodeBytes, versionAfter) = zkClient.getDataAndVersion(FeatureZNode.path) + val newZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get) + if (interBrokerProtocolVersion >= KAFKA_2_7_IV0) { + val emptyZNode = new FeatureZNode(FeatureZNodeStatus.Enabled, Features.emptyFinalizedFeatures) + initialZNode match { + case Some(node) => { + node.status match { + case FeatureZNodeStatus.Enabled => + assertEquals(versionBeforeOpt.get, versionAfter) + assertEquals(node, newZNode) + case FeatureZNodeStatus.Disabled => + assertEquals(versionBeforeOpt.get + 1, versionAfter) + assertEquals(emptyZNode, newZNode) + } + } + case None => + assertEquals(0, versionAfter) + assertEquals(new FeatureZNode(FeatureZNodeStatus.Enabled, Features.emptyFinalizedFeatures), newZNode) + } + } else { + val emptyZNode = new FeatureZNode(FeatureZNodeStatus.Disabled, Features.emptyFinalizedFeatures) + initialZNode match { + case Some(node) => { + node.status match { + case FeatureZNodeStatus.Enabled => + assertEquals(versionBeforeOpt.get + 1, versionAfter) + assertEquals(emptyZNode, newZNode) + case FeatureZNodeStatus.Disabled => + assertEquals(versionBeforeOpt.get, versionAfter) + assertEquals(emptyZNode, newZNode) + } + } + case None => + assertEquals(0, versionAfter) + assertEquals(new FeatureZNode(FeatureZNodeStatus.Disabled, Features.emptyFinalizedFeatures), newZNode) + } + } } @Test @@ -840,6 +936,7 @@ class ControllerIntegrationTest extends ZooKeeperTestHarness { listeners : Option[String] = None, listenerSecurityProtocolMap : Option[String] = None, controlPlaneListenerName : Option[String] = None, + interBrokerProtocolVersion: Option[ApiVersion] = None, logDirCount: Int = 1) = { val configs = TestUtils.createBrokerConfigs(numConfigs, zkConnect, enableControlledShutdown = enableControlledShutdown, logDirCount = logDirCount) configs.foreach { config => @@ -849,6 +946,7 @@ class ControllerIntegrationTest extends ZooKeeperTestHarness { listeners.foreach(listener => config.setProperty(KafkaConfig.ListenersProp, listener)) listenerSecurityProtocolMap.foreach(listenerMap => config.setProperty(KafkaConfig.ListenerSecurityProtocolMapProp, listenerMap)) controlPlaneListenerName.foreach(controlPlaneListener => config.setProperty(KafkaConfig.ControlPlaneListenerNameProp, controlPlaneListener)) + interBrokerProtocolVersion.foreach(ibp => config.setProperty(KafkaConfig.InterBrokerProtocolVersionProp, ibp.toString)) } configs.map(config => TestUtils.createServer(KafkaConfig.fromProps(config))) } diff --git a/core/src/test/scala/unit/kafka/server/BrokerFeaturesTest.scala b/core/src/test/scala/unit/kafka/server/BrokerFeaturesTest.scala new file mode 100644 index 0000000000000..2fe08fb3e1317 --- /dev/null +++ b/core/src/test/scala/unit/kafka/server/BrokerFeaturesTest.scala @@ -0,0 +1,106 @@ +/** + * 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.feature.{Features, FinalizedVersionRange, SupportedVersionRange} +import org.junit.Assert.{assertEquals, assertFalse, assertTrue} +import org.junit.Test + +import scala.jdk.CollectionConverters._ + +class BrokerFeaturesTest { + + @Test + def testEmpty(): Unit = { + assertTrue(BrokerFeatures.createDefault().supportedFeatures.empty) + } + + @Test + def testIncompatibilitiesDueToAbsentFeature(): Unit = { + val brokerFeatures = BrokerFeatures.createDefault() + val supportedFeatures = Features.supportedFeatures(Map[String, SupportedVersionRange]( + "test_feature_1" -> new SupportedVersionRange(1, 4), + "test_feature_2" -> new SupportedVersionRange(1, 3)).asJava) + brokerFeatures.setSupportedFeatures(supportedFeatures) + + val compatibleFeatures = Map[String, FinalizedVersionRange]( + "test_feature_1" -> new FinalizedVersionRange(2, 3)) + val inCompatibleFeatures = Map[String, FinalizedVersionRange]( + "test_feature_3" -> new FinalizedVersionRange(3, 4)) + val features = compatibleFeatures++inCompatibleFeatures + val finalizedFeatures = Features.finalizedFeatures(features.asJava) + + assertEquals( + Features.finalizedFeatures(inCompatibleFeatures.asJava), + brokerFeatures.incompatibleFeatures(finalizedFeatures)) + assertTrue(BrokerFeatures.hasIncompatibleFeatures(supportedFeatures, finalizedFeatures)) + } + + @Test + def testIncompatibilitiesDueToIncompatibleFeature(): Unit = { + val brokerFeatures = BrokerFeatures.createDefault() + val supportedFeatures = Features.supportedFeatures(Map[String, SupportedVersionRange]( + "test_feature_1" -> new SupportedVersionRange(1, 4), + "test_feature_2" -> new SupportedVersionRange(1, 3)).asJava) + brokerFeatures.setSupportedFeatures(supportedFeatures) + + val compatibleFeatures = Map[String, FinalizedVersionRange]( + "test_feature_1" -> new FinalizedVersionRange(2, 3)) + val inCompatibleFeatures = Map[String, FinalizedVersionRange]( + "test_feature_2" -> new FinalizedVersionRange(1, 4)) + val features = compatibleFeatures++inCompatibleFeatures + val finalizedFeatures = Features.finalizedFeatures(features.asJava) + + assertEquals( + Features.finalizedFeatures(inCompatibleFeatures.asJava), + brokerFeatures.incompatibleFeatures(finalizedFeatures)) + assertTrue(BrokerFeatures.hasIncompatibleFeatures(supportedFeatures, finalizedFeatures)) + } + + @Test + def testCompatibleFeatures(): Unit = { + val brokerFeatures = BrokerFeatures.createDefault() + val supportedFeatures = Features.supportedFeatures(Map[String, SupportedVersionRange]( + "test_feature_1" -> new SupportedVersionRange(1, 4), + "test_feature_2" -> new SupportedVersionRange(1, 3)).asJava) + brokerFeatures.setSupportedFeatures(supportedFeatures) + + val compatibleFeatures = Map[String, FinalizedVersionRange]( + "test_feature_1" -> new FinalizedVersionRange(2, 3), + "test_feature_2" -> new FinalizedVersionRange(1, 3)) + val finalizedFeatures = Features.finalizedFeatures(compatibleFeatures.asJava) + assertTrue(brokerFeatures.incompatibleFeatures(finalizedFeatures).empty()) + assertFalse(BrokerFeatures.hasIncompatibleFeatures(supportedFeatures, finalizedFeatures)) + } + + @Test + def testDefaultFinalizedFeatures(): Unit = { + val brokerFeatures = BrokerFeatures.createDefault() + val supportedFeatures = Features.supportedFeatures(Map[String, SupportedVersionRange]( + "test_feature_1" -> new SupportedVersionRange(1, 4), + "test_feature_2" -> new SupportedVersionRange(1, 3), + "test_feature_3" -> new SupportedVersionRange(3, 7)).asJava) + brokerFeatures.setSupportedFeatures(supportedFeatures) + + val expectedFeatures = Map[String, FinalizedVersionRange]( + "test_feature_1" -> new FinalizedVersionRange(1, 4), + "test_feature_2" -> new FinalizedVersionRange(1, 3), + "test_feature_3" -> new FinalizedVersionRange(3, 7)) + assertEquals(Features.finalizedFeatures(expectedFeatures.asJava), brokerFeatures.defaultFinalizedFeatures) + } +} diff --git a/core/src/test/scala/unit/kafka/server/FinalizedFeatureCacheTest.scala b/core/src/test/scala/unit/kafka/server/FinalizedFeatureCacheTest.scala index e664ea4669792..2f0f70ee0bad3 100644 --- a/core/src/test/scala/unit/kafka/server/FinalizedFeatureCacheTest.scala +++ b/core/src/test/scala/unit/kafka/server/FinalizedFeatureCacheTest.scala @@ -19,98 +19,100 @@ package kafka.server import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange} import org.junit.Assert.{assertEquals, assertThrows, assertTrue} -import org.junit.{Before, Test} +import org.junit.Test import scala.jdk.CollectionConverters._ class FinalizedFeatureCacheTest { - @Before - def setUp(): Unit = { - FinalizedFeatureCache.clear() - SupportedFeatures.clear() - } - @Test def testEmpty(): Unit = { - assertTrue(FinalizedFeatureCache.get.isEmpty) + assertTrue(new FinalizedFeatureCache(BrokerFeatures.createDefault()).get.isEmpty) } @Test def testUpdateOrThrowFailedDueToInvalidEpoch(): Unit = { val supportedFeatures = Map[String, SupportedVersionRange]( "feature_1" -> new SupportedVersionRange(1, 4)) - SupportedFeatures.update(Features.supportedFeatures(supportedFeatures.asJava)) + val brokerFeatures = BrokerFeatures.createDefault() + brokerFeatures.setSupportedFeatures(Features.supportedFeatures(supportedFeatures.asJava)) val features = Map[String, FinalizedVersionRange]( "feature_1" -> new FinalizedVersionRange(1, 4)) val finalizedFeatures = Features.finalizedFeatures(features.asJava) - FinalizedFeatureCache.updateOrThrow(finalizedFeatures, 10) - assertTrue(FinalizedFeatureCache.get.isDefined) - assertEquals(finalizedFeatures, FinalizedFeatureCache.get.get.features) - assertEquals(10, FinalizedFeatureCache.get.get.epoch) + val cache = new FinalizedFeatureCache(brokerFeatures) + cache.updateOrThrow(finalizedFeatures, 10) + assertTrue(cache.get.isDefined) + assertEquals(finalizedFeatures, cache.get.get.features) + assertEquals(10, cache.get.get.epoch) assertThrows( classOf[FeatureCacheUpdateException], - () => FinalizedFeatureCache.updateOrThrow(finalizedFeatures, 9)) + () => cache.updateOrThrow(finalizedFeatures, 9)) // Check that the failed updateOrThrow call did not make any mutations. - assertTrue(FinalizedFeatureCache.get.isDefined) - assertEquals(finalizedFeatures, FinalizedFeatureCache.get.get.features) - assertEquals(10, FinalizedFeatureCache.get.get.epoch) + assertTrue(cache.get.isDefined) + assertEquals(finalizedFeatures, cache.get.get.features) + assertEquals(10, cache.get.get.epoch) } @Test def testUpdateOrThrowFailedDueToInvalidFeatures(): Unit = { - val supportedFeatures = Map[String, SupportedVersionRange]( - "feature_1" -> new SupportedVersionRange(1, 1)) - SupportedFeatures.update(Features.supportedFeatures(supportedFeatures.asJava)) + val supportedFeatures = + Map[String, SupportedVersionRange]("feature_1" -> new SupportedVersionRange(1, 1)) + val brokerFeatures = BrokerFeatures.createDefault() + brokerFeatures.setSupportedFeatures(Features.supportedFeatures(supportedFeatures.asJava)) val features = Map[String, FinalizedVersionRange]( "feature_1" -> new FinalizedVersionRange(1, 2)) val finalizedFeatures = Features.finalizedFeatures(features.asJava) + val cache = new FinalizedFeatureCache(brokerFeatures) assertThrows( classOf[FeatureCacheUpdateException], - () => FinalizedFeatureCache.updateOrThrow(finalizedFeatures, 12)) + () => cache.updateOrThrow(finalizedFeatures, 12)) // Check that the failed updateOrThrow call did not make any mutations. - assertTrue(FinalizedFeatureCache.isEmpty) + assertTrue(cache.isEmpty) } @Test def testUpdateOrThrowSuccess(): Unit = { - val supportedFeatures = Map[String, SupportedVersionRange]( - "feature_1" -> new SupportedVersionRange(1, 4)) - SupportedFeatures.update(Features.supportedFeatures(supportedFeatures.asJava)) + val supportedFeatures = + Map[String, SupportedVersionRange]("feature_1" -> new SupportedVersionRange(1, 4)) + val brokerFeatures = BrokerFeatures.createDefault() + brokerFeatures.setSupportedFeatures(Features.supportedFeatures(supportedFeatures.asJava)) val features = Map[String, FinalizedVersionRange]( "feature_1" -> new FinalizedVersionRange(2, 3)) val finalizedFeatures = Features.finalizedFeatures(features.asJava) - FinalizedFeatureCache.updateOrThrow(finalizedFeatures, 12) - assertTrue(FinalizedFeatureCache.get.isDefined) - assertEquals(finalizedFeatures, FinalizedFeatureCache.get.get.features) - assertEquals(12, FinalizedFeatureCache.get.get.epoch) + val cache = new FinalizedFeatureCache(brokerFeatures) + cache.updateOrThrow(finalizedFeatures, 12) + assertTrue(cache.get.isDefined) + assertEquals(finalizedFeatures, cache.get.get.features) + assertEquals(12, cache.get.get.epoch) } @Test def testClear(): Unit = { - val supportedFeatures = Map[String, SupportedVersionRange]( - "feature_1" -> new SupportedVersionRange(1, 4)) - SupportedFeatures.update(Features.supportedFeatures(supportedFeatures.asJava)) + val supportedFeatures = + Map[String, SupportedVersionRange]("feature_1" -> new SupportedVersionRange(1, 4)) + val brokerFeatures = BrokerFeatures.createDefault() + brokerFeatures.setSupportedFeatures(Features.supportedFeatures(supportedFeatures.asJava)) val features = Map[String, FinalizedVersionRange]( "feature_1" -> new FinalizedVersionRange(2, 3)) val finalizedFeatures = Features.finalizedFeatures(features.asJava) - FinalizedFeatureCache.updateOrThrow(finalizedFeatures, 12) - assertTrue(FinalizedFeatureCache.get.isDefined) - assertEquals(finalizedFeatures, FinalizedFeatureCache.get.get.features) - assertEquals(12, FinalizedFeatureCache.get.get.epoch) + val cache = new FinalizedFeatureCache(brokerFeatures) + cache.updateOrThrow(finalizedFeatures, 12) + assertTrue(cache.get.isDefined) + assertEquals(finalizedFeatures, cache.get.get.features) + assertEquals(12, cache.get.get.epoch) - FinalizedFeatureCache.clear() - assertTrue(FinalizedFeatureCache.isEmpty) + cache.clear() + assertTrue(cache.isEmpty) } } diff --git a/core/src/test/scala/unit/kafka/server/FinalizedFeatureChangeListenerTest.scala b/core/src/test/scala/unit/kafka/server/FinalizedFeatureChangeListenerTest.scala index e4bba50f41276..cb8a661317794 100644 --- a/core/src/test/scala/unit/kafka/server/FinalizedFeatureChangeListenerTest.scala +++ b/core/src/test/scala/unit/kafka/server/FinalizedFeatureChangeListenerTest.scala @@ -17,32 +17,27 @@ package kafka.server -import java.util.concurrent.CountDownLatch +import java.util.concurrent.{CountDownLatch, TimeoutException} import kafka.zk.{FeatureZNode, FeatureZNodeStatus, ZkVersion, ZooKeeperTestHarness} import kafka.utils.TestUtils import org.apache.kafka.common.utils.Exit import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange} +import org.apache.kafka.test.{TestUtils => JTestUtils} import org.junit.Assert.{assertEquals, assertFalse, assertNotEquals, assertThrows, assertTrue} -import org.junit.{Before, Test} +import org.junit.Test -import scala.concurrent.TimeoutException import scala.jdk.CollectionConverters._ class FinalizedFeatureChangeListenerTest extends ZooKeeperTestHarness { - @Before - override def setUp(): Unit = { - super.setUp() - FinalizedFeatureCache.clear() - SupportedFeatures.clear() - } - private def createSupportedFeatures(): Features[SupportedVersionRange] = { + private def createBrokerFeatures(): BrokerFeatures = { val supportedFeaturesMap = Map[String, SupportedVersionRange]( "feature_1" -> new SupportedVersionRange(1, 4), "feature_2" -> new SupportedVersionRange(1, 3)) - SupportedFeatures.update(Features.supportedFeatures(supportedFeaturesMap.asJava)) - SupportedFeatures.get + val brokerFeatures = BrokerFeatures.createDefault() + brokerFeatures.setSupportedFeatures(Features.supportedFeatures(supportedFeaturesMap.asJava)) + brokerFeatures } private def createFinalizedFeatures(): FinalizedFeaturesAndEpoch = { @@ -56,20 +51,23 @@ class FinalizedFeatureChangeListenerTest extends ZooKeeperTestHarness { FinalizedFeaturesAndEpoch(finalizedFeatures, version) } - private def createListener(expectedCacheContent: Option[FinalizedFeaturesAndEpoch]): FinalizedFeatureChangeListener = { - val listener = new FinalizedFeatureChangeListener(zkClient) + private def createListener( + cache: FinalizedFeatureCache, + expectedCacheContent: Option[FinalizedFeaturesAndEpoch] + ): FinalizedFeatureChangeListener = { + val listener = new FinalizedFeatureChangeListener(cache, zkClient) assertFalse(listener.isListenerInitiated) - assertTrue(FinalizedFeatureCache.isEmpty) + assertTrue(cache.isEmpty) listener.initOrThrow(15000) assertTrue(listener.isListenerInitiated) if (expectedCacheContent.isDefined) { - val mayBeNewCacheContent = FinalizedFeatureCache.get + val mayBeNewCacheContent = cache.get assertFalse(mayBeNewCacheContent.isEmpty) val newCacheContent = mayBeNewCacheContent.get assertEquals(expectedCacheContent.get.features, newCacheContent.features) assertEquals(expectedCacheContent.get.epoch, newCacheContent.epoch) } else { - val mayBeNewCacheContent = FinalizedFeatureCache.get + val mayBeNewCacheContent = cache.get assertTrue(mayBeNewCacheContent.isEmpty) } listener @@ -78,25 +76,42 @@ class FinalizedFeatureChangeListenerTest extends ZooKeeperTestHarness { /** * Tests that the listener can be initialized, and that it can listen to ZK notifications * successfully from an "Enabled" FeatureZNode (the ZK data has no feature incompatibilities). + * Particularly the test checks if multiple notifications can be processed in ZK + * (i.e. whether the FeatureZNode watch can be re-established). */ @Test def testInitSuccessAndNotificationSuccess(): Unit = { - createSupportedFeatures() val initialFinalizedFeatures = createFinalizedFeatures() - val listener = createListener(Some(initialFinalizedFeatures)) + val brokerFeatures = createBrokerFeatures() + val cache = new FinalizedFeatureCache(brokerFeatures) + val listener = createListener(cache, Some(initialFinalizedFeatures)) - val updatedFinalizedFeaturesMap = Map[String, FinalizedVersionRange]( - "feature_1" -> new FinalizedVersionRange(2, 4)) - val updatedFinalizedFeatures = Features.finalizedFeatures(updatedFinalizedFeaturesMap.asJava) - zkClient.updateFeatureZNode(FeatureZNode(FeatureZNodeStatus.Enabled, updatedFinalizedFeatures)) - val (mayBeFeatureZNodeNewBytes, updatedVersion) = zkClient.getDataAndVersion(FeatureZNode.path) - assertNotEquals(updatedVersion, ZkVersion.UnknownVersion) - assertFalse(mayBeFeatureZNodeNewBytes.isEmpty) - assertTrue(updatedVersion > initialFinalizedFeatures.epoch) - TestUtils.waitUntilTrue(() => { - FinalizedFeatureCache.get.get.equals(FinalizedFeaturesAndEpoch(updatedFinalizedFeatures, updatedVersion)) - }, "Timed out waiting for FinalizedFeatureCache to be updated with new features") - assertTrue(listener.isListenerInitiated) + def updateAndCheckCache(finalizedFeatures: Features[FinalizedVersionRange]): Unit = { + zkClient.updateFeatureZNode(FeatureZNode(FeatureZNodeStatus.Enabled, finalizedFeatures)) + val (mayBeFeatureZNodeNewBytes, updatedVersion) = zkClient.getDataAndVersion(FeatureZNode.path) + assertNotEquals(updatedVersion, ZkVersion.UnknownVersion) + assertFalse(mayBeFeatureZNodeNewBytes.isEmpty) + assertTrue(updatedVersion > initialFinalizedFeatures.epoch) + + cache.waitUntilEpochOrThrow(updatedVersion, JTestUtils.DEFAULT_MAX_WAIT_MS) + assertEquals(FinalizedFeaturesAndEpoch(finalizedFeatures, updatedVersion), cache.get.get) + assertTrue(listener.isListenerInitiated) + } + + // Check if the write succeeds and a ZK notification is received that causes the feature cache + // to be populated. + updateAndCheckCache( + Features.finalizedFeatures( + Map[String, FinalizedVersionRange]( + "feature_1" -> new FinalizedVersionRange(2, 4)).asJava)) + // Check if second write succeeds and a ZK notification is again received that causes the cache + // to be populated. This check is needed to verify that the watch on the FeatureZNode was + // re-established after the notification was received due to the first write above. + updateAndCheckCache( + Features.finalizedFeatures( + Map[String, FinalizedVersionRange]( + "feature_1" -> new FinalizedVersionRange(2, 4), + "feature_2" -> new FinalizedVersionRange(1, 3)).asJava)) } /** @@ -105,16 +120,17 @@ class FinalizedFeatureChangeListenerTest extends ZooKeeperTestHarness { */ @Test def testFeatureZNodeDeleteNotificationProcessing(): Unit = { - createSupportedFeatures() + val brokerFeatures = createBrokerFeatures() + val cache = new FinalizedFeatureCache(brokerFeatures) val initialFinalizedFeatures = createFinalizedFeatures() - val listener = createListener(Some(initialFinalizedFeatures)) + val listener = createListener(cache, Some(initialFinalizedFeatures)) zkClient.deleteFeatureZNode() val (mayBeFeatureZNodeDeletedBytes, deletedVersion) = zkClient.getDataAndVersion(FeatureZNode.path) assertEquals(deletedVersion, ZkVersion.UnknownVersion) assertTrue(mayBeFeatureZNodeDeletedBytes.isEmpty) TestUtils.waitUntilTrue(() => { - FinalizedFeatureCache.isEmpty + cache.isEmpty }, "Timed out waiting for FinalizedFeatureCache to become empty") assertTrue(listener.isListenerInitiated) } @@ -125,9 +141,9 @@ class FinalizedFeatureChangeListenerTest extends ZooKeeperTestHarness { */ @Test def testFeatureZNodeDisablingNotificationProcessing(): Unit = { - createSupportedFeatures() + val brokerFeatures = createBrokerFeatures() + val cache = new FinalizedFeatureCache(brokerFeatures) val initialFinalizedFeatures = createFinalizedFeatures() - val listener = createListener(Some(initialFinalizedFeatures)) val updatedFinalizedFeaturesMap = Map[String, FinalizedVersionRange]() val updatedFinalizedFeatures = Features.finalizedFeatures(updatedFinalizedFeaturesMap.asJava) @@ -136,10 +152,36 @@ class FinalizedFeatureChangeListenerTest extends ZooKeeperTestHarness { assertNotEquals(updatedVersion, ZkVersion.UnknownVersion) assertFalse(mayBeFeatureZNodeNewBytes.isEmpty) assertTrue(updatedVersion > initialFinalizedFeatures.epoch) + assertTrue(cache.get.isEmpty) + } - TestUtils.waitUntilTrue(() => { - FinalizedFeatureCache.get.isEmpty - }, "Timed out waiting for FinalizedFeatureCache to become empty") + /** + * Tests that the wait operation on the cache fails (as expected) when an epoch can never be + * reached. Also tests that the wait operation on the cache succeeds when an epoch is expected to + * be reached. + */ + @Test + def testCacheUpdateWaitFailsForUnreachableVersion(): Unit = { + val initialFinalizedFeatures = createFinalizedFeatures() + val cache = new FinalizedFeatureCache(createBrokerFeatures()) + val listener = createListener(cache, Some(initialFinalizedFeatures)) + + assertThrows( + classOf[TimeoutException], + () => cache.waitUntilEpochOrThrow(initialFinalizedFeatures.epoch + 1, JTestUtils.DEFAULT_MAX_WAIT_MS)) + + val updatedFinalizedFeaturesMap = Map[String, FinalizedVersionRange]() + val updatedFinalizedFeatures = Features.finalizedFeatures(updatedFinalizedFeaturesMap.asJava) + zkClient.updateFeatureZNode(FeatureZNode(FeatureZNodeStatus.Disabled, updatedFinalizedFeatures)) + val (mayBeFeatureZNodeNewBytes, updatedVersion) = zkClient.getDataAndVersion(FeatureZNode.path) + assertNotEquals(updatedVersion, ZkVersion.UnknownVersion) + assertFalse(mayBeFeatureZNodeNewBytes.isEmpty) + assertTrue(updatedVersion > initialFinalizedFeatures.epoch) + + assertThrows( + classOf[TimeoutException], + () => cache.waitUntilEpochOrThrow(updatedVersion, JTestUtils.DEFAULT_MAX_WAIT_MS)) + assertTrue(cache.get.isEmpty) assertTrue(listener.isListenerInitiated) } @@ -149,7 +191,8 @@ class FinalizedFeatureChangeListenerTest extends ZooKeeperTestHarness { */ @Test def testInitFailureDueToFeatureIncompatibility(): Unit = { - createSupportedFeatures() + val brokerFeatures = createBrokerFeatures() + val cache = new FinalizedFeatureCache(brokerFeatures) val incompatibleFinalizedFeaturesMap = Map[String, FinalizedVersionRange]( "feature_1" -> new FinalizedVersionRange(2, 5)) @@ -162,14 +205,14 @@ class FinalizedFeatureChangeListenerTest extends ZooKeeperTestHarness { val exitLatch = new CountDownLatch(1) Exit.setExitProcedure((_, _) => exitLatch.countDown()) try { - val listener = new FinalizedFeatureChangeListener(zkClient) + val listener = new FinalizedFeatureChangeListener(cache, zkClient) assertFalse(listener.isListenerInitiated) - assertTrue(FinalizedFeatureCache.isEmpty) + assertTrue(cache.isEmpty) assertThrows(classOf[TimeoutException], () => listener.initOrThrow(5000)) exitLatch.await() assertFalse(listener.isListenerInitiated) assertTrue(listener.isListenerDead) - assertTrue(FinalizedFeatureCache.isEmpty) + assertTrue(cache.isEmpty) } finally { Exit.resetExitProcedure() } @@ -180,7 +223,9 @@ class FinalizedFeatureChangeListenerTest extends ZooKeeperTestHarness { */ @Test def testInitFailureDueToInvalidWaitTime(): Unit = { - val listener = new FinalizedFeatureChangeListener(zkClient) + val brokerFeatures = createBrokerFeatures() + val cache = new FinalizedFeatureCache(brokerFeatures) + val listener = new FinalizedFeatureChangeListener(cache, zkClient) assertThrows(classOf[IllegalArgumentException], () => listener.initOrThrow(0)) assertThrows(classOf[IllegalArgumentException], () => listener.initOrThrow(-1)) } @@ -191,14 +236,17 @@ class FinalizedFeatureChangeListenerTest extends ZooKeeperTestHarness { */ @Test def testNotificationFailureDueToFeatureIncompatibility(): Unit = { - createSupportedFeatures() + val brokerFeatures = createBrokerFeatures() + val cache = new FinalizedFeatureCache(brokerFeatures) val initialFinalizedFeatures = createFinalizedFeatures() - val listener = createListener(Some(initialFinalizedFeatures)) + val listener = createListener(cache, Some(initialFinalizedFeatures)) val exitLatch = new CountDownLatch(1) Exit.setExitProcedure((_, _) => exitLatch.countDown()) val incompatibleFinalizedFeaturesMap = Map[String, FinalizedVersionRange]( - "feature_1" -> new FinalizedVersionRange(2, 5)) + "feature_1" -> new FinalizedVersionRange( + brokerFeatures.supportedFeatures.get("feature_1").min(), + (brokerFeatures.supportedFeatures.get("feature_1").max() + 1).asInstanceOf[Short])) val incompatibleFinalizedFeatures = Features.finalizedFeatures(incompatibleFinalizedFeaturesMap.asJava) zkClient.updateFeatureZNode(FeatureZNode(FeatureZNodeStatus.Enabled, incompatibleFinalizedFeatures)) val (mayBeFeatureZNodeIncompatibleBytes, updatedVersion) = zkClient.getDataAndVersion(FeatureZNode.path) @@ -216,7 +264,7 @@ class FinalizedFeatureChangeListenerTest extends ZooKeeperTestHarness { listener.isListenerDead && // Make sure the cache contents are as expected, and, the incompatible features were not // applied. - FinalizedFeatureCache.get.get.equals(initialFinalizedFeatures) + cache.get.get.equals(initialFinalizedFeatures) }, "Timed out waiting for listener death and FinalizedFeatureCache to be updated") } finally { Exit.resetExitProcedure() diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala index 5ee740119944a..90be674438574 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala @@ -115,6 +115,8 @@ class KafkaApisTest { def createKafkaApis(interBrokerProtocolVersion: ApiVersion = ApiVersion.latestVersion, authorizer: Option[Authorizer] = None): KafkaApis = { + val brokerFeatures = BrokerFeatures.createDefault() + val cache = new FinalizedFeatureCache(brokerFeatures) val properties = TestUtils.createBrokerConfig(brokerId, "zk") properties.put(KafkaConfig.InterBrokerProtocolVersionProp, interBrokerProtocolVersion.toString) properties.put(KafkaConfig.LogMessageFormatVersionProp, interBrokerProtocolVersion.toString) @@ -135,8 +137,9 @@ class KafkaApisTest { brokerTopicStats, clusterId, time, - null - ) + null, + brokerFeatures, + cache) } @Test diff --git a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala index 301ecd70dc0ec..81307f792e7d0 100644 --- a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala +++ b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala @@ -582,6 +582,9 @@ class RequestQuotaTest extends BaseRequestTest { case ApiKeys.ALTER_ISR => new AlterIsrRequest.Builder(new AlterIsrRequestData()) + case ApiKeys.UPDATE_FEATURES => + new UpdateFeaturesRequest.Builder(new UpdateFeaturesRequestData()) + case _ => throw new IllegalArgumentException("Unsupported API key " + apiKey) } diff --git a/core/src/test/scala/unit/kafka/server/SupportedFeaturesTest.scala b/core/src/test/scala/unit/kafka/server/SupportedFeaturesTest.scala deleted file mode 100644 index c9a5aabb3291f..0000000000000 --- a/core/src/test/scala/unit/kafka/server/SupportedFeaturesTest.scala +++ /dev/null @@ -1,56 +0,0 @@ -/** - * 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.feature.{Features, FinalizedVersionRange, SupportedVersionRange} -import org.junit.Assert.{assertEquals, assertTrue} -import org.junit.{Before, Test} - -import scala.jdk.CollectionConverters._ - -class SupportedFeaturesTest { - @Before - def setUp(): Unit = { - SupportedFeatures.clear() - } - - @Test - def testEmpty(): Unit = { - assertTrue(SupportedFeatures.get.empty) - } - - @Test - def testIncompatibleFeatures(): Unit = { - val supportedFeatures = Map[String, SupportedVersionRange]( - "feature_1" -> new SupportedVersionRange(1, 4), - "feature_2" -> new SupportedVersionRange(1, 3)) - SupportedFeatures.update(Features.supportedFeatures(supportedFeatures.asJava)) - - val compatibleFeatures = Map[String, FinalizedVersionRange]( - "feature_1" -> new FinalizedVersionRange(2, 3)) - val inCompatibleFeatures = Map[String, FinalizedVersionRange]( - "feature_2" -> new FinalizedVersionRange(1, 4), - "feature_3" -> new FinalizedVersionRange(3, 4)) - val features = compatibleFeatures++inCompatibleFeatures - val finalizedFeatures = Features.finalizedFeatures(features.asJava) - - assertEquals( - Features.finalizedFeatures(inCompatibleFeatures.asJava), - SupportedFeatures.incompatibleFeatures(finalizedFeatures)) - } -} diff --git a/core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala b/core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala new file mode 100644 index 0000000000000..a16ff30d543e5 --- /dev/null +++ b/core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala @@ -0,0 +1,581 @@ +/** + * 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.{Optional, Properties} +import java.util.concurrent.ExecutionException + +import kafka.api.KAFKA_2_7_IV0 +import kafka.utils.TestUtils +import kafka.zk.{FeatureZNode, FeatureZNodeStatus, ZkVersion} +import kafka.utils.TestUtils.waitUntilTrue +import org.apache.kafka.clients.admin.{Admin, DescribeFeaturesOptions, FeatureUpdate, UpdateFeaturesOptions, UpdateFeaturesResult} +import org.apache.kafka.common.errors.InvalidRequestException +import org.apache.kafka.common.feature.FinalizedVersionRange +import org.apache.kafka.common.feature.{Features, SupportedVersionRange} +import org.apache.kafka.common.message.UpdateFeaturesRequestData +import org.apache.kafka.common.message.UpdateFeaturesRequestData.FeatureUpdateKeyCollection +import org.apache.kafka.common.protocol.Errors +import org.apache.kafka.common.requests.{UpdateFeaturesRequest, UpdateFeaturesResponse} +import org.apache.kafka.common.utils.Utils +import org.junit.Test +import org.junit.Assert.{assertEquals, assertFalse, assertNotEquals, assertNotNull, assertTrue} +import org.scalatest.Assertions.intercept + +import scala.jdk.CollectionConverters._ +import scala.reflect.ClassTag +import scala.util.matching.Regex + +class UpdateFeaturesTest extends BaseRequestTest { + + override def brokerCount = 3 + + override def brokerPropertyOverrides(props: Properties): Unit = { + props.put(KafkaConfig.InterBrokerProtocolVersionProp, KAFKA_2_7_IV0.toString) + } + + private def defaultSupportedFeatures(): Features[SupportedVersionRange] = { + Features.supportedFeatures(Utils.mkMap(Utils.mkEntry("feature_1", new SupportedVersionRange(1, 3)))) + } + + private def defaultFinalizedFeatures(): Features[FinalizedVersionRange] = { + Features.finalizedFeatures(Utils.mkMap(Utils.mkEntry("feature_1", new FinalizedVersionRange(1, 2)))) + } + + private def updateSupportedFeatures( + features: Features[SupportedVersionRange], targetServers: Set[KafkaServer]): Unit = { + targetServers.foreach(s => { + s.brokerFeatures.setSupportedFeatures(features) + s.zkClient.updateBrokerInfo(s.createBrokerInfo) + }) + + // Wait until updates to all BrokerZNode supported features propagate to the controller. + val brokerIds = targetServers.map(s => s.config.brokerId) + waitUntilTrue( + () => servers.exists(s => { + if (s.kafkaController.isActive) { + s.kafkaController.controllerContext.liveOrShuttingDownBrokers + .filter(b => brokerIds.contains(b.id)) + .forall(b => { + b.features.equals(features) + }) + } else { + false + } + }), + "Controller did not get broker updates") + } + + private def updateSupportedFeaturesInAllBrokers(features: Features[SupportedVersionRange]): Unit = { + updateSupportedFeatures(features, Set[KafkaServer]() ++ servers) + } + + private def updateFeatureZNode(features: Features[FinalizedVersionRange]): Int = { + val server = serverForId(0).get + val newNode = new FeatureZNode(FeatureZNodeStatus.Enabled, features) + val newVersion = server.zkClient.updateFeatureZNode(newNode) + servers.foreach(s => { + s.featureCache.waitUntilEpochOrThrow(newVersion, s.config.zkConnectionTimeoutMs) + }) + newVersion + } + + private def getFeatureZNode(): FeatureZNode = { + val (mayBeFeatureZNodeBytes, version) = serverForId(0).get.zkClient.getDataAndVersion(FeatureZNode.path) + assertNotEquals(version, ZkVersion.UnknownVersion) + FeatureZNode.decode(mayBeFeatureZNodeBytes.get) + } + + private def finalizedFeatures(features: java.util.Map[String, org.apache.kafka.clients.admin.FinalizedVersionRange]): Features[FinalizedVersionRange] = { + Features.finalizedFeatures(features.asScala.map { + case(name, versionRange) => + (name, new FinalizedVersionRange(versionRange.minVersionLevel(), versionRange.maxVersionLevel())) + }.asJava) + } + + private def supportedFeatures(features: java.util.Map[String, org.apache.kafka.clients.admin.SupportedVersionRange]): Features[SupportedVersionRange] = { + Features.supportedFeatures(features.asScala.map { + case(name, versionRange) => + (name, new SupportedVersionRange(versionRange.minVersion(), versionRange.maxVersion())) + }.asJava) + } + + private def checkFeatures(client: Admin, + expectedNode: FeatureZNode, + expectedFinalizedFeatures: Features[FinalizedVersionRange], + expectedFinalizedFeaturesEpoch: Long, + expectedSupportedFeatures: Features[SupportedVersionRange]): Unit = { + assertEquals(expectedNode, getFeatureZNode()) + val featureMetadata = client.describeFeatures( + new DescribeFeaturesOptions().sendRequestToController(true)).featureMetadata.get + assertEquals(expectedFinalizedFeatures, finalizedFeatures(featureMetadata.finalizedFeatures)) + assertEquals(expectedSupportedFeatures, supportedFeatures(featureMetadata.supportedFeatures)) + assertEquals(Optional.of(expectedFinalizedFeaturesEpoch), featureMetadata.finalizedFeaturesEpoch) + } + + private def checkException[ExceptionType <: Throwable](result: UpdateFeaturesResult, + featureExceptionMsgPatterns: Map[String, Regex]) + (implicit tag: ClassTag[ExceptionType]): Unit = { + featureExceptionMsgPatterns.foreach { + case (feature, exceptionMsgPattern) => + val exception = intercept[ExecutionException] { + result.values().get(feature).get() + } + val cause = exception.getCause + assertNotNull(cause) + assertEquals(cause.getClass, tag.runtimeClass) + assertTrue(s"Received unexpected error message: ${cause.getMessage}", + exceptionMsgPattern.findFirstIn(cause.getMessage).isDefined) + } + } + + /** + * Tests whether an invalid feature update does not get processed on the server as expected, + * and raises the ExceptionType on the client side as expected. + * + * @param feature the feature to be updated + * @param invalidUpdate the invalid feature update to be sent in the + * updateFeatures request to the server + * @param exceptionMsgPattern a pattern for the expected exception message + */ + private def testWithInvalidFeatureUpdate[ExceptionType <: Throwable](feature: String, + invalidUpdate: FeatureUpdate, + exceptionMsgPattern: Regex) + (implicit tag: ClassTag[ExceptionType]): Unit = { + TestUtils.waitUntilControllerElected(zkClient) + + updateSupportedFeaturesInAllBrokers(defaultSupportedFeatures()) + val versionBefore = updateFeatureZNode(defaultFinalizedFeatures()) + val adminClient = createAdminClient() + val nodeBefore = getFeatureZNode() + + val result = adminClient.updateFeatures(Utils.mkMap(Utils.mkEntry(feature, invalidUpdate)), new UpdateFeaturesOptions()) + + checkException[ExceptionType](result, Map(feature -> exceptionMsgPattern)) + checkFeatures( + adminClient, + nodeBefore, + defaultFinalizedFeatures(), + versionBefore, + defaultSupportedFeatures()) + } + + /** + * Tests that an UpdateFeatures request sent to a non-Controller node fails as expected. + */ + @Test + def testShouldFailRequestIfNotController(): Unit = { + TestUtils.waitUntilControllerElected(zkClient) + + updateSupportedFeaturesInAllBrokers(defaultSupportedFeatures()) + val versionBefore = updateFeatureZNode(defaultFinalizedFeatures()) + + val nodeBefore = getFeatureZNode() + val validUpdates = new FeatureUpdateKeyCollection() + val validUpdate = new UpdateFeaturesRequestData.FeatureUpdateKey(); + validUpdate.setFeature("feature_1"); + validUpdate.setMaxVersionLevel(defaultSupportedFeatures().get("feature_1").max()) + validUpdate.setAllowDowngrade(false) + validUpdates.add(validUpdate) + + val response = connectAndReceive[UpdateFeaturesResponse]( + new UpdateFeaturesRequest.Builder(new UpdateFeaturesRequestData().setFeatureUpdates(validUpdates)).build(), + notControllerSocketServer) + + assertEquals(Errors.NOT_CONTROLLER, Errors.forCode(response.data.errorCode())) + assertNotNull(response.data.errorMessage()) + assertEquals(0, response.data.results.size) + checkFeatures( + createAdminClient(), + nodeBefore, + defaultFinalizedFeatures(), + versionBefore, + defaultSupportedFeatures()) + } + + /** + * Tests that an UpdateFeatures request fails in the Controller, when, for a feature the + * allowDowngrade flag is not set during a downgrade request. + */ + @Test + def testShouldFailRequestWhenDowngradeFlagIsNotSetDuringDowngrade(): Unit = { + val targetMaxVersionLevel = (defaultFinalizedFeatures().get("feature_1").max() - 1).asInstanceOf[Short] + testWithInvalidFeatureUpdate[InvalidRequestException]( + "feature_1", + new FeatureUpdate(targetMaxVersionLevel,false), + ".*Can not downgrade finalized feature.*allowDowngrade.*".r) + } + + /** + * Tests that an UpdateFeatures request fails in the Controller, when, for a feature the downgrade + * is attempted to a max version level higher than the existing max version level. + */ + @Test + def testShouldFailRequestWhenDowngradeToHigherVersionLevelIsAttempted(): Unit = { + val targetMaxVersionLevel = (defaultFinalizedFeatures().get("feature_1").max() + 1).asInstanceOf[Short] + testWithInvalidFeatureUpdate[InvalidRequestException]( + "feature_1", + new FeatureUpdate(targetMaxVersionLevel, true), + ".*When the allowDowngrade flag set in the request, the provided maxVersionLevel:3.*existing maxVersionLevel:2.*".r) + } + + /** + * Tests that an UpdateFeatures request fails in the Controller, when, a feature deletion is + * attempted without setting the allowDowngrade flag. + */ + @Test + def testShouldFailRequestInServerWhenDowngradeFlagIsNotSetDuringDeletion(): Unit = { + TestUtils.waitUntilControllerElected(zkClient) + + updateSupportedFeaturesInAllBrokers(defaultSupportedFeatures()) + val versionBefore = updateFeatureZNode(defaultFinalizedFeatures()) + + val adminClient = createAdminClient() + val nodeBefore = getFeatureZNode() + + val invalidUpdates + = new UpdateFeaturesRequestData.FeatureUpdateKeyCollection(); + val invalidUpdate = new UpdateFeaturesRequestData.FeatureUpdateKey(); + invalidUpdate.setFeature("feature_1") + invalidUpdate.setMaxVersionLevel(0) + invalidUpdate.setAllowDowngrade(false) + invalidUpdates.add(invalidUpdate); + val requestData = new UpdateFeaturesRequestData() + requestData.setFeatureUpdates(invalidUpdates); + + val response = connectAndReceive[UpdateFeaturesResponse]( + new UpdateFeaturesRequest.Builder(new UpdateFeaturesRequestData().setFeatureUpdates(invalidUpdates)).build(), + controllerSocketServer) + + assertEquals(1, response.data().results().size()) + val result = response.data.results.asScala.head + assertEquals("feature_1", result.feature) + assertEquals(Errors.INVALID_REQUEST, Errors.forCode(result.errorCode)) + assertNotNull(result.errorMessage) + assertFalse(result.errorMessage.isEmpty) + val exceptionMsgPattern = ".*Can not provide maxVersionLevel: 0 less than 1.*allowDowngrade.*".r + assertTrue(result.errorMessage, exceptionMsgPattern.findFirstIn(result.errorMessage).isDefined) + checkFeatures( + adminClient, + nodeBefore, + defaultFinalizedFeatures(), + versionBefore, + defaultSupportedFeatures()) + } + + /** + * Tests that an UpdateFeatures request fails in the Controller, when, a feature version level + * upgrade is attempted for a non-existing feature. + */ + @Test + def testShouldFailRequestDuringDeletionOfNonExistingFeature(): Unit = { + testWithInvalidFeatureUpdate[InvalidRequestException]( + "feature_non_existing", + new FeatureUpdate(3, true), + ".*Could not apply finalized feature update because the provided feature is not supported.*".r) + } + + /** + * Tests that an UpdateFeatures request fails in the Controller, when, a feature version level + * upgrade is attempted to a version level same as the existing max version level. + */ + @Test + def testShouldFailRequestWhenUpgradingToSameVersionLevel(): Unit = { + val targetMaxVersionLevel = defaultFinalizedFeatures().get("feature_1").max() + testWithInvalidFeatureUpdate[InvalidRequestException]( + "feature_1", + new FeatureUpdate(targetMaxVersionLevel, false), + ".*Can not upgrade a finalized feature.*to the same value.*".r) + } + + private def testShouldFailRequestDuringBrokerMaxVersionLevelIncompatibility( + featureName: String, + supportedVersionRange: SupportedVersionRange, + initialFinalizedVersionRange: Option[FinalizedVersionRange] + ): Unit = { + TestUtils.waitUntilControllerElected(zkClient) + + val controller = servers.filter { server => server.kafkaController.isActive}.head + val nonControllerServers = servers.filter { server => !server.kafkaController.isActive} + // We setup the supported features on the broker such that 1/3 of the brokers does not + // support an expected feature version, while 2/3 brokers support the expected feature + // version. + val brokersWithVersionIncompatibility = Set[KafkaServer](nonControllerServers.head) + val versionCompatibleBrokers = Set[KafkaServer](nonControllerServers(1), controller) + + val supportedFeatures = Features.supportedFeatures(Utils.mkMap(Utils.mkEntry(featureName, supportedVersionRange))) + updateSupportedFeatures(supportedFeatures, versionCompatibleBrokers) + + val unsupportedMaxVersion = (supportedVersionRange.max() - 1).asInstanceOf[Short] + val supportedFeaturesWithVersionIncompatibility = Features.supportedFeatures( + Utils.mkMap( + Utils.mkEntry("feature_1", + new SupportedVersionRange( + supportedVersionRange.min(), + unsupportedMaxVersion)))) + updateSupportedFeatures(supportedFeaturesWithVersionIncompatibility, brokersWithVersionIncompatibility) + + val initialFinalizedFeatures = initialFinalizedVersionRange.map( + versionRange => Features.finalizedFeatures(Utils.mkMap(Utils.mkEntry(featureName, versionRange))) + ).getOrElse(Features.emptyFinalizedFeatures()) + val versionBefore = updateFeatureZNode(initialFinalizedFeatures) + + val invalidUpdate = new FeatureUpdate(supportedVersionRange.max(), false) + val nodeBefore = getFeatureZNode() + val adminClient = createAdminClient() + val result = adminClient.updateFeatures( + Utils.mkMap(Utils.mkEntry("feature_1", invalidUpdate)), + new UpdateFeaturesOptions()) + + checkException[InvalidRequestException](result, Map("feature_1" -> ".*brokers.*incompatible.*".r)) + checkFeatures( + adminClient, + nodeBefore, + initialFinalizedFeatures, + versionBefore, + supportedFeatures) + } + + /** + * Tests that an UpdateFeatures request fails in the Controller, when for an existing finalized + * feature, a version level upgrade introduces a version incompatibility with existing supported + * features. + */ + @Test + def testShouldFailRequestDuringBrokerMaxVersionLevelIncompatibilityForExistingFinalizedFeature(): Unit = { + val feature = "feature_1" + testShouldFailRequestDuringBrokerMaxVersionLevelIncompatibility( + feature, + defaultSupportedFeatures().get(feature), + Some(defaultFinalizedFeatures().get(feature))) + } + + /** + * Tests that an UpdateFeatures request fails in the Controller, when for a non-existing finalized + * feature, a version level upgrade introduces a version incompatibility with existing supported + * features. + */ + @Test + def testShouldFailRequestDuringBrokerMaxVersionLevelIncompatibilityWithNoExistingFinalizedFeature(): Unit = { + val feature = "feature_1" + testShouldFailRequestDuringBrokerMaxVersionLevelIncompatibility( + feature, + defaultSupportedFeatures().get(feature), + Option.empty) + } + + /** + * Tests that an UpdateFeatures request succeeds in the Controller, when, there are no existing + * finalized features in FeatureZNode when the test starts. + */ + @Test + def testSuccessfulFeatureUpgradeAndWithNoExistingFinalizedFeatures(): Unit = { + TestUtils.waitUntilControllerElected(zkClient) + + val supportedFeatures = + Features.supportedFeatures( + Utils.mkMap( + Utils.mkEntry("feature_1", new SupportedVersionRange(1, 3)), + Utils.mkEntry("feature_2", new SupportedVersionRange(2, 5)))) + updateSupportedFeaturesInAllBrokers(supportedFeatures) + val versionBefore = updateFeatureZNode(Features.emptyFinalizedFeatures()) + + val targetFinalizedFeatures = Features.finalizedFeatures( + Utils.mkMap( + Utils.mkEntry("feature_1", new FinalizedVersionRange(1, 3)), + Utils.mkEntry("feature_2", new FinalizedVersionRange(2, 3)))) + val update1 = new FeatureUpdate(targetFinalizedFeatures.get("feature_1").max(), false) + val update2 = new FeatureUpdate(targetFinalizedFeatures.get("feature_2").max(), false) + + val adminClient = createAdminClient() + adminClient.updateFeatures( + Utils.mkMap(Utils.mkEntry("feature_1", update1), Utils.mkEntry("feature_2", update2)), + new UpdateFeaturesOptions() + ).all().get() + + checkFeatures( + adminClient, + new FeatureZNode(FeatureZNodeStatus.Enabled, targetFinalizedFeatures), + targetFinalizedFeatures, + versionBefore + 1, + supportedFeatures) + } + + /** + * Tests that an UpdateFeatures request succeeds in the Controller, when, the request contains + * both a valid feature version level upgrade as well as a downgrade request. + */ + @Test + def testSuccessfulFeatureUpgradeAndDowngrade(): Unit = { + TestUtils.waitUntilControllerElected(zkClient) + + val supportedFeatures = Features.supportedFeatures( + Utils.mkMap( + Utils.mkEntry("feature_1", new SupportedVersionRange(1, 3)), + Utils.mkEntry("feature_2", new SupportedVersionRange(2, 5)))) + updateSupportedFeaturesInAllBrokers(supportedFeatures) + val initialFinalizedFeatures = Features.finalizedFeatures( + Utils.mkMap( + Utils.mkEntry("feature_1", new FinalizedVersionRange(1, 2)), + Utils.mkEntry("feature_2", new FinalizedVersionRange(2, 4)))) + val versionBefore = updateFeatureZNode(initialFinalizedFeatures) + + // Below we aim to do the following: + // - Valid upgrade of feature_1 maxVersionLevel from 2 to 3 + // - Valid downgrade of feature_2 maxVersionLevel from 4 to 3 + val targetFinalizedFeatures = Features.finalizedFeatures( + Utils.mkMap( + Utils.mkEntry("feature_1", new FinalizedVersionRange(1, 3)), + Utils.mkEntry("feature_2", new FinalizedVersionRange(2, 3)))) + val update1 = new FeatureUpdate(targetFinalizedFeatures.get("feature_1").max(), false) + val update2 = new FeatureUpdate(targetFinalizedFeatures.get("feature_2").max(), true) + + val adminClient = createAdminClient() + adminClient.updateFeatures( + Utils.mkMap(Utils.mkEntry("feature_1", update1), Utils.mkEntry("feature_2", update2)), + new UpdateFeaturesOptions() + ).all().get() + + checkFeatures( + adminClient, + new FeatureZNode(FeatureZNodeStatus.Enabled, targetFinalizedFeatures), + targetFinalizedFeatures, + versionBefore + 1, + supportedFeatures) + } + + /** + * Tests that an UpdateFeatures request succeeds partially in the Controller, when, the request + * contains a valid feature version level upgrade and an invalid version level downgrade. + * i.e. expect the upgrade operation to succeed, and the downgrade operation to fail. + */ + @Test + def testPartialSuccessDuringValidFeatureUpgradeAndInvalidDowngrade(): Unit = { + TestUtils.waitUntilControllerElected(zkClient) + + val supportedFeatures = Features.supportedFeatures( + Utils.mkMap( + Utils.mkEntry("feature_1", new SupportedVersionRange(1, 3)), + Utils.mkEntry("feature_2", new SupportedVersionRange(2, 5)))) + updateSupportedFeaturesInAllBrokers(supportedFeatures) + val initialFinalizedFeatures = Features.finalizedFeatures( + Utils.mkMap( + Utils.mkEntry("feature_1", new FinalizedVersionRange(1, 2)), + Utils.mkEntry("feature_2", new FinalizedVersionRange(2, 4)))) + val versionBefore = updateFeatureZNode(initialFinalizedFeatures) + + // Below we aim to do the following: + // - Valid upgrade of feature_1 maxVersionLevel from 2 to 3 + // - Invalid downgrade of feature_2 maxVersionLevel from 4 to 3 + // (because we intentionally do not set the allowDowngrade flag) + val targetFinalizedFeatures = Features.finalizedFeatures( + Utils.mkMap( + Utils.mkEntry("feature_1", new FinalizedVersionRange(1, 3)), + Utils.mkEntry("feature_2", new FinalizedVersionRange(2, 3)))) + val validUpdate = new FeatureUpdate(targetFinalizedFeatures.get("feature_1").max(), false) + val invalidUpdate = new FeatureUpdate(targetFinalizedFeatures.get("feature_2").max(), false) + + val adminClient = createAdminClient() + val result = adminClient.updateFeatures( + Utils.mkMap(Utils.mkEntry("feature_1", validUpdate), Utils.mkEntry("feature_2", invalidUpdate)), + new UpdateFeaturesOptions()) + + // Expect update for "feature_1" to have succeeded. + result.values().get("feature_1").get() + // Expect update for "feature_2" to have failed. + checkException[InvalidRequestException]( + result, Map("feature_2" -> ".*Can not downgrade finalized feature.*allowDowngrade.*".r)) + val expectedFeatures = Features.finalizedFeatures( + Utils.mkMap( + Utils.mkEntry("feature_1", targetFinalizedFeatures.get("feature_1")), + Utils.mkEntry("feature_2", initialFinalizedFeatures.get("feature_2")))) + checkFeatures( + adminClient, + FeatureZNode(FeatureZNodeStatus.Enabled, expectedFeatures), + expectedFeatures, + versionBefore + 1, + supportedFeatures) + } + + /** + * Tests that an UpdateFeatures request succeeds partially in the Controller, when, the request + * contains an invalid feature version level upgrade and a valid version level downgrade. + * i.e. expect the downgrade operation to succeed, and the upgrade operation to fail. + */ + @Test + def testPartialSuccessDuringInvalidFeatureUpgradeAndValidDowngrade(): Unit = { + TestUtils.waitUntilControllerElected(zkClient) + + val controller = servers.filter { server => server.kafkaController.isActive}.head + val nonControllerServers = servers.filter { server => !server.kafkaController.isActive} + // We setup the supported features on the broker such that 1/3 of the brokers does not + // support an expected feature version, while 2/3 brokers support the expected feature + // version. + val brokersWithVersionIncompatibility = Set[KafkaServer](nonControllerServers.head) + val versionCompatibleBrokers = Set[KafkaServer](nonControllerServers(1), controller) + + val supportedFeatures = Features.supportedFeatures( + Utils.mkMap( + Utils.mkEntry("feature_1", new SupportedVersionRange(1, 3)), + Utils.mkEntry("feature_2", new SupportedVersionRange(2, 5)))) + updateSupportedFeatures(supportedFeatures, versionCompatibleBrokers) + + val supportedFeaturesWithVersionIncompatibility = Features.supportedFeatures( + Utils.mkMap( + Utils.mkEntry("feature_1", new SupportedVersionRange(1, 2)), + Utils.mkEntry("feature_2", supportedFeatures.get("feature_2")))) + updateSupportedFeatures(supportedFeaturesWithVersionIncompatibility, brokersWithVersionIncompatibility) + + val initialFinalizedFeatures = Features.finalizedFeatures( + Utils.mkMap( + Utils.mkEntry("feature_1", new FinalizedVersionRange(1, 2)), + Utils.mkEntry("feature_2", new FinalizedVersionRange(2, 4)))) + val versionBefore = updateFeatureZNode(initialFinalizedFeatures) + + // Below we aim to do the following: + // - Invalid upgrade of feature_1 maxVersionLevel from 2 to 3 + // (because one of the brokers does not support the max version: 3) + // - Valid downgrade of feature_2 maxVersionLevel from 4 to 3 + val targetFinalizedFeatures = Features.finalizedFeatures( + Utils.mkMap( + Utils.mkEntry("feature_1", new FinalizedVersionRange(1, 3)), + Utils.mkEntry("feature_2", new FinalizedVersionRange(2, 3)))) + val invalidUpdate = new FeatureUpdate(targetFinalizedFeatures.get("feature_1").max(), false) + val validUpdate = new FeatureUpdate(targetFinalizedFeatures.get("feature_2").max(), true) + + val adminClient = createAdminClient() + val result = adminClient.updateFeatures( + Utils.mkMap(Utils.mkEntry("feature_1", invalidUpdate), Utils.mkEntry("feature_2", validUpdate)), + new UpdateFeaturesOptions()) + + // Expect update for "feature_2" to have succeeded. + result.values().get("feature_2").get() + // Expect update for "feature_1" to have failed. + checkException[InvalidRequestException](result, Map("feature_1" -> ".*brokers.*incompatible.*".r)) + val expectedFeatures = Features.finalizedFeatures( + Utils.mkMap( + Utils.mkEntry("feature_1", initialFinalizedFeatures.get("feature_1")), + Utils.mkEntry("feature_2", targetFinalizedFeatures.get("feature_2")))) + checkFeatures( + adminClient, + FeatureZNode(FeatureZNodeStatus.Enabled, expectedFeatures), + expectedFeatures, + versionBefore + 1, + supportedFeatures) + } +} diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/metadata/MetadataRequestBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/metadata/MetadataRequestBenchmark.java index 98a7afe4a7417..e2242c17e406f 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/metadata/MetadataRequestBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/metadata/MetadataRequestBenchmark.java @@ -22,11 +22,13 @@ import kafka.coordinator.transaction.TransactionCoordinator; import kafka.network.RequestChannel; import kafka.server.AdminManager; +import kafka.server.BrokerFeatures; import kafka.server.BrokerTopicStats; import kafka.server.ClientQuotaManager; import kafka.server.ClientRequestQuotaManager; import kafka.server.ControllerMutationQuotaManager; import kafka.server.FetchManager; +import kafka.server.FinalizedFeatureCache; import kafka.server.KafkaApis; import kafka.server.KafkaConfig; import kafka.server.KafkaConfig$; @@ -164,6 +166,7 @@ private KafkaApis createKafkaApis() { Properties kafkaProps = new Properties(); kafkaProps.put(KafkaConfig$.MODULE$.ZkConnectProp(), "zk"); kafkaProps.put(KafkaConfig$.MODULE$.BrokerIdProp(), brokerId + ""); + BrokerFeatures brokerFeatures = BrokerFeatures.createDefault(); return new KafkaApis(requestChannel, replicaManager, adminManager, @@ -181,7 +184,9 @@ private KafkaApis createKafkaApis() { brokerTopicStats, "clusterId", new SystemTime(), - null); + null, + brokerFeatures, + new FinalizedFeatureCache(brokerFeatures)); } @TearDown(Level.Trial)