From 824e2f7141f208acc5cdb661043949d0977725c4 Mon Sep 17 00:00:00 2001 From: Kowshik Prakasam Date: Fri, 10 Jul 2020 22:46:38 -0700 Subject: [PATCH 01/41] Implement KIP-584 write path --- .../org/apache/kafka/clients/admin/Admin.java | 64 +++ .../admin/DescribeFeaturesOptions.java | 35 ++ .../clients/admin/DescribeFeaturesResult.java | 16 + .../kafka/clients/admin/FeatureMetadata.java | 84 ++++ .../clients/admin/FinalizedFeatureUpdate.java | 87 ++++ .../kafka/clients/admin/KafkaAdminClient.java | 104 ++++ .../admin/UpdateFinalizedFeaturesOptions.java | 15 + .../admin/UpdateFinalizedFeaturesResult.java | 15 + ...FinalizedFeatureUpdateFailedException.java | 13 + .../apache/kafka/common/feature/Features.java | 9 + .../common/feature/FinalizedVersionRange.java | 4 + .../apache/kafka/common/protocol/ApiKeys.java | 5 + .../apache/kafka/common/protocol/Errors.java | 4 +- .../common/requests/AbstractRequest.java | 2 + .../common/requests/AbstractResponse.java | 2 + .../common/requests/ApiVersionsResponse.java | 61 ++- .../UpdateFinalizedFeaturesRequest.java | 69 +++ .../UpdateFinalizedFeaturesResponse.java | 54 +++ .../UpdateFinalizedFeaturesRequest.json | 33 ++ .../UpdateFinalizedFeaturesResponse.json | 28 ++ .../clients/admin/KafkaAdminClientTest.java | 141 ++++++ .../kafka/clients/admin/MockAdminClient.java | 10 + .../kafka/controller/ControllerState.scala | 7 +- .../kafka/controller/KafkaController.scala | 223 ++++++++- .../scala/kafka/server/BrokerFeatures.scala | 180 +++++++ .../kafka/server/FinalizedFeatureCache.scala | 88 +++- .../FinalizedFeatureChangeListener.scala | 12 +- .../main/scala/kafka/server/KafkaApis.scala | 142 +++++- .../main/scala/kafka/server/KafkaServer.scala | 14 +- .../kafka/server/SupportedFeatures.scala | 93 ---- .../main/scala/kafka/zk/KafkaZkClient.scala | 3 +- core/src/main/scala/kafka/zk/ZkData.scala | 36 -- .../ControllerIntegrationTest.scala | 37 +- .../kafka/server/BrokerFeaturesTest.scala | 83 ++++ .../server/FinalizedFeatureCacheTest.scala | 66 +-- .../FinalizedFeatureChangeListenerTest.scala | 114 +++-- .../unit/kafka/server/KafkaApisTest.scala | 7 +- .../kafka/server/SupportedFeaturesTest.scala | 56 --- .../server/UpdateFinalizedFeaturesTest.scala | 450 ++++++++++++++++++ 39 files changed, 2175 insertions(+), 291 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/FinalizedFeatureUpdate.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/admin/UpdateFinalizedFeaturesOptions.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/admin/UpdateFinalizedFeaturesResult.java create mode 100644 clients/src/main/java/org/apache/kafka/common/errors/FinalizedFeatureUpdateFailedException.java create mode 100644 clients/src/main/java/org/apache/kafka/common/requests/UpdateFinalizedFeaturesRequest.java create mode 100644 clients/src/main/java/org/apache/kafka/common/requests/UpdateFinalizedFeaturesResponse.java create mode 100644 clients/src/main/resources/common/message/UpdateFinalizedFeaturesRequest.json create mode 100644 clients/src/main/resources/common/message/UpdateFinalizedFeaturesResponse.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/UpdateFinalizedFeaturesTest.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..091ea30910044 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 @@ -1306,6 +1306,70 @@ default AlterUserScramCredentialsResult alterUserScramCredentials(List alterations, AlterUserScramCredentialsOptions options); + /** + * Describes finalized as well as supported features. By default, the request is issued to any + * broker, but it can be optionally directed only to the controller via DescribeFeaturesOptions + * parameter. + *

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

+ *

+ * @param options the options to use + * + * @return the DescribeFeaturesResult containing the result + */ + DescribeFeaturesResult describeFeatures(DescribeFeaturesOptions options); + + /** + * Applies specified updates to finalized features. The API is atomic, meaning that if a single + * feature update in the request can't succeed on the controller, then none of the feature + * updates are carried out. This request is issued only to the controller since the API is + * only served by the controller. + *

+ * The API takes as input a set of FinalizedFeatureUpdate that need to be applied. Each such + * update specifies the finalized feature to be added or updated or deleted, along with the new + * max feature version level value. + *

+ *

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

+ *

+ * This operation is supported by brokers with version 2.7.0 or higher. + + * @param featureUpdates the set of finalized feature updates + * @param options the options to use + * + * @return the UpdateFinalizedFeaturesResult containing the result + */ + UpdateFinalizedFeaturesResult updateFinalizedFeatures( + Set featureUpdates, UpdateFinalizedFeaturesOptions 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..2a073f6008c86 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/admin/DescribeFeaturesOptions.java @@ -0,0 +1,35 @@ +package org.apache.kafka.clients.admin; + +import org.apache.kafka.common.annotation.InterfaceStability; + +@InterfaceStability.Evolving +public class DescribeFeaturesOptions extends AbstractOptions { + /** + * - True means the {@link Admin#describeFeatures(DescribeFeaturesOptions)} request can be + * issued only to the controller. + * - False means the {@link Admin#describeFeatures(DescribeFeaturesOptions)} request can be + * issued to any random broker. + */ + private boolean shouldSendRequestToController = false; + + /** + * Sets a flag indicating that the describe features request should be issued to the controller. + */ + public DescribeFeaturesOptions sendRequestToController(boolean shouldSendRequestToController) { + this.shouldSendRequestToController = shouldSendRequestToController; + return this; + } + + public boolean sendRequestToController() { + return shouldSendRequestToController; + } + + /** + * Sets the timeout in milliseconds for this operation or {@code null} if the default API + * timeout for the AdminClient should be used. + */ + public DescribeFeaturesOptions timeoutMs(Integer timeoutMs) { + this.timeoutMs = timeoutMs; + return this; + } +} 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..e9306b997342f --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/admin/DescribeFeaturesResult.java @@ -0,0 +1,16 @@ +package org.apache.kafka.clients.admin; + +import org.apache.kafka.common.KafkaFuture; + +public class DescribeFeaturesResult { + + private final KafkaFuture future; + + public 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..aaf4ab177731e --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/admin/FeatureMetadata.java @@ -0,0 +1,84 @@ +package org.apache.kafka.clients.admin; + +import java.util.Objects; +import org.apache.kafka.common.feature.Features; +import org.apache.kafka.common.feature.FinalizedVersionRange; +import org.apache.kafka.common.feature.SupportedVersionRange; + +/** + * 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 Features finalizedFeatures; + + private final int finalizedFeaturesEpoch; + + private final Features supportedFeatures; + + public FeatureMetadata( + final Features finalizedFeatures, + final int finalizedFeaturesEpoch, + final Features supportedFeatures + ) { + Objects.requireNonNull(finalizedFeatures, "Provided finalizedFeatures can not be null."); + Objects.requireNonNull(supportedFeatures, "Provided supportedFeatures can not be null."); + this.finalizedFeatures = finalizedFeatures; + this.finalizedFeaturesEpoch = finalizedFeaturesEpoch; + this.supportedFeatures = supportedFeatures; + } + + /** + * A map of finalized feature versions, with key being finalized feature name and value + * containing the min/max version levels for the finalized feature. + */ + public Features finalizedFeatures() { + return finalizedFeatures; + } + + /** + * The epoch for the finalized features. + * Valid values are >= 0. A value < 0 means the finalized features are absent/unavailable. + */ + public int finalizedFeaturesEpoch() { + return finalizedFeaturesEpoch; + } + + /** + * A map of supported feature versions, with key being supported feature name and value + * containing the min/max version for the supported feature. + */ + public Features supportedFeatures() { + return 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); + } + + @Override + public String toString() { + return String.format( + "FeatureMetadata{finalized:%s, finalizedFeaturesEpoch:%d, supported:%s}", + finalizedFeatures, + finalizedFeaturesEpoch, + supportedFeatures); + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/FinalizedFeatureUpdate.java b/clients/src/main/java/org/apache/kafka/clients/admin/FinalizedFeatureUpdate.java new file mode 100644 index 0000000000000..76c74071c3181 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/admin/FinalizedFeatureUpdate.java @@ -0,0 +1,87 @@ +package org.apache.kafka.clients.admin; + +import java.util.Objects; +import java.util.Set; +import org.apache.kafka.common.message.UpdateFinalizedFeaturesRequestData; + +/** + * Encapsulates details about an update to a finalized feature. This is particularly useful to + * define each feature update in the + * {@link Admin#updateFinalizedFeatures(Set, UpdateFinalizedFeaturesOptions)} API request. + */ +public class FinalizedFeatureUpdate { + private final String featureName; + private final short maxVersionLevel; + private final boolean allowDowngrade; + + /** + * @param featureName the name of the finalized feature to be updated. + * @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 FinalizedFeatureUpdate( + final String featureName, final short maxVersionLevel, final boolean allowDowngrade) { + Objects.requireNonNull(featureName, "Provided feature name can not be null."); + if (maxVersionLevel < 1 && !allowDowngrade) { + throw new IllegalArgumentException( + String.format( + "For featureName: %s, the allowDowngrade flag is not set when the" + + " provided maxVersionLevel:%d is < 1.", featureName, maxVersionLevel)); + } + this.featureName = featureName; + this.maxVersionLevel = maxVersionLevel; + this.allowDowngrade = allowDowngrade; + } + + /** + * @return the name of the finalized feature to be updated. + */ + public String featureName() { + return featureName; + } + + /** + * @return the new maximum version level for the finalized feature. + */ + public short maxVersionLevel() { + return maxVersionLevel; + } + + /** + * @return - true, if this feature update was meant to downgrade the maximum version level of + * the finalized feature. + * - false, otherwise. + */ + public boolean allowDowngrade() { + return allowDowngrade; + } + + /** + * Helper function that creates {@link UpdateFinalizedFeaturesRequestData} from a set of + * {@link FinalizedFeatureUpdate}. + * + * @param updates the set of {@link FinalizedFeatureUpdate} + * + * @return a newly constructed UpdateFinalizedFeaturesRequestData object + */ + public static UpdateFinalizedFeaturesRequestData createRequest(Set updates) { + final UpdateFinalizedFeaturesRequestData.FinalizedFeatureUpdateKeyCollection items + = new UpdateFinalizedFeaturesRequestData.FinalizedFeatureUpdateKeyCollection(); + for (FinalizedFeatureUpdate update : updates) { + final UpdateFinalizedFeaturesRequestData.FinalizedFeatureUpdateKey item = + new UpdateFinalizedFeaturesRequestData.FinalizedFeatureUpdateKey(); + item.setName(update.featureName()); + item.setMaxVersionLevel(update.maxVersionLevel()); + item.setAllowDowngrade(update.allowDowngrade()); + items.add(item); + } + final UpdateFinalizedFeaturesRequestData data = new UpdateFinalizedFeaturesRequestData(); + data.setFinalizedFeatureUpdates(items); + return data; + } +} 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..4c0bcabf9e9dd 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 @@ -169,6 +169,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 +228,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.UpdateFinalizedFeaturesRequest; +import org.apache.kafka.common.requests.UpdateFinalizedFeaturesResponse; 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 +4339,106 @@ 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(); + Call callViaLeastLoadedNode = new Call("describeFeatures", calcDeadlineMs(now, options.timeoutMs()), + new LeastLoadedNodeProvider()) { + + @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( + new FeatureMetadata( + apiVersionsResponse.finalizedFeatures(), + apiVersionsResponse.finalizedFeaturesEpoch(), + apiVersionsResponse.supportedFeatures())); + } else { + future.completeExceptionally( + Errors.forCode(apiVersionsResponse.data.errorCode()).exception()); + } + } + + @Override + void handleFailure(Throwable throwable) { + completeAllExceptionally(Collections.singletonList(future), throwable); + } + }; + + Call call = callViaLeastLoadedNode; + if (options.sendRequestToController()) { + call = new Call("describeFeatures", calcDeadlineMs(now, options.timeoutMs()), + new ControllerNodeProvider()) { + + @Override + ApiVersionsRequest.Builder createRequest(int timeoutMs) { + return (ApiVersionsRequest.Builder) callViaLeastLoadedNode.createRequest(timeoutMs); + } + + @Override + void handleResponse(AbstractResponse response) { + final ApiVersionsResponse apiVersionsResponse = (ApiVersionsResponse) response; + if (apiVersionsResponse.data.errorCode() == Errors.NOT_CONTROLLER.code()) { + handleNotControllerError(Errors.NOT_CONTROLLER); + } else { + callViaLeastLoadedNode.handleResponse(response); + } + } + + @Override + void handleFailure(Throwable throwable) { + callViaLeastLoadedNode.handleFailure(throwable); + } + }; + } + runnable.call(call, now); + return new DescribeFeaturesResult(future); + } + + @Override + public UpdateFinalizedFeaturesResult updateFinalizedFeatures( + final Set featureUpdates, final UpdateFinalizedFeaturesOptions options) { + final KafkaFutureImpl future = new KafkaFutureImpl<>(); + final long now = time.milliseconds(); + + final Call call = new Call("updateFinalizedFeatures", calcDeadlineMs(now, options.timeoutMs()), + new ControllerNodeProvider()) { + + @Override + UpdateFinalizedFeaturesRequest.Builder createRequest(int timeoutMs) { + return new UpdateFinalizedFeaturesRequest.Builder(FinalizedFeatureUpdate.createRequest(featureUpdates)); + } + + @Override + void handleResponse(AbstractResponse response) { + final UpdateFinalizedFeaturesResponse featuresResponse = + (UpdateFinalizedFeaturesResponse) response; + final Errors error = Errors.forCode(featuresResponse.data().errorCode()); + if (error == Errors.NONE) { + future.complete(null); + } else if (error == Errors.NOT_CONTROLLER) { + handleNotControllerError(error); + } else { + future.completeExceptionally( + error.exception(featuresResponse.data.errorMessage())); + } + } + + @Override + void handleFailure(Throwable throwable) { + completeAllExceptionally(Collections.singletonList(future), throwable); + } + }; + runnable.call(call, now); + return new UpdateFinalizedFeaturesResult(future); + } + /** * 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/UpdateFinalizedFeaturesOptions.java b/clients/src/main/java/org/apache/kafka/clients/admin/UpdateFinalizedFeaturesOptions.java new file mode 100644 index 0000000000000..1a7ed8d538ced --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/admin/UpdateFinalizedFeaturesOptions.java @@ -0,0 +1,15 @@ +package org.apache.kafka.clients.admin; + +import org.apache.kafka.common.annotation.InterfaceStability; + +@InterfaceStability.Evolving +public class UpdateFinalizedFeaturesOptions extends AbstractOptions { + /** + * Sets the timeout in milliseconds for this operation or {@code null} if the default API + * timeout for the AdminClient should be used. + */ + public UpdateFinalizedFeaturesOptions timeoutMs(Integer timeoutMs) { + this.timeoutMs = timeoutMs; + return this; + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/UpdateFinalizedFeaturesResult.java b/clients/src/main/java/org/apache/kafka/clients/admin/UpdateFinalizedFeaturesResult.java new file mode 100644 index 0000000000000..2516d200ac908 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/admin/UpdateFinalizedFeaturesResult.java @@ -0,0 +1,15 @@ +package org.apache.kafka.clients.admin; + +import org.apache.kafka.common.KafkaFuture; + +public class UpdateFinalizedFeaturesResult { + private final KafkaFuture future; + + public UpdateFinalizedFeaturesResult(KafkaFuture future) { + this.future = future; + } + + public KafkaFuture result() { + return future; + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/errors/FinalizedFeatureUpdateFailedException.java b/clients/src/main/java/org/apache/kafka/common/errors/FinalizedFeatureUpdateFailedException.java new file mode 100644 index 0000000000000..ae53fd271156b --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/errors/FinalizedFeatureUpdateFailedException.java @@ -0,0 +1,13 @@ +package org.apache.kafka.common.errors; + +public class FinalizedFeatureUpdateFailedException extends ApiException { + private static final long serialVersionUID = 1L; + + public FinalizedFeatureUpdateFailedException(String message) { + super(message); + } + + public FinalizedFeatureUpdateFailedException(String message, Throwable cause) { + super(message, cause); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/feature/Features.java b/clients/src/main/java/org/apache/kafka/common/feature/Features.java index 4006d71947fb7..1b842f2e48e9a 100644 --- a/clients/src/main/java/org/apache/kafka/common/feature/Features.java +++ b/clients/src/main/java/org/apache/kafka/common/feature/Features.java @@ -93,6 +93,15 @@ public VersionRangeType get(String feature) { return features.get(feature); } + /** + * @param feature the feature to be checked + * @return - true, if the feature is present + * - false, otherwise. + */ + public boolean has(String feature) { + return get(feature) != null; + } + public String toString() { return String.format( "Features{%s}", 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..35c08f19b520d 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 @@ -50,4 +50,8 @@ public static FinalizedVersionRange fromMap(Map versionRangeMap) public boolean isIncompatibleWith(SupportedVersionRange supportedVersionRange) { return min() < supportedVersionRange.min() || max() > supportedVersionRange.max(); } + + public boolean isCompatibleWith(SupportedVersionRange supportedVersionRange) { + return !isIncompatibleWith(supportedVersionRange); + } } 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..37ac45a1baa7b 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.UpdateFinalizedFeaturesRequestData; +import org.apache.kafka.common.message.UpdateFinalizedFeaturesResponseData; import org.apache.kafka.common.message.UpdateMetadataRequestData; import org.apache.kafka.common.message.UpdateMetadataResponseData; import org.apache.kafka.common.message.VoteRequestData; @@ -248,6 +250,9 @@ public Struct parseResponse(short version, ByteBuffer buffer) { DESCRIBE_QUORUM(55, "DescribeQuorum", true, false, DescribeQuorumRequestData.SCHEMAS, DescribeQuorumResponseData.SCHEMAS), ALTER_ISR(56, "AlterIsr", AlterIsrRequestData.SCHEMAS, AlterIsrResponseData.SCHEMAS); + DescribeQuorumRequestData.SCHEMAS, DescribeQuorumResponseData.SCHEMAS), + UPDATE_FINALIZED_FEATURES(57, "UpdateFinalizedFeatures", + UpdateFinalizedFeaturesRequestData.SCHEMAS, UpdateFinalizedFeaturesResponseData.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..36a84d41a2405 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 @@ -37,6 +37,7 @@ import org.apache.kafka.common.errors.FencedInstanceIdException; import org.apache.kafka.common.errors.FencedLeaderEpochException; import org.apache.kafka.common.errors.InvalidUpdateVersionException; +import org.apache.kafka.common.errors.FinalizedFeatureUpdateFailedException; import org.apache.kafka.common.errors.FetchSessionIdNotFoundException; import org.apache.kafka.common.errors.GroupAuthorizationException; import org.apache.kafka.common.errors.GroupIdNotFoundException; @@ -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), + FINALIZED_FEATURE_UPDATE_FAILED(96, "Unable to update finalized features.", FinalizedFeatureUpdateFailedException::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..d29fe46da80f4 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_FINALIZED_FEATURES: + return new UpdateFinalizedFeaturesRequest(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..15d724e521b98 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_FINALIZED_FEATURES: + return new UpdateFinalizedFeaturesResponse(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..a357e9ad78978 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 @@ -16,6 +16,7 @@ */ package org.apache.kafka.common.requests; +import java.util.HashMap; import org.apache.kafka.common.feature.Features; import org.apache.kafka.common.feature.FinalizedVersionRange; import org.apache.kafka.common.feature.SupportedVersionRange; @@ -67,6 +68,36 @@ public ApiVersionsResponse(Struct struct, short version) { this(new ApiVersionsResponseData(struct, version)); } + public ApiVersionsResponseData data() { + return data; + } + + public Features supportedFeatures() { + final Map features = new HashMap<>(); + + for (SupportedFeatureKey key : data.supportedFeatures().valuesSet()) { + features.put(key.name(), new SupportedVersionRange(key.minVersion(), key.maxVersion())); + } + + return Features.supportedFeatures(features); + } + + public Features finalizedFeatures() { + final Map features = new HashMap<>(); + + for (FinalizedFeatureKey key : data.finalizedFeatures().valuesSet()) { + features.put( + key.name(), + new FinalizedVersionRange(key.minVersionLevel(), key.maxVersionLevel())); + } + + return Features.finalizedFeatures(features); + } + + public int finalizedFeaturesEpoch() { + return data.finalizedFeaturesEpoch(); + } + @Override protected Struct toStruct(short version) { return this.data.toStruct(version); @@ -142,7 +173,13 @@ public static ApiVersionsResponse apiVersionsResponse( Features finalizedFeatures, int 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); @@ -176,15 +213,33 @@ public static ApiVersionsResponse createApiVersionsResponse( } } + 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 int finalizedFeaturesEpoch + ) { 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( diff --git a/clients/src/main/java/org/apache/kafka/common/requests/UpdateFinalizedFeaturesRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/UpdateFinalizedFeaturesRequest.java new file mode 100644 index 0000000000000..918575b786072 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/UpdateFinalizedFeaturesRequest.java @@ -0,0 +1,69 @@ +package org.apache.kafka.common.requests; + +import java.nio.ByteBuffer; +import org.apache.kafka.common.message.UpdateFinalizedFeaturesResponseData; +import org.apache.kafka.common.message.UpdateFinalizedFeaturesRequestData; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.types.Struct; + +public class UpdateFinalizedFeaturesRequest extends AbstractRequest { + + public static class Builder extends AbstractRequest.Builder { + + private final UpdateFinalizedFeaturesRequestData data; + + public Builder(UpdateFinalizedFeaturesRequestData data) { + super(ApiKeys.UPDATE_FINALIZED_FEATURES); + this.data = data; + } + + @Override + public UpdateFinalizedFeaturesRequest build(short version) { + return new UpdateFinalizedFeaturesRequest(data, version); + } + + @Override + public String toString() { + return data.toString(); + } + } + + public final UpdateFinalizedFeaturesRequestData data; + + public UpdateFinalizedFeaturesRequest(UpdateFinalizedFeaturesRequestData data, short version) { + super(ApiKeys.UPDATE_FINALIZED_FEATURES, version); + this.data = data; + } + + public UpdateFinalizedFeaturesRequest(Struct struct, short version) { + super(ApiKeys.UPDATE_FINALIZED_FEATURES, version); + this.data = new UpdateFinalizedFeaturesRequestData(struct, version); + } + + @Override + public AbstractResponse getErrorResponse(int throttleTimeMsIgnored, Throwable e) { + final ApiError apiError = ApiError.fromThrowable(e); + return new UpdateFinalizedFeaturesResponse( + new UpdateFinalizedFeaturesResponseData() + .setErrorCode(apiError.error().code()) + .setErrorMessage(apiError.message())); + } + + @Override + protected Struct toStruct() { + return data.toStruct(version()); + } + + public UpdateFinalizedFeaturesRequestData data() { + return data; + } + + public static UpdateFinalizedFeaturesRequest parse(ByteBuffer buffer, short version) { + return new UpdateFinalizedFeaturesRequest( + ApiKeys.UPDATE_FINALIZED_FEATURES.parseRequest(version, buffer), version); + } + + public static boolean isDeleteRequest(UpdateFinalizedFeaturesRequestData.FinalizedFeatureUpdateKey update) { + return update.maxVersionLevel() < 1; + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/UpdateFinalizedFeaturesResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/UpdateFinalizedFeaturesResponse.java new file mode 100644 index 0000000000000..e52e67eedf869 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/UpdateFinalizedFeaturesResponse.java @@ -0,0 +1,54 @@ +package org.apache.kafka.common.requests; + +import java.nio.ByteBuffer; +import java.util.Map; +import org.apache.kafka.common.message.AlterPartitionReassignmentsResponseData; +import org.apache.kafka.common.message.UpdateFinalizedFeaturesResponseData; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.types.Struct; + +public class UpdateFinalizedFeaturesResponse extends AbstractResponse { + + public final UpdateFinalizedFeaturesResponseData data; + + public UpdateFinalizedFeaturesResponse(UpdateFinalizedFeaturesResponseData data) { + this.data = data; + } + + public UpdateFinalizedFeaturesResponse(Struct struct) { + final short latestVersion = (short) (UpdateFinalizedFeaturesResponseData.SCHEMAS.length - 1); + this.data = new UpdateFinalizedFeaturesResponseData(struct, latestVersion); + } + + public UpdateFinalizedFeaturesResponse(Struct struct, short version) { + this.data = new UpdateFinalizedFeaturesResponseData(struct, version); + } + + public Errors error() { + return Errors.forCode(data.errorCode()); + } + + @Override + public Map errorCounts() { + return errorCounts(Errors.forCode(data.errorCode())); + } + + @Override + protected Struct toStruct(short version) { + return data.toStruct(version); + } + + @Override + public String toString() { + return data.toString(); + } + + public UpdateFinalizedFeaturesResponseData data() { + return data; + } + + public static UpdateFinalizedFeaturesResponse parse(ByteBuffer buffer, short version) { + return new UpdateFinalizedFeaturesResponse(ApiKeys.UPDATE_FINALIZED_FEATURES.parseResponse(version, buffer), version); + } +} diff --git a/clients/src/main/resources/common/message/UpdateFinalizedFeaturesRequest.json b/clients/src/main/resources/common/message/UpdateFinalizedFeaturesRequest.json new file mode 100644 index 0000000000000..c7ed42642e6a0 --- /dev/null +++ b/clients/src/main/resources/common/message/UpdateFinalizedFeaturesRequest.json @@ -0,0 +1,33 @@ +// 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": 50, + "type": "request", + "name": "UpdateFinalizedFeaturesRequest", + "validVersions": "0", + "flexibleVersions": "0+", + "fields": [ + { "name": "FinalizedFeatureUpdates", "type": "[]FinalizedFeatureUpdateKey", "versions": "0+", + "about": "The list of updates to finalized features.", "fields": [ + {"name": "Name", "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."} + ]} + ] +} diff --git a/clients/src/main/resources/common/message/UpdateFinalizedFeaturesResponse.json b/clients/src/main/resources/common/message/UpdateFinalizedFeaturesResponse.json new file mode 100644 index 0000000000000..a018d03cb7076 --- /dev/null +++ b/clients/src/main/resources/common/message/UpdateFinalizedFeaturesResponse.json @@ -0,0 +1,28 @@ +// 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": 50, + "type": "response", + "name": "UpdateFinalizedFeaturesResponse", + "validVersions": "0", + "flexibleVersions": "0+", + "fields": [ + { "name": "ErrorCode", "type": "int16", "versions": "0+", + "about": "The error code or `0` if there was no error." }, + { "name": "ErrorMessage", "type": "string", "versions": "0+", "nullableVersions": "0+", + "about": "The error message or `null` if there was no error." } + ] +} 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..cb698811b4290 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,15 @@ 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.feature.FinalizedVersionRange; +import org.apache.kafka.common.feature.SupportedVersionRange; 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; @@ -111,6 +115,7 @@ import org.apache.kafka.common.message.OffsetDeleteResponseData.OffsetDeleteResponsePartitionCollection; import org.apache.kafka.common.message.OffsetDeleteResponseData.OffsetDeleteResponseTopic; import org.apache.kafka.common.message.OffsetDeleteResponseData.OffsetDeleteResponseTopicCollection; +import org.apache.kafka.common.message.UpdateFinalizedFeaturesResponseData; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.quota.ClientQuotaAlteration; @@ -122,6 +127,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 +159,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.UpdateFinalizedFeaturesRequest; +import org.apache.kafka.common.requests.UpdateFinalizedFeaturesResponse; import org.apache.kafka.common.resource.PatternType; import org.apache.kafka.common.resource.ResourcePattern; import org.apache.kafka.common.resource.ResourcePatternFilter; @@ -474,6 +483,40 @@ private static DescribeGroupsResponseData prepareDescribeGroupsResponseData(Stri Collections.emptySet())); return data; } + + private static UpdateFinalizedFeaturesResponse prepareUpdateFinalizedFeaturesResponse(Errors error) { + final UpdateFinalizedFeaturesResponseData data = new UpdateFinalizedFeaturesResponseData(); + data.setErrorCode(error.code()); + return new UpdateFinalizedFeaturesResponse(data); + } + + private static FeatureMetadata getDefaultFeatureMetadata() { + return new FeatureMetadata( + Features.finalizedFeatures(new HashMap() {{ + put("test_feature_1", new FinalizedVersionRange((short) 2, (short) 3)); + }}), + 1, + Features.supportedFeatures(new HashMap() {{ + put("test_feature_1", new SupportedVersionRange((short) 1, (short) 5)); + }}) + ); + } + + 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(), + getDefaultFeatureMetadata().supportedFeatures(), + getDefaultFeatureMetadata().finalizedFeatures(), + getDefaultFeatureMetadata().finalizedFeaturesEpoch())); + } + final ApiVersionsResponseData data = new ApiVersionsResponseData(); + data.setErrorCode(error.code()); + return new ApiVersionsResponse(data); + } + /** * Test that the client properly times out when we don't receive any metadata. */ @@ -3884,6 +3927,104 @@ public void testListOffsetsNonRetriableErrors() throws Exception { } } + @Test + public void testUpdateFinalizedFeaturesDuringSuccess() throws Exception { + testUpdateFinalizedFeaturesDuringError(Errors.NONE); + } + + @Test + public void testUpdateFinalizedFeaturesInvalidRequestError() throws Exception { + testUpdateFinalizedFeaturesDuringError(Errors.INVALID_REQUEST); + } + + @Test + public void testUpdateFinalizedFeaturesUpdateFailedError() throws Exception { + testUpdateFinalizedFeaturesDuringError(Errors.FINALIZED_FEATURE_UPDATE_FAILED); + } + + private void testUpdateFinalizedFeaturesDuringError(Errors error) throws Exception { + try (final AdminClientUnitTestEnv env = mockClientEnv()) { + env.kafkaClient().prepareResponse( + body -> body instanceof UpdateFinalizedFeaturesRequest, + prepareUpdateFinalizedFeaturesResponse(error)); + final KafkaFuture future = env.adminClient().updateFinalizedFeatures( + new HashSet<>( + Arrays.asList( + new FinalizedFeatureUpdate( + "test_feature_1", (short) 2, false), + new FinalizedFeatureUpdate( + "test_feature_2", (short) 3, true))), + new UpdateFinalizedFeaturesOptions().timeoutMs(10000)).result(); + if (error.exception() == null) { + future.get(); + } else { + final ExecutionException e = assertThrows(ExecutionException.class, + () -> future.get()); + assertEquals(e.getCause().getClass(), error.exception().getClass()); + } + } + } + + @Test + public void testUpdateFinalizedFeaturesHandleNotControllerException() throws Exception { + try (final AdminClientUnitTestEnv env = mockClientEnv()) { + env.kafkaClient().prepareResponseFrom( + prepareUpdateFinalizedFeaturesResponse(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( + prepareUpdateFinalizedFeaturesResponse(Errors.NONE), + env.cluster().nodeById(1)); + final KafkaFuture future = env.adminClient().updateFinalizedFeatures( + new HashSet<>( + Arrays.asList( + new FinalizedFeatureUpdate( + "test_feature_1",(short) 2, false), + new FinalizedFeatureUpdate( + "test_feature_2", (short) 3, true))), + new UpdateFinalizedFeaturesOptions().timeoutMs(10000)).result(); + future.get(); + } + } + + @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(); + FeatureMetadata metadata = future.get(); + assertEquals(getDefaultFeatureMetadata(), 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..0582a15c7204a 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, UpdateFinalizedFeaturesOptions 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..8412fa375c8e9 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 UpdateFinalizedFeatures 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, + UpdateFinalizedFeatures) } diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 583d5368fcd68..dce47502e0f9f 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -23,18 +23,21 @@ import kafka.admin.AdminOperationException import kafka.api._ import kafka.common._ import kafka.controller.KafkaController.{AlterIsrCallback, AlterReassignmentsCallback, ElectLeadersCallback, ListReassignmentsCallback} +import kafka.cluster.Broker +import kafka.controller.KafkaController.{AlterReassignmentsCallback, ElectLeadersCallback, ListReassignmentsCallback, UpdateFinalizedFeaturesCallback} 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.metrics.Metrics import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{AbstractControlRequest, ApiError, LeaderAndIsrResponse, UpdateMetadataResponse} @@ -60,6 +63,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 UpdateFinalizedFeaturesCallback = (Errors, Option[String]) => Unit } class KafkaController(val config: KafkaConfig, @@ -69,6 +73,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 +225,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 = { + setupFeatureVersioning() + info("Registering handlers") // before reading source of truth from zookeeper, register the listeners to get broker/topic callbacks @@ -272,6 +280,156 @@ 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) + } + + /** + * Enables the feature versioning system (KIP-584). + * + * Sets up the FeatureZNode with enabled status. This status means the feature versioning system + * (KIP-584) is enabled, and, the finalized features stored in the FeatureZNode are active. This + * 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: + * 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. Assuming this is the + * case, then here is how we it: 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 default supported features as its finalized features. + * + * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0: + * Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the + * Broker binary has been upgraded to a newer version that 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 + * 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. The reason to do this is that enabling all the possible features immediately after + * an upgrade could be harmful to the cluster. + * 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 absent, then 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. 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. Broker binary upgraded, with existing cluster IBP config >= KAFKA_2_7_IV0: + * Imagine 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 needs to scan the + * existing finalized features and mutate them for the purpose of version level deprecation + * (if needed). + * This is how we handle this case: If an existing finalized feature is present in the default + * finalized features, then, it's existing minimum version level is updated to the default + * minimum version level maintained in the BrokerFeatures object. The goal of this mutation is + * to permanently deprecate one or more feature version levels. The range of feature version + * levels deprecated are from the closed range: [existing_min_version_level, default_min_version_level]. + * NOTE: Deprecating a feature version level is an incompatible change, which requires a major + * release of Kafka. In such a release, the minimum version level maintained within the + * BrokerFeatures class is updated suitably to record the deprecation of the feature. + * + * 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 defaultFinalizedFeatures = brokerFeatures.getDefaultFinalizedFeatures + val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path) + if (version == ZkVersion.UnknownVersion) { + val newVersion = createFeatureZNode(new FeatureZNode(FeatureZNodeStatus.Enabled, defaultFinalizedFeatures)) + featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs) + } else { + val existingFeatureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get) + var newFeatures: Features[FinalizedVersionRange] = Features.emptyFinalizedFeatures() + if (existingFeatureZNode.status.equals(FeatureZNodeStatus.Enabled)) { + newFeatures = Features.finalizedFeatures(existingFeatureZNode.features.features().asScala.map { + case (featureName, existingVersionRange) => { + val updatedVersionRange = defaultFinalizedFeatures.get(featureName) + if (updatedVersionRange == null) { + warn(s"Existing finalized feature: $featureName with $existingVersionRange" + + s" is absent in default finalized $defaultFinalizedFeatures") + (featureName, existingVersionRange) + } else if (existingVersionRange.max() >= updatedVersionRange.min()) { + // Through this change, we deprecate all version levels in the closed range: + // [existingVersionRange.min(), updatedVersionRange.min() - 1] + (featureName, new FinalizedVersionRange(updatedVersionRange.min(), existingVersionRange.max())) + } else { + // This is a special case: If the existing version levels fall completely outside the + // range of the default finalized version levels (i.e. no intersection), then, this + // case is not eligible for deprecation. This requires that the max version level be + // upgraded first to a value that's equal to the the default minimum version level. + info(s"Can not update minimum version level in finalized feature: $featureName," + + s" since the existing $existingVersionRange does not intersect with the default" + + s" $updatedVersionRange.") + (featureName, existingVersionRange) + } + } + }.asJava) + } + 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: when this method returns, existing finalized features (if any) will be cleared from the + * FeatureZNode. + */ + private def disableFeatureVersioning(): Unit = { + val newNode = FeatureZNode(FeatureZNodeStatus.Disabled, Features.emptyFinalizedFeatures()) + val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path) + if (version == ZkVersion.UnknownVersion) { + createFeatureZNode(newNode) + featureCache.waitUntilEmptyOrThrow(config.zkConnectionTimeoutMs) + } else { + val existingFeatureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get) + if (!existingFeatureZNode.status.equals(FeatureZNodeStatus.Disabled)) { + updateFeatureZNode(newNode) + featureCache.waitUntilEmptyOrThrow(config.zkConnectionTimeoutMs) + } + } + } + + private def setupFeatureVersioning(): Unit = { + if (config.isFeatureVersioningEnabled) { + enableFeatureVersioning() + } else { + disableFeatureVersioning() + } + } + private def scheduleAutoLeaderRebalanceTask(delay: Long, unit: TimeUnit): Unit = { kafkaScheduler.schedule("auto-leader-rebalance-task", () => eventManager.put(AutoPreferredReplicaLeaderElection), delay = delay, unit = unit) @@ -996,8 +1154,25 @@ class KafkaController(val config: KafkaConfig, */ private[controller] def sendUpdateMetadataRequest(brokers: Seq[Int], partitions: Set[TopicPartition]): Unit = { try { + val filteredBrokers = scala.collection.mutable.Set[Int]() ++ brokers + if (config.isFeatureVersioningEnabled) { + def hasIncompatibleFeatures(broker: Broker): Boolean = { + val latestFinalizedFeatures = featureCache.get + if (latestFinalizedFeatures.isDefined) { + BrokerFeatures.hasIncompatibleFeatures(broker.features, latestFinalizedFeatures.get.features) + } else { + false + } + } + controllerContext.liveOrShuttingDownBrokers.foreach(broker => { + if (filteredBrokers.contains(broker.id) && hasIncompatibleFeatures(broker)) { + warn(s"Ignoring UpdateMetadataRequest to broker: ${broker.id} due to incompatible features") + filteredBrokers -= broker.id + } + }) + } brokerRequestBatch.newBatch() - brokerRequestBatch.addUpdateMetadataRequestForBrokers(brokers, partitions) + brokerRequestBatch.addUpdateMetadataRequestForBrokers(filteredBrokers.toSeq, partitions) brokerRequestBatch.sendRequestsToBrokers(epoch) } catch { case e: IllegalStateException => @@ -1404,7 +1579,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 +1831,36 @@ class KafkaController(val config: KafkaConfig, } } + private def processUpdateFinalizedFeatures(newFeatures: Features[FinalizedVersionRange], + callback: UpdateFinalizedFeaturesCallback): Unit = { + if (isActive) { + val incompatibleBrokers = controllerContext.liveOrShuttingDownBrokers.filter(broker => { + BrokerFeatures.hasIncompatibleFeatures(broker.features, newFeatures) + }) + if (incompatibleBrokers.size > 0) { + callback( + Errors.INVALID_REQUEST, + Some( + s"Could not apply finalized feature updates because ${incompatibleBrokers.size} brokers" + + s" were found to have incompatible features. newFeatures: $newFeatures" + + s", incompatibleBrokers: $incompatibleBrokers.")) + } else { + try { + val newVersion = zkClient.updateFeatureZNode( + new FeatureZNode(FeatureZNodeStatus.Enabled, newFeatures)) + featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs) + callback(Errors.NONE, Option.empty) + } catch { + case e: Exception => callback( + Errors.FINALIZED_FEATURE_UPDATE_FAILED, + Some(Errors.FINALIZED_FEATURE_UPDATE_FAILED.message() + " Error: " + e)) + } + } + } else { + callback(Errors.NOT_CONTROLLER, Option.empty) + } + } + private def processIsrChangeNotification(): Unit = { def processUpdateNotifications(partitions: Seq[TopicPartition]): Unit = { val liveBrokers: Seq[Int] = controllerContext.liveOrShuttingDownBrokerIds.toSeq @@ -1690,6 +1895,11 @@ class KafkaController(val config: KafkaConfig, eventManager.put(ListPartitionReassignments(partitions, callback)) } + def updateFinalizedFeatures(newFeatures: Features[FinalizedVersionRange], + callback: UpdateFinalizedFeaturesCallback): Unit = { + eventManager.put(UpdateFinalizedFeatures(newFeatures, callback)) + } + def alterPartitionReassignments(partitions: Map[TopicPartition, Option[Seq[Int]]], callback: AlterReassignmentsCallback): Unit = { eventManager.put(ApiPartitionReassignment(partitions, callback)) @@ -1979,6 +2189,8 @@ class KafkaController(val config: KafkaConfig, processZkPartitionReassignment() case ListPartitionReassignments(partitions, callback) => processListPartitionReassignments(partitions, callback) + case UpdateFinalizedFeatures(request, callback) => + processUpdateFinalizedFeatures(request, callback) case PartitionReassignmentIsrChange(partition) => processPartitionReassignmentIsrChange(partition) case IsrChangeNotification => @@ -2272,6 +2484,11 @@ case class ListPartitionReassignments(partitionsOpt: Option[Set[TopicPartition]] override def preempt(): Unit = callback(Right(new ApiError(Errors.NOT_CONTROLLER, null))) } +case class UpdateFinalizedFeatures(newFeatures: Features[FinalizedVersionRange], + callback: UpdateFinalizedFeaturesCallback) extends ControllerEvent { + override def state: ControllerState = ControllerState.UpdateFinalizedFeatures +} + // 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..ab39f1fb8665f --- /dev/null +++ b/core/src/main/scala/kafka/server/BrokerFeatures.scala @@ -0,0 +1,180 @@ +/** + * 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 following: + * + * 1. The latest features supported by the Broker. + * + * 2. The default minimum version levels for specific features. This map enables feature + * version level deprecation. This is how it works: in order to deprecate feature version levels, + * in this map the default minimum version level of a feature can be set to a new value that's + * higher than 1 (let's call this new_min_version_level). In doing so, the feature version levels + * in the closed range: [1, latest_min_version_level - 1] get deprecated by the controller logic + * that applies this map to persistent finalized feature state in ZK (this mutation happens + * during controller and during finalized feature updates via the + * APIKeys.UPDATE_FINALIZED_FEATURES api). + * + * This class also provides APIs to check for incompatibilities between the features supported by + * the Broker and finalized features. + * + * NOTE: the update*() and clear*() APIs of this class should be used only for testing purposes. + */ +class BrokerFeatures private (@volatile var supportedFeatures: Features[SupportedVersionRange], + @volatile var defaultFeatureMinVersionLevels: Map[String, Short]) { + require(BrokerFeatures.areFeatureMinVersionLevelsCompatible( + supportedFeatures, defaultFeatureMinVersionLevels)) + + // For testing only. + def setSupportedFeatures(newFeatures: Features[SupportedVersionRange]): Unit = { + require( + BrokerFeatures.areFeatureMinVersionLevelsCompatible(newFeatures, defaultFeatureMinVersionLevels)) + supportedFeatures = newFeatures + } + + /** + * Returns the default minimum version level for a specific feature. + * + * @param feature the name of the feature + * + * @return the default minimum version level for the feature if its defined. + * otherwise, returns 1. + */ + def defaultMinVersionLevel(feature: String): Short = { + defaultFeatureMinVersionLevels.getOrElse(feature, 1) + } + + // For testing only. + def setDefaultMinVersionLevels(newMinVersionLevels: Map[String, Short]): Unit = { + require( + BrokerFeatures.areFeatureMinVersionLevelsCompatible(supportedFeatures, newMinVersionLevels)) + defaultFeatureMinVersionLevels = newMinVersionLevels + } + + /** + * Returns the default finalized features that a new Kafka cluster with IBP config >= KAFKA_2_7_IV0 + * needs to be bootstrapped with. + */ + def getDefaultFinalizedFeatures: Features[FinalizedVersionRange] = { + Features.finalizedFeatures( + supportedFeatures.features.asScala.map { + case(name, versionRange) => ( + name, new FinalizedVersionRange(defaultMinVersionLevel(name), 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 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] = { + BrokerFeatures.incompatibleFeatures(supportedFeatures, finalized, 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, Map[String, Short]()) + } + + /** + * 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, false).empty + } + + private def incompatibleFeatures(supportedFeatures: Features[SupportedVersionRange], + finalizedFeatures: Features[FinalizedVersionRange], + logIncompatibilities: Boolean): Features[FinalizedVersionRange] = { + val incompatibilities = 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 && incompatibilities.nonEmpty) { + warn( + "Feature incompatibilities seen: " + incompatibilities.map{ + case(_, _, errorReason) => errorReason }) + } + Features.finalizedFeatures(incompatibilities.map{ + case(feature, versionLevels, _) => (feature, versionLevels) }.toMap.asJava) + } + + /** + * A check that ensures each feature defined with min version level is a supported feature, and + * the min version level value is valid (i.e. it is compatible with the supported version range). + * + * @param supportedFeatures the supported features + * @param featureMinVersionLevels the feature minimum version levels + * + * @return - true, if the above described check passes. + * - false, otherwise. + */ + private def areFeatureMinVersionLevelsCompatible( + supportedFeatures: Features[SupportedVersionRange], + featureMinVersionLevels: Map[String, Short] + ): Boolean = { + featureMinVersionLevels.forall { + case(featureName, minVersionLevel) => { + val supportedFeature = supportedFeatures.get(featureName) + (supportedFeature != null) && + new FinalizedVersionRange(minVersionLevel, supportedFeature.max()) + .isCompatibleWith(supportedFeature) + } + } + } +} diff --git a/core/src/main/scala/kafka/server/FinalizedFeatureCache.scala b/core/src/main/scala/kafka/server/FinalizedFeatureCache.scala index 6ff7e4c01b690..c648755ecdc91 100644 --- a/core/src/main/scala/kafka/server/FinalizedFeatureCache.scala +++ b/core/src/main/scala/kafka/server/FinalizedFeatureCache.scala @@ -20,6 +20,9 @@ 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) { } @@ -39,7 +42,7 @@ case class FinalizedFeaturesAndEpoch(features: Features[FinalizedVersionRange], * * @see FinalizedFeatureChangeListener */ -object FinalizedFeatureCache extends Logging { +class FinalizedFeatureCache(private val brokerFeatures: BrokerFeatures) extends Logging { @volatile private var featuresAndEpoch: Option[FinalizedFeaturesAndEpoch] = Option.empty /** @@ -53,11 +56,47 @@ 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: Int, timeoutMs: Long): Unit = { + if(minExpectedEpoch < 0) { + throw new IllegalArgumentException( + s"Expected minExpectedEpoch >= 0, but $minExpectedEpoch was provided.") + } + waitUntilConditionOrThrow( + () => featuresAndEpoch.isDefined && featuresAndEpoch.get.epoch >= minExpectedEpoch, + timeoutMs) + } + + /** + * Waits no more than timeoutMs for the cache to become empty. + * + * @param timeoutMs the timeout (in milli seconds) + * + * @throws TimeoutException if the cache's epoch has not become empty within timeoutMs. + */ + def waitUntilEmptyOrThrow(timeoutMs: Long): Unit = { + waitUntilConditionOrThrow( + () => featuresAndEpoch.isEmpty, + 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") } @@ -82,18 +121,55 @@ object FinalizedFeatureCache extends Logging { " The existing cache contents are %s").format(latest, oldFeatureAndEpoch) 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( + val logMsg = "Updated cache from existing %s to latest %s".format( oldFeatureAndEpoch, latest) - featuresAndEpoch = Some(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.") + } + synchronized { + var sleptTimeMs = 0L + while (!waitCondition()) { + val timeoutLeftMs = timeoutMs - sleptTimeMs + if (timeoutLeftMs <= 0) { + 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 timeBeforeNanos = System.nanoTime + wait(timeoutLeftMs) + sleptTimeMs += max(1L, (System.nanoTime - timeBeforeNanos) / 1_000_000) + } + } + } } diff --git a/core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala b/core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala index 91f7e0dadd547..6dda650f11af8 100644 --- a/core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala +++ b/core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala @@ -34,7 +34,7 @@ import scala.concurrent.TimeoutException * * @param zkClient the Zookeeper client */ -class FinalizedFeatureChangeListener(zkClient: KafkaZkClient) extends Logging { +class FinalizedFeatureChangeListener(private val featureCache: FinalizedFeatureCache, private val zkClient: KafkaZkClient) extends Logging { /** * Helper class used to update the FinalizedFeatureCache. @@ -85,7 +85,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() + featureCache.clear() } else { var maybeFeatureZNode: Option[FeatureZNode] = Option.empty try { @@ -93,17 +93,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() + featureCache.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() + featureCache.clear() } case FeatureZNodeStatus.Enabled => { - FinalizedFeatureCache.updateOrThrow(featureZNode.features, version) + featureCache.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..f77d6a1306b1c 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -47,6 +47,7 @@ import org.apache.kafka.common.acl.{AclBinding, AclOperation} import org.apache.kafka.common.acl.AclOperation._ import org.apache.kafka.common.config.ConfigResource import org.apache.kafka.common.errors._ +import org.apache.kafka.common.feature.{Features, FinalizedVersionRange} import org.apache.kafka.common.internals.{FatalExitError, Topic} import org.apache.kafka.common.internals.Topic.{GROUP_METADATA_TOPIC_NAME, TRANSACTION_STATE_TOPIC_NAME, isInternal} import org.apache.kafka.common.message.AlterConfigsResponseData.AlterConfigsResourceResponse @@ -65,6 +66,7 @@ import org.apache.kafka.common.message.LeaveGroupResponseData.MemberResponse import org.apache.kafka.common.message.ListOffsetRequestData.ListOffsetPartition import org.apache.kafka.common.message.ListOffsetResponseData import org.apache.kafka.common.message.ListOffsetResponseData.{ListOffsetPartitionResponse, ListOffsetTopicResponse} +import org.apache.kafka.common.message.UpdateFinalizedFeaturesResponseData import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.network.{ListenerName, Send} import org.apache.kafka.common.protocol.{ApiKeys, Errors} @@ -112,8 +114,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 featureCache: FinalizedFeatureCache) extends ApiRequestHandler with Logging { type FetchResponseStats = Map[TopicPartition, RecordConversionStats] this.logIdent = "[KafkaApi-%d] ".format(brokerId) @@ -186,7 +189,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_FINALIZED_FEATURES => handleUpdateFinalizedFeatures(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,8 +1736,8 @@ 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 + val supportedFeatures = brokerFeatures.supportedFeatures + val finalizedFeatures = featureCache.get if (finalizedFeatures.isEmpty) { ApiVersionsResponse.apiVersionsResponse( requestThrottleMs, @@ -3109,6 +3112,135 @@ class KafkaApis(val requestChannel: RequestChannel, } } + def handleUpdateFinalizedFeatures(request: RequestChannel.Request): Unit = { + val updateFinalizedFeaturesRequest = request.body[UpdateFinalizedFeaturesRequest] + def sendResponseCallback(error: Errors, msgOverride: Option[String]): Unit = { + sendResponseExemptThrottle(request, new UpdateFinalizedFeaturesResponse( + new UpdateFinalizedFeaturesResponseData() + .setErrorCode(error.code()) + .setErrorMessage(msgOverride.getOrElse(error.message())))) + } + + if (!authorize(request.context, ALTER, CLUSTER, CLUSTER_NAME)) { + sendResponseCallback(Errors.CLUSTER_AUTHORIZATION_FAILED, Option.empty) + } else if (!controller.isActive) { + sendResponseCallback(Errors.NOT_CONTROLLER, Option.empty) + } else if (!config.isFeatureVersioningEnabled) { + sendResponseCallback(Errors.INVALID_REQUEST, Some("Feature versioning system is disabled.")) + } else { + val targetFeaturesOrError = getTargetFinalizedFeaturesOrError(updateFinalizedFeaturesRequest) + targetFeaturesOrError match { + case Left(targetFeatures) => + controller.updateFinalizedFeatures(targetFeatures, sendResponseCallback) + case Right(error) => + sendResponseCallback(error.error, Some(error.message)) + } + } + } + + /** + * Validates the provided UpdateFinalizedFeaturesRequest, checking for various error cases. + * If the validation is successful, returns the target finalized features constructed from the + * request. + * + * @param request the request to be validated + * + * @return - the target finalized features, if request validation is successful + * - an ApiError if request validation fails + */ + def getTargetFinalizedFeaturesOrError(request: UpdateFinalizedFeaturesRequest): + Either[Features[FinalizedVersionRange], ApiError] = { + val updates = request.data.finalizedFeatureUpdates + if (updates.isEmpty) { + return Right(new ApiError(Errors.INVALID_REQUEST, + "Can not provide empty FinalizedFeatureUpdates in the request.")) + } + val latestFeatures = featureCache.get + val newFeatures = scala.collection.mutable.Map[String, FinalizedVersionRange]() + updates.asScala.foreach( + update => { + // Rule #1) Check that the feature name is not empty. + if (update.name.isEmpty) { + return Right( + new ApiError(Errors.INVALID_REQUEST, + "Can not contain empty feature name in the request.")) + } + + val cacheEntry = latestFeatures.map(lf => lf.features.get(update.name)).orNull + + // We handle deletion requests separately from non-deletion requests. + if (UpdateFinalizedFeaturesRequest.isDeleteRequest(update)) { // Deletion request + // Rule #2) Disallow deletion of a finalized feature without allowDowngrade flag set. + if (!update.allowDowngrade) { + return Right( + new ApiError(Errors.INVALID_REQUEST, + s"Can not delete feature: '${update.name}' without setting the" + + " allowDowngrade flag in the request.")) + } + + // Rule #3) Disallow deletion of a non-existing finalized feature. + if (cacheEntry == null) { + return Right( + new ApiError(Errors.INVALID_REQUEST, + s"Can not delete non-existing finalized feature: '${update.name}'")) + } + } else { // Non-deletion request + if (cacheEntry != null) { + // Rule 4) Disallow a case where target maxVersionLevel matches + // existing maxVersionLevel. + if (update.maxVersionLevel == cacheEntry.max()) { + return Right( + new ApiError(Errors.INVALID_REQUEST, + s"Can not upgrade a finalized feature: '${update.name}' from" + + s" existing maxVersionLevel:${cacheEntry.max} to the same value.")) + } + + // Rule #5) Disallow downgrade of a finalized feature without the + // allowDowngrade flag set. + if (update.maxVersionLevel < cacheEntry.max && !update.allowDowngrade) { + return Right( + new ApiError(Errors.INVALID_REQUEST, + s"Can not downgrade finalized feature: '${update.name}' from" + + s" existing maxVersionLevel:${cacheEntry.max} to provided" + + s" maxVersionLevel:${update.maxVersionLevel} without setting the" + + " allowDowngrade flag in the request.")) + } + + // Rule #6) Disallow a request that sets allowDowngrade flag without specifying a + // maxVersionLevel that's lower than the existing maxVersionLevel. + if (update.allowDowngrade && update.maxVersionLevel > cacheEntry.max) { + return Right( + new ApiError(Errors.INVALID_REQUEST, + s"When finalized feature: '${update.name}' has the allowDowngrade flag" + + s" set in the request, the provided maxVersionLevel:${update.maxVersionLevel}" + + s" can not be greater than existing maxVersionLevel:${cacheEntry.max}.")) + } + + // Rule #7) Disallow downgrade of a finalized feature below the existing finalized + // minVersionLevel. + if (update.maxVersionLevel() < cacheEntry.min()) { + return Right( + new ApiError(Errors.INVALID_REQUEST, + s"Can not downgrade finalized feature: '${update.name}' to" + + s" maxVersionLevel:${update.maxVersionLevel} because it's lower than the" + + s" existing minVersionLevel:${cacheEntry.min}.")) + } + } + + // NOTE: if the finalized feature already exists, then, below we set the default minimum + // version level. This can cause deprecation of all version levels in the closed range: + // [existingVersionRange.min(), defaultMinVersionLevel - 1] + val defaultMinVersionLevel = brokerFeatures.defaultMinVersionLevel(update.name) + newFeatures += ( + update.name -> new FinalizedVersionRange( + defaultMinVersionLevel, + update.maxVersionLevel)) + } + } + ) + Left(Features.finalizedFeatures(newFeatures.asJava)) + } + // private package for testing private[server] def authorize(requestContext: RequestContext, operation: AclOperation, diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 8defa4204c800..1a963ee70417e 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,7 +233,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = Time.SYSTEM, threadNameP initZkClient(time) /* initialize features */ - _featureChangeListener = new FinalizedFeatureChangeListener(_zkClient) + _featureChangeListener = new FinalizedFeatureChangeListener(featureCache, _zkClient) if (config.isFeatureVersioningEnabled) { _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) @@ -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..87c2335c538a6 100644 --- a/core/src/main/scala/kafka/zk/ZkData.scala +++ b/core/src/main/scala/kafka/zk/ZkData.scala @@ -797,42 +797,6 @@ 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 { val Disabled, Enabled = Value diff --git a/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala b/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala index 62e8d42ff2558..524c19e16b7c6 100644 --- a/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala @@ -21,11 +21,14 @@ 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 org.junit.{After, Before, Test} +import org.junit.Assert.{assertEquals, assertNotEquals, assertTrue} +import org.apache.kafka.common.{ElectionType, TopicPartition} import org.apache.kafka.common.errors.{ControllerMovedException, StaleBrokerEpochException} import org.apache.kafka.common.metrics.KafkaMetric import org.apache.kafka.common.protocol.Errors @@ -595,6 +598,16 @@ class ControllerIntegrationTest extends ZooKeeperTestHarness { testControllerMove(() => zkClient.createPartitionReassignment(reassignment)) } + @Test + def testControllerFeatureZNodeSetupWhenFeatureVersioningIsEnabled(): Unit = { + testControllerFeatureZNodeSetup(KAFKA_2_7_IV0) + } + + @Test + def testControllerFeatureZNodeSetupWhenFeatureVersioningIsDisabled(): Unit = { + testControllerFeatureZNodeSetup(KAFKA_2_6_IV0) + } + @Test def testControllerDetectsBouncedBrokers(): Unit = { servers = makeServers(2, enableControlledShutdown = false) @@ -679,6 +692,7 @@ class ControllerIntegrationTest extends ZooKeeperTestHarness { controller.eventManager.thread = spyThread val processedEvent = new MockEvent(ControllerState.TopicChange) { override def process(): Unit = latch.await() + override def preempt(): Unit = {} } val tp0 = new TopicPartition("t", 0) @@ -697,7 +711,7 @@ class ControllerIntegrationTest extends ZooKeeperTestHarness { case Failure(e) => assertEquals(classOf[ControllerMovedException], e.getClass) }) - val event3 = ApiPartitionReassignment(Map(tp0 -> None, tp1 -> None), { + val event3 = ApiPartitionReassignment(Map(tp0 -> None, tp1 -> None), { case Left(_) => fail("api partition reassignment should error") case Right(e) => assertEquals(Errors.NOT_CONTROLLER, e.error()) }) @@ -715,7 +729,22 @@ class ControllerIntegrationTest extends ZooKeeperTestHarness { doAnswer((_: InvocationOnMock) => { latch.countDown() }).doCallRealMethod().when(spyThread).awaitShutdown() - controller.shutdown() + controller.shutdown() + } + + private def testControllerFeatureZNodeSetup(interBrokerProtocolVersion: ApiVersion): Unit = { + servers = makeServers(1, interBrokerProtocolVersion = Some(interBrokerProtocolVersion)) + TestUtils.waitUntilControllerElected(zkClient) + + val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path) + assertNotEquals(version, ZkVersion.UnknownVersion) + val featureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get) + if (interBrokerProtocolVersion >= KAFKA_2_7_IV0) { + assertEquals(FeatureZNodeStatus.Enabled, featureZNode.status) + } else { + assertEquals(FeatureZNodeStatus.Disabled, featureZNode.status) + } + assertTrue(featureZNode.features.empty) } @Test @@ -840,6 +869,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 +879,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..642ecdd2a5f75 --- /dev/null +++ b/core/src/test/scala/unit/kafka/server/BrokerFeaturesTest.scala @@ -0,0 +1,83 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.server + +import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange} +import org.junit.Assert.{assertEquals, assertThrows, assertTrue} +import org.junit.Test + +import scala.jdk.CollectionConverters._ + +class BrokerFeaturesTest { + @Test + def testEmpty(): Unit = { + assertTrue(BrokerFeatures.createDefault().supportedFeatures.empty) + } + + @Test + def testIncompatibleFeatures(): Unit = { + val brokerFeatures = BrokerFeatures.createDefault() + val supportedFeatures = Map[String, SupportedVersionRange]( + "test_feature_1" -> new SupportedVersionRange(1, 4), + "test_feature_2" -> new SupportedVersionRange(1, 3)) + brokerFeatures.setSupportedFeatures(Features.supportedFeatures(supportedFeatures.asJava)) + + val compatibleFeatures = Map[String, FinalizedVersionRange]( + "test_feature_1" -> new FinalizedVersionRange(2, 3)) + val inCompatibleFeatures = Map[String, FinalizedVersionRange]( + "test_feature_2" -> new FinalizedVersionRange(1, 4), + "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)) + } + + @Test + def testFeatureVersionAssertions(): 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 defaultMinVersionLevelsWithNonExistingFeature = Map[String, Short]( + "test_feature_1" -> 2, + "test_feature_2" -> 2, + "test_feature_non_existing" -> 5) + assertThrows( + classOf[IllegalArgumentException], + () => brokerFeatures.setDefaultMinVersionLevels(defaultMinVersionLevelsWithNonExistingFeature)) + + val defaultMinVersionLevelsWithInvalidValue1 = Map[String, Short]( + "test_feature_1" -> 2, + "test_feature_2" -> (supportedFeatures.get("test_feature_2").min() - 1).asInstanceOf[Short]) + assertThrows( + classOf[IllegalArgumentException], + () => brokerFeatures.setDefaultMinVersionLevels(defaultMinVersionLevelsWithInvalidValue1)) + + val defaultMinVersionLevelsWithInvalidValue2 = Map[String, Short]( + "test_feature_1" -> 2, + "test_feature_2" -> (supportedFeatures.get("test_feature_2").max() + 1).asInstanceOf[Short]) + assertThrows( + classOf[IllegalArgumentException], + () => brokerFeatures.setDefaultMinVersionLevels(defaultMinVersionLevelsWithInvalidValue2)) + } +} diff --git a/core/src/test/scala/unit/kafka/server/FinalizedFeatureCacheTest.scala b/core/src/test/scala/unit/kafka/server/FinalizedFeatureCacheTest.scala index e664ea4669792..bdaaeca4f6080 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 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 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 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..d8b6bf6275c44 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 @@ -81,9 +79,10 @@ class FinalizedFeatureChangeListenerTest extends ZooKeeperTestHarness { */ @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)) @@ -93,9 +92,9 @@ class FinalizedFeatureChangeListenerTest extends ZooKeeperTestHarness { 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") + + cache.waitUntilEpochOrThrow(updatedVersion, JTestUtils.DEFAULT_MAX_WAIT_MS) + assertEquals(FinalizedFeaturesAndEpoch(updatedFinalizedFeatures, updatedVersion), cache.get.get) assertTrue(listener.isListenerInitiated) } @@ -105,16 +104,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 +125,10 @@ 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 listener = createListener(cache, Some(initialFinalizedFeatures)) val updatedFinalizedFeaturesMap = Map[String, FinalizedVersionRange]() val updatedFinalizedFeatures = Features.finalizedFeatures(updatedFinalizedFeaturesMap.asJava) @@ -137,9 +138,42 @@ class FinalizedFeatureChangeListenerTest extends ZooKeeperTestHarness { assertFalse(mayBeFeatureZNodeNewBytes.isEmpty) assertTrue(updatedVersion > initialFinalizedFeatures.epoch) - TestUtils.waitUntilTrue(() => { - FinalizedFeatureCache.get.isEmpty - }, "Timed out waiting for FinalizedFeatureCache to become empty") + cache.waitUntilEmptyOrThrow(JTestUtils.DEFAULT_MAX_WAIT_MS) + assertTrue(cache.get.isEmpty) + assertTrue(listener.isListenerInitiated) + } + + /** + * 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)) + assertThrows( + classOf[TimeoutException], + () => cache.waitUntilEmptyOrThrow(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)) + cache.waitUntilEmptyOrThrow(JTestUtils.DEFAULT_MAX_WAIT_MS) + assertTrue(cache.get.isEmpty) assertTrue(listener.isListenerInitiated) } @@ -149,7 +183,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 +197,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 +215,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,9 +228,10 @@ 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()) @@ -216,7 +254,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/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/UpdateFinalizedFeaturesTest.scala b/core/src/test/scala/unit/kafka/server/UpdateFinalizedFeaturesTest.scala new file mode 100644 index 0000000000000..cbc48c0b2456f --- /dev/null +++ b/core/src/test/scala/unit/kafka/server/UpdateFinalizedFeaturesTest.scala @@ -0,0 +1,450 @@ +package kafka.server + +import java.util +import java.util.Arrays +import java.util.Collections +import java.util.HashSet +import java.util.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, FeatureMetadata, FinalizedFeatureUpdate, UpdateFinalizedFeaturesOptions, UpdateFinalizedFeaturesResult} +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.UpdateFinalizedFeaturesRequestData +import org.apache.kafka.common.protocol.Errors +import org.apache.kafka.common.requests.{UpdateFinalizedFeaturesRequest, UpdateFinalizedFeaturesResponse} +import org.apache.kafka.common.utils.Utils +import org.junit.Test +import org.junit.Assert.{assertEquals, assertNotEquals, assertNotNull, assertTrue} +import org.scalatest.Assertions.{assertThrows, intercept} + +import scala.reflect.ClassTag +import scala.util.matching.Regex + +class UpdateFinalizedFeaturesTest 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 updateSupportedFeatures(features: Features[SupportedVersionRange]): Unit = { + updateSupportedFeatures(features, Set[KafkaServer]() ++ servers) + } + + private def updateDefaultMinVersionLevels(newMinVersionLevels: Map[String, Short]): Unit = { + servers.foreach(s => { + s.brokerFeatures.setDefaultMinVersionLevels(newMinVersionLevels) + }) + } + + 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 checkFeatures(client: Admin, expectedNode: FeatureZNode, expectedMetadata: FeatureMetadata): Unit = { + assertEquals(expectedNode, getFeatureZNode()) + val featureMetadata = client.describeFeatures( + new DescribeFeaturesOptions().sendRequestToController(true)).featureMetadata().get() + assertEquals(expectedMetadata, featureMetadata) + } + + private def checkException[ExceptionType <: Throwable]( + result: UpdateFinalizedFeaturesResult, + exceptionMsgPattern: Regex + )(implicit tag: ClassTag[ExceptionType]): Unit = { + val exception = intercept[ExecutionException] { + result.result().get() + } + assertNotNull(exception.getCause) + assertEquals(exception.getCause.getClass, tag.runtimeClass) + assertTrue(exceptionMsgPattern.findFirstIn(exception.getCause.getMessage).isDefined) + } + + /** + * Tests whether an invalid feature update does not get processed on the server (as expected) + * and raises the InvalidRequestException on the client side (as expected). + * + * @param invalidUpdate the invalid feature update to be sent in the + * updateFinalizedFeatures request to the server + * @param exceptionMsgPattern a pattern for the expected exception message + */ + private def testWithInvalidFeatureUpdate[ExceptionType <: Throwable]( + invalidUpdate: FinalizedFeatureUpdate, + exceptionMsgPattern: Regex + )(implicit tag: ClassTag[ExceptionType]): Unit = { + TestUtils.waitUntilControllerElected(zkClient) + + updateSupportedFeatures(defaultSupportedFeatures()) + val versionBefore = updateFeatureZNode(defaultFinalizedFeatures()) + + val adminClient = createAdminClient() + val nodeBefore = getFeatureZNode() + + val result = adminClient.updateFinalizedFeatures( + new HashSet[FinalizedFeatureUpdate](Collections.singletonList(invalidUpdate)), new UpdateFinalizedFeaturesOptions()) + + checkException[ExceptionType](result, exceptionMsgPattern) + checkFeatures( + adminClient, + nodeBefore, + new FeatureMetadata(defaultFinalizedFeatures(), versionBefore, defaultSupportedFeatures())) + } + + @Test + def testShouldFailRequestIfNotController(): Unit = { + TestUtils.waitUntilControllerElected(zkClient) + + updateSupportedFeatures(defaultSupportedFeatures()) + val versionBefore = updateFeatureZNode(defaultFinalizedFeatures()) + + val nodeBefore = getFeatureZNode() + val requestData = FinalizedFeatureUpdate.createRequest( + new util.HashSet[FinalizedFeatureUpdate]( + Collections.singletonList(new FinalizedFeatureUpdate("feature_1", + defaultSupportedFeatures().get("feature_1").max(), + false)))) + + val response = connectAndReceive[UpdateFinalizedFeaturesResponse]( + new UpdateFinalizedFeaturesRequest.Builder(requestData).build(), notControllerSocketServer) + + assertEquals(Errors.NOT_CONTROLLER, response.error()) + checkFeatures( + createAdminClient(), + nodeBefore, + new FeatureMetadata(defaultFinalizedFeatures(), versionBefore, defaultSupportedFeatures())) + } + + @Test + def testShouldFailRequestForInvalidFeatureName(): Unit = { + testWithInvalidFeatureUpdate[InvalidRequestException](new FinalizedFeatureUpdate( + "", + defaultSupportedFeatures().get("feature_1").max(), + false), + ".*empty feature name.*".r) + } + + @Test + def testShouldFailRequestWhenDowngradeFlagIsNotSetDuringDowngrade(): Unit = { + testWithInvalidFeatureUpdate[InvalidRequestException](new FinalizedFeatureUpdate( + "feature_1", + (defaultFinalizedFeatures().get("feature_1").max() - 1).asInstanceOf[Short], + false), + ".*Can not downgrade finalized feature: 'feature_1'.*allowDowngrade.*".r) + } + + @Test + def testShouldFailRequestWhenDowngradeToHigherVersionLevelIsAttempted(): Unit = { + testWithInvalidFeatureUpdate[InvalidRequestException]( + new FinalizedFeatureUpdate( + "feature_1", + defaultSupportedFeatures().get("feature_1").max(), + true), + ".*finalized feature: 'feature_1'.*allowDowngrade.* provided maxVersionLevel:3.*existing maxVersionLevel:2.*".r) + } + + @Test + def testShouldFailRequestInClientWhenDowngradeFlagIsNotSetDuringDeletion(): Unit = { + assertThrows[IllegalArgumentException] { + new FinalizedFeatureUpdate("feature_1", 0, false) + } + } + + @Test + def testShouldFailRequestInServerWhenDowngradeFlagIsNotSetDuringDeletion(): Unit = { + TestUtils.waitUntilControllerElected(zkClient) + + updateSupportedFeatures(defaultSupportedFeatures()) + val versionBefore = updateFeatureZNode(defaultFinalizedFeatures()) + + val adminClient = createAdminClient() + val nodeBefore = getFeatureZNode() + + val featureUpdates + = new UpdateFinalizedFeaturesRequestData.FinalizedFeatureUpdateKeyCollection(); + val featureUpdate = new UpdateFinalizedFeaturesRequestData.FinalizedFeatureUpdateKey(); + featureUpdate.setName("feature_1") + featureUpdate.setMaxVersionLevel(0) + featureUpdate.setAllowDowngrade(false) + featureUpdates.add(featureUpdate); + val requestData = new UpdateFinalizedFeaturesRequestData() + requestData.setFinalizedFeatureUpdates(featureUpdates); + + val response = connectAndReceive[UpdateFinalizedFeaturesResponse]( + new UpdateFinalizedFeaturesRequest.Builder(requestData).build(), controllerSocketServer) + + assertEquals(Errors.INVALID_REQUEST, response.error) + val exceptionMsgPattern = ".*Can not delete feature: 'feature_1'.*allowDowngrade.*".r + assertTrue(exceptionMsgPattern.findFirstIn(response.data.errorMessage).isDefined) + checkFeatures( + adminClient, + nodeBefore, + new FeatureMetadata(defaultFinalizedFeatures(), versionBefore, defaultSupportedFeatures())) + } + + @Test + def testShouldFailRequestDuringDeletionOfNonExistingFeature(): Unit = { + testWithInvalidFeatureUpdate[InvalidRequestException]( + new FinalizedFeatureUpdate("feature_non_existing", 0, true), + ".*Can not delete non-existing finalized feature: 'feature_non_existing'.*".r) + } + + @Test + def testShouldFailRequestWhenUpgradingToSameVersionLevel(): Unit = { + testWithInvalidFeatureUpdate[InvalidRequestException]( + new FinalizedFeatureUpdate( + "feature_1", defaultFinalizedFeatures().get("feature_1").max(), false), + ".*Can not upgrade a finalized feature: 'feature_1'.*to the same value.*".r) + } + + @Test + def testShouldFailRequestWhenDowngradingBelowMinVersionLevel(): Unit = { + TestUtils.waitUntilControllerElected(zkClient) + + updateSupportedFeatures(defaultSupportedFeatures()) + val minVersionLevel = 2.asInstanceOf[Short] + updateDefaultMinVersionLevels(Map[String, Short]("feature_1" -> minVersionLevel)) + val initialFinalizedFeatures = Features.finalizedFeatures( + Utils.mkMap(Utils.mkEntry("feature_1", new FinalizedVersionRange(minVersionLevel, 2)))) + val versionBefore = updateFeatureZNode(initialFinalizedFeatures) + + val update = new FinalizedFeatureUpdate( + "feature_1", (minVersionLevel - 1).asInstanceOf[Short], true) + val adminClient = createAdminClient() + val nodeBefore = getFeatureZNode() + + val result = adminClient.updateFinalizedFeatures( + new HashSet[FinalizedFeatureUpdate](Collections.singletonList(update)), new UpdateFinalizedFeaturesOptions()) + + checkException[InvalidRequestException]( + result, ".*Can not downgrade finalized feature: 'feature_1' to maxVersionLevel:1.*existing minVersionLevel:2.*".r) + checkFeatures( + adminClient, + nodeBefore, + new FeatureMetadata(initialFinalizedFeatures, versionBefore, defaultSupportedFeatures())) + } + + @Test + def testShouldFailRequestDuringBrokerMaxVersionLevelIncompatibility(): Unit = { + TestUtils.waitUntilControllerElected(zkClient) + + val controller = servers.filter { server => server.kafkaController.isActive}.head + val nonControllerServers = servers.filter { server => !server.kafkaController.isActive} + val unsupportedBrokers = Set[KafkaServer](nonControllerServers(0)) + val supportedBrokers = Set[KafkaServer](nonControllerServers(1), controller) + + updateSupportedFeatures(defaultSupportedFeatures(), supportedBrokers) + + val validMinVersion = defaultSupportedFeatures().get("feature_1").min() + val unsupportedMaxVersion = + (defaultSupportedFeatures().get("feature_1").max() - 1).asInstanceOf[Short] + val badSupportedFeatures = Features.supportedFeatures( + Utils.mkMap( + Utils.mkEntry("feature_1", + new SupportedVersionRange( + validMinVersion, + unsupportedMaxVersion)))) + updateSupportedFeatures(badSupportedFeatures, unsupportedBrokers) + + val versionBefore = updateFeatureZNode(defaultFinalizedFeatures()) + + val invalidUpdate = new FinalizedFeatureUpdate( + "feature_1", defaultSupportedFeatures().get("feature_1").max(), false) + val nodeBefore = getFeatureZNode() + val adminClient = createAdminClient() + val result = adminClient.updateFinalizedFeatures( + new HashSet[FinalizedFeatureUpdate](Collections.singletonList(invalidUpdate)), + new UpdateFinalizedFeaturesOptions()) + + checkException[InvalidRequestException]( + result, ".*1 broker.*incompatible.*".r) + checkFeatures( + adminClient, + nodeBefore, + new FeatureMetadata(defaultFinalizedFeatures(), versionBefore, defaultSupportedFeatures())) + } + + @Test + def testSuccessFeatureUpgradeAndDowngrade(): Unit = { + TestUtils.waitUntilControllerElected(zkClient) + + updateSupportedFeatures( + Features.supportedFeatures( + Utils.mkMap( + Utils.mkEntry("feature_1", new SupportedVersionRange(1, 3)), + Utils.mkEntry("feature_2", new SupportedVersionRange(2, 5))))) + updateDefaultMinVersionLevels(Map[String, Short]("feature_1" -> 1, "feature_2" -> 2)) + val versionBefore = updateFeatureZNode( + Features.finalizedFeatures( + Utils.mkMap( + Utils.mkEntry("feature_1", new FinalizedVersionRange(1, 2)), + Utils.mkEntry("feature_2", new FinalizedVersionRange(2, 4))))) + + 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 FinalizedFeatureUpdate( + "feature_1", targetFinalizedFeatures.get("feature_1").max(), false) + val update2 = new FinalizedFeatureUpdate( + "feature_2", targetFinalizedFeatures.get("feature_2").max(), true) + + val expected = new FeatureMetadata( + targetFinalizedFeatures, + versionBefore + 1, + Features.supportedFeatures( + Utils.mkMap( + Utils.mkEntry("feature_1", new SupportedVersionRange(1, 3)), + Utils.mkEntry("feature_2", new SupportedVersionRange(2, 5))))) + + val adminClient = createAdminClient() + adminClient.updateFinalizedFeatures( + new HashSet[FinalizedFeatureUpdate](Arrays.asList(update1, update2)), + new UpdateFinalizedFeaturesOptions()).result().get() + + checkFeatures( + adminClient, + new FeatureZNode(FeatureZNodeStatus.Enabled, targetFinalizedFeatures), + expected) + } + + @Test + def testShouldFailRequestDuringValidFeatureUpgradeAndInvalidDowngrade(): Unit = { + TestUtils.waitUntilControllerElected(zkClient) + + val initialSupportedFeatures = Features.supportedFeatures( + Utils.mkMap( + Utils.mkEntry("feature_1", new SupportedVersionRange(1, 3)), + Utils.mkEntry("feature_2", new SupportedVersionRange(2, 5)))) + updateSupportedFeatures(initialSupportedFeatures) + updateDefaultMinVersionLevels(Map[String, Short]("feature_1" -> 1, "feature_2" -> 2)) + 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) + + 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 FinalizedFeatureUpdate( + "feature_1", targetFinalizedFeatures.get("feature_1").max(), false) + val invalidUpdate = new FinalizedFeatureUpdate( + "feature_2", targetFinalizedFeatures.get("feature_2").max(), false) + + val nodeBefore = getFeatureZNode() + val adminClient = createAdminClient() + val result = adminClient.updateFinalizedFeatures( + new HashSet[FinalizedFeatureUpdate](Arrays.asList(validUpdate, invalidUpdate)), + new UpdateFinalizedFeaturesOptions()) + + checkException[InvalidRequestException]( + result, ".*Can not downgrade finalized feature: 'feature_2'.*allowDowngrade.*".r) + checkFeatures( + adminClient, + nodeBefore, + new FeatureMetadata(initialFinalizedFeatures, versionBefore, initialSupportedFeatures)) + } + + @Test + def testShouldFailRequestDuringInvalidFeatureUpgradeAndValidDowngrade(): Unit = { + TestUtils.waitUntilControllerElected(zkClient) + + val controller = servers.filter { server => server.kafkaController.isActive}.head + val nonControllerServers = servers.filter { server => !server.kafkaController.isActive} + val unsupportedBrokers = Set[KafkaServer](nonControllerServers(0)) + val supportedBrokers = Set[KafkaServer](nonControllerServers(1), controller) + + val initialSupportedFeatures = Features.supportedFeatures( + Utils.mkMap( + Utils.mkEntry("feature_1", new SupportedVersionRange(1, 3)), + Utils.mkEntry("feature_2", new SupportedVersionRange(2, 5)))) + updateSupportedFeatures(initialSupportedFeatures, supportedBrokers) + + val badSupportedFeatures = Features.supportedFeatures( + Utils.mkMap( + Utils.mkEntry("feature_1", new SupportedVersionRange(1, 2)), + Utils.mkEntry("feature_2", new SupportedVersionRange(2, 5)))) + updateSupportedFeatures(badSupportedFeatures, unsupportedBrokers) + + updateDefaultMinVersionLevels(Map[String, Short]("feature_1" -> 1, "feature_2" -> 2)) + + 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) + + 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 FinalizedFeatureUpdate( + "feature_1", targetFinalizedFeatures.get("feature_1").max(), false) + val validUpdate = new FinalizedFeatureUpdate( + "feature_2", targetFinalizedFeatures.get("feature_2").max(), true) + + val nodeBefore = getFeatureZNode() + val adminClient = createAdminClient() + val result = adminClient.updateFinalizedFeatures( + new HashSet[FinalizedFeatureUpdate](Arrays.asList(invalidUpdate, validUpdate)), + new UpdateFinalizedFeaturesOptions()) + + checkException[InvalidRequestException](result, ".*1 broker.*incompatible.*".r) + checkFeatures( + adminClient, + nodeBefore, + new FeatureMetadata(initialFinalizedFeatures, versionBefore, initialSupportedFeatures)) + } +} From cc23765e8c9244a4701c0e6ccbd281af6bd8b794 Mon Sep 17 00:00:00 2001 From: Kowshik Prakasam Date: Mon, 13 Jul 2020 12:16:35 -0700 Subject: [PATCH 02/41] Fix checkstyle issues --- .../clients/admin/DescribeFeaturesOptions.java | 16 ++++++++++++++++ .../clients/admin/DescribeFeaturesResult.java | 16 ++++++++++++++++ .../kafka/clients/admin/FeatureMetadata.java | 16 ++++++++++++++++ .../clients/admin/FinalizedFeatureUpdate.java | 16 ++++++++++++++++ .../admin/UpdateFinalizedFeaturesOptions.java | 16 ++++++++++++++++ .../admin/UpdateFinalizedFeaturesResult.java | 16 ++++++++++++++++ .../FinalizedFeatureUpdateFailedException.java | 16 ++++++++++++++++ .../apache/kafka/common/feature/Features.java | 9 --------- .../UpdateFinalizedFeaturesRequest.java | 16 ++++++++++++++++ .../UpdateFinalizedFeaturesResponse.java | 17 ++++++++++++++++- .../clients/admin/KafkaAdminClientTest.java | 18 +++++++++++------- .../jmh/metadata/MetadataRequestBenchmark.java | 7 ++++++- 12 files changed, 161 insertions(+), 18 deletions(-) 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 index 2a073f6008c86..45bd656cb85bf 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/DescribeFeaturesOptions.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/DescribeFeaturesOptions.java @@ -1,3 +1,19 @@ +/* + * 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; 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 index e9306b997342f..f16f904ce16e3 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/DescribeFeaturesResult.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/DescribeFeaturesResult.java @@ -1,3 +1,19 @@ +/* + * 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; 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 index aaf4ab177731e..149df5f99b203 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/FeatureMetadata.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/FeatureMetadata.java @@ -1,3 +1,19 @@ +/* + * 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; diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/FinalizedFeatureUpdate.java b/clients/src/main/java/org/apache/kafka/clients/admin/FinalizedFeatureUpdate.java index 76c74071c3181..fcd4599a4f04a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/FinalizedFeatureUpdate.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/FinalizedFeatureUpdate.java @@ -1,3 +1,19 @@ +/* + * 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; diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/UpdateFinalizedFeaturesOptions.java b/clients/src/main/java/org/apache/kafka/clients/admin/UpdateFinalizedFeaturesOptions.java index 1a7ed8d538ced..6f615665ee0f7 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/UpdateFinalizedFeaturesOptions.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/UpdateFinalizedFeaturesOptions.java @@ -1,3 +1,19 @@ +/* + * 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; diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/UpdateFinalizedFeaturesResult.java b/clients/src/main/java/org/apache/kafka/clients/admin/UpdateFinalizedFeaturesResult.java index 2516d200ac908..e9dbffe9520cb 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/UpdateFinalizedFeaturesResult.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/UpdateFinalizedFeaturesResult.java @@ -1,3 +1,19 @@ +/* + * 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; diff --git a/clients/src/main/java/org/apache/kafka/common/errors/FinalizedFeatureUpdateFailedException.java b/clients/src/main/java/org/apache/kafka/common/errors/FinalizedFeatureUpdateFailedException.java index ae53fd271156b..174f8bc39a21c 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/FinalizedFeatureUpdateFailedException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/FinalizedFeatureUpdateFailedException.java @@ -1,3 +1,19 @@ +/* + * 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 FinalizedFeatureUpdateFailedException extends ApiException { diff --git a/clients/src/main/java/org/apache/kafka/common/feature/Features.java b/clients/src/main/java/org/apache/kafka/common/feature/Features.java index 1b842f2e48e9a..4006d71947fb7 100644 --- a/clients/src/main/java/org/apache/kafka/common/feature/Features.java +++ b/clients/src/main/java/org/apache/kafka/common/feature/Features.java @@ -93,15 +93,6 @@ public VersionRangeType get(String feature) { return features.get(feature); } - /** - * @param feature the feature to be checked - * @return - true, if the feature is present - * - false, otherwise. - */ - public boolean has(String feature) { - return get(feature) != null; - } - public String toString() { return String.format( "Features{%s}", diff --git a/clients/src/main/java/org/apache/kafka/common/requests/UpdateFinalizedFeaturesRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/UpdateFinalizedFeaturesRequest.java index 918575b786072..19810507911f5 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/UpdateFinalizedFeaturesRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/UpdateFinalizedFeaturesRequest.java @@ -1,3 +1,19 @@ +/* + * 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; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/UpdateFinalizedFeaturesResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/UpdateFinalizedFeaturesResponse.java index e52e67eedf869..b00377701a674 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/UpdateFinalizedFeaturesResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/UpdateFinalizedFeaturesResponse.java @@ -1,8 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.kafka.common.requests; import java.nio.ByteBuffer; import java.util.Map; -import org.apache.kafka.common.message.AlterPartitionReassignmentsResponseData; import org.apache.kafka.common.message.UpdateFinalizedFeaturesResponseData; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; 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 cb698811b4290..e28bb0683024a 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 @@ -492,13 +492,17 @@ private static UpdateFinalizedFeaturesResponse prepareUpdateFinalizedFeaturesRes private static FeatureMetadata getDefaultFeatureMetadata() { return new FeatureMetadata( - Features.finalizedFeatures(new HashMap() {{ - put("test_feature_1", new FinalizedVersionRange((short) 2, (short) 3)); - }}), + Features.finalizedFeatures(new HashMap() { + { + put("test_feature_1", new FinalizedVersionRange((short) 2, (short) 3)); + } + }), 1, - Features.supportedFeatures(new HashMap() {{ - put("test_feature_1", new SupportedVersionRange((short) 1, (short) 5)); - }}) + Features.supportedFeatures(new HashMap() { + { + put("test_feature_1", new SupportedVersionRange((short) 1, (short) 5)); + } + }) ); } @@ -3982,7 +3986,7 @@ public void testUpdateFinalizedFeaturesHandleNotControllerException() throws Exc new HashSet<>( Arrays.asList( new FinalizedFeatureUpdate( - "test_feature_1",(short) 2, false), + "test_feature_1", (short) 2, false), new FinalizedFeatureUpdate( "test_feature_2", (short) 3, true))), new UpdateFinalizedFeaturesOptions().timeoutMs(10000)).result(); 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) From 114e24df5f08a42a4b9175cc57f1efc13e25df6e Mon Sep 17 00:00:00 2001 From: Kowshik Prakasam Date: Mon, 13 Jul 2020 12:19:13 -0700 Subject: [PATCH 03/41] Minor: Improved comment --- .../scala/unit/kafka/server/UpdateFinalizedFeaturesTest.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/test/scala/unit/kafka/server/UpdateFinalizedFeaturesTest.scala b/core/src/test/scala/unit/kafka/server/UpdateFinalizedFeaturesTest.scala index cbc48c0b2456f..fe699c34030f2 100644 --- a/core/src/test/scala/unit/kafka/server/UpdateFinalizedFeaturesTest.scala +++ b/core/src/test/scala/unit/kafka/server/UpdateFinalizedFeaturesTest.scala @@ -112,8 +112,8 @@ class UpdateFinalizedFeaturesTest extends BaseRequestTest { } /** - * Tests whether an invalid feature update does not get processed on the server (as expected) - * and raises the InvalidRequestException on the client side (as expected). + * 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 invalidUpdate the invalid feature update to be sent in the * updateFinalizedFeatures request to the server From 9af2a010ae81d2dc31383eb986cdf55fcd6f6509 Mon Sep 17 00:00:00 2001 From: Kowshik Prakasam Date: Tue, 14 Jul 2020 02:20:40 -0700 Subject: [PATCH 04/41] Minor: cosmetics --- core/src/main/scala/kafka/server/BrokerFeatures.scala | 2 +- core/src/main/scala/kafka/server/KafkaApis.scala | 4 +++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/kafka/server/BrokerFeatures.scala b/core/src/main/scala/kafka/server/BrokerFeatures.scala index ab39f1fb8665f..6ec977e7c181f 100644 --- a/core/src/main/scala/kafka/server/BrokerFeatures.scala +++ b/core/src/main/scala/kafka/server/BrokerFeatures.scala @@ -34,7 +34,7 @@ import scala.jdk.CollectionConverters._ * higher than 1 (let's call this new_min_version_level). In doing so, the feature version levels * in the closed range: [1, latest_min_version_level - 1] get deprecated by the controller logic * that applies this map to persistent finalized feature state in ZK (this mutation happens - * during controller and during finalized feature updates via the + * during controller election and during finalized feature updates via the * APIKeys.UPDATE_FINALIZED_FEATURES api). * * This class also provides APIs to check for incompatibilities between the features supported by diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index f77d6a1306b1c..ec82d1a73f571 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -3155,6 +3155,7 @@ class KafkaApis(val requestChannel: RequestChannel, return Right(new ApiError(Errors.INVALID_REQUEST, "Can not provide empty FinalizedFeatureUpdates in the request.")) } + val latestFeatures = featureCache.get val newFeatures = scala.collection.mutable.Map[String, FinalizedVersionRange]() updates.asScala.foreach( @@ -3229,7 +3230,8 @@ class KafkaApis(val requestChannel: RequestChannel, // NOTE: if the finalized feature already exists, then, below we set the default minimum // version level. This can cause deprecation of all version levels in the closed range: - // [existingVersionRange.min(), defaultMinVersionLevel - 1] + // [existingVersionRange.min(), defaultMinVersionLevel - 1], unless it was previously + // deprecated. val defaultMinVersionLevel = brokerFeatures.defaultMinVersionLevel(update.name) newFeatures += ( update.name -> new FinalizedVersionRange( From afa3ab951dc92586165eca6baa7faa939e54deff Mon Sep 17 00:00:00 2001 From: Kowshik Prakasam Date: Tue, 14 Jul 2020 11:25:32 -0700 Subject: [PATCH 05/41] Fix small bug --- .../scala/kafka/controller/KafkaController.scala | 10 ++++++---- .../scala/kafka/server/FinalizedFeatureCache.scala | 13 ------------- .../server/FinalizedFeatureChangeListenerTest.scala | 8 -------- 3 files changed, 6 insertions(+), 25 deletions(-) diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index dce47502e0f9f..d750cb4661ad0 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -404,20 +404,22 @@ class KafkaController(val config: KafkaConfig, * 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: when this method returns, existing finalized features (if any) will be cleared from the - * FeatureZNode. + * 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 caceh (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) - featureCache.waitUntilEmptyOrThrow(config.zkConnectionTimeoutMs) } else { val existingFeatureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get) if (!existingFeatureZNode.status.equals(FeatureZNodeStatus.Disabled)) { updateFeatureZNode(newNode) - featureCache.waitUntilEmptyOrThrow(config.zkConnectionTimeoutMs) } } } diff --git a/core/src/main/scala/kafka/server/FinalizedFeatureCache.scala b/core/src/main/scala/kafka/server/FinalizedFeatureCache.scala index c648755ecdc91..8e5cac3d8b5ed 100644 --- a/core/src/main/scala/kafka/server/FinalizedFeatureCache.scala +++ b/core/src/main/scala/kafka/server/FinalizedFeatureCache.scala @@ -76,19 +76,6 @@ class FinalizedFeatureCache(private val brokerFeatures: BrokerFeatures) extends timeoutMs) } - /** - * Waits no more than timeoutMs for the cache to become empty. - * - * @param timeoutMs the timeout (in milli seconds) - * - * @throws TimeoutException if the cache's epoch has not become empty within timeoutMs. - */ - def waitUntilEmptyOrThrow(timeoutMs: Long): Unit = { - waitUntilConditionOrThrow( - () => featuresAndEpoch.isEmpty, - timeoutMs) - } - /** * Clears all existing finalized features and epoch from the cache. */ diff --git a/core/src/test/scala/unit/kafka/server/FinalizedFeatureChangeListenerTest.scala b/core/src/test/scala/unit/kafka/server/FinalizedFeatureChangeListenerTest.scala index d8b6bf6275c44..5f0141a975ce3 100644 --- a/core/src/test/scala/unit/kafka/server/FinalizedFeatureChangeListenerTest.scala +++ b/core/src/test/scala/unit/kafka/server/FinalizedFeatureChangeListenerTest.scala @@ -128,7 +128,6 @@ class FinalizedFeatureChangeListenerTest extends ZooKeeperTestHarness { val brokerFeatures = createBrokerFeatures() val cache = new FinalizedFeatureCache(brokerFeatures) val initialFinalizedFeatures = createFinalizedFeatures() - val listener = createListener(cache, Some(initialFinalizedFeatures)) val updatedFinalizedFeaturesMap = Map[String, FinalizedVersionRange]() val updatedFinalizedFeatures = Features.finalizedFeatures(updatedFinalizedFeaturesMap.asJava) @@ -137,10 +136,7 @@ class FinalizedFeatureChangeListenerTest extends ZooKeeperTestHarness { assertNotEquals(updatedVersion, ZkVersion.UnknownVersion) assertFalse(mayBeFeatureZNodeNewBytes.isEmpty) assertTrue(updatedVersion > initialFinalizedFeatures.epoch) - - cache.waitUntilEmptyOrThrow(JTestUtils.DEFAULT_MAX_WAIT_MS) assertTrue(cache.get.isEmpty) - assertTrue(listener.isListenerInitiated) } /** @@ -157,9 +153,6 @@ class FinalizedFeatureChangeListenerTest extends ZooKeeperTestHarness { assertThrows( classOf[TimeoutException], () => cache.waitUntilEpochOrThrow(initialFinalizedFeatures.epoch + 1, JTestUtils.DEFAULT_MAX_WAIT_MS)) - assertThrows( - classOf[TimeoutException], - () => cache.waitUntilEmptyOrThrow(JTestUtils.DEFAULT_MAX_WAIT_MS)) val updatedFinalizedFeaturesMap = Map[String, FinalizedVersionRange]() val updatedFinalizedFeatures = Features.finalizedFeatures(updatedFinalizedFeaturesMap.asJava) @@ -172,7 +165,6 @@ class FinalizedFeatureChangeListenerTest extends ZooKeeperTestHarness { assertThrows( classOf[TimeoutException], () => cache.waitUntilEpochOrThrow(updatedVersion, JTestUtils.DEFAULT_MAX_WAIT_MS)) - cache.waitUntilEmptyOrThrow(JTestUtils.DEFAULT_MAX_WAIT_MS) assertTrue(cache.get.isEmpty) assertTrue(listener.isListenerInitiated) } From 59d80385ea222eca250b6459cc53212770353109 Mon Sep 17 00:00:00 2001 From: Kowshik Prakasam Date: Fri, 17 Jul 2020 10:26:18 -0700 Subject: [PATCH 06/41] Minor: improve one of the tests slightly to handle +1 case --- .../FinalizedFeatureChangeListenerTest.scala | 33 ++++++++++++------- 1 file changed, 22 insertions(+), 11 deletions(-) diff --git a/core/src/test/scala/unit/kafka/server/FinalizedFeatureChangeListenerTest.scala b/core/src/test/scala/unit/kafka/server/FinalizedFeatureChangeListenerTest.scala index 5f0141a975ce3..1d4bee4c902dd 100644 --- a/core/src/test/scala/unit/kafka/server/FinalizedFeatureChangeListenerTest.scala +++ b/core/src/test/scala/unit/kafka/server/FinalizedFeatureChangeListenerTest.scala @@ -76,6 +76,8 @@ 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 = { @@ -84,18 +86,27 @@ class FinalizedFeatureChangeListenerTest extends ZooKeeperTestHarness { 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) + 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(updatedFinalizedFeatures, updatedVersion), cache.get.get) - assertTrue(listener.isListenerInitiated) + cache.waitUntilEpochOrThrow(updatedVersion, JTestUtils.DEFAULT_MAX_WAIT_MS) + assertEquals(FinalizedFeaturesAndEpoch(finalizedFeatures, updatedVersion), cache.get.get) + assertTrue(listener.isListenerInitiated) + } + + updateAndCheckCache( + Features.finalizedFeatures( + Map[String, FinalizedVersionRange]( + "feature_1" -> new FinalizedVersionRange(2, 4)).asJava)) + updateAndCheckCache( + Features.finalizedFeatures( + Map[String, FinalizedVersionRange]( + "feature_1" -> new FinalizedVersionRange(2, 4), + "feature_2" -> new FinalizedVersionRange(1, 3)).asJava)) } /** From 9cdfc3134b217df5adf90947bbff65662d822d68 Mon Sep 17 00:00:00 2001 From: Kowshik Prakasam Date: Tue, 21 Jul 2020 02:53:55 -0700 Subject: [PATCH 07/41] Address comments from Boyang --- .../org/apache/kafka/clients/admin/Admin.java | 27 +-- .../kafka/clients/admin/FeatureMetadata.java | 13 +- ...dFeatureUpdate.java => FeatureUpdate.java} | 44 ++--- .../kafka/clients/admin/KafkaAdminClient.java | 72 +++++--- ...ptions.java => UpdateFeaturesOptions.java} | 4 +- ...sResult.java => UpdateFeaturesResult.java} | 4 +- ...java => FeatureUpdateFailedException.java} | 6 +- .../apache/kafka/common/protocol/ApiKeys.java | 11 +- .../apache/kafka/common/protocol/Errors.java | 4 +- .../common/requests/AbstractRequest.java | 4 +- .../common/requests/AbstractResponse.java | 4 +- ...equest.java => UpdateFeaturesRequest.java} | 44 ++--- ...ponse.java => UpdateFeaturesResponse.java} | 33 ++-- ...equest.json => UpdateFeaturesRequest.json} | 6 +- ...ponse.json => UpdateFeaturesResponse.json} | 2 +- .../clients/admin/KafkaAdminClientTest.java | 54 +++--- .../kafka/clients/admin/MockAdminClient.java | 2 +- .../kafka/controller/ControllerState.scala | 4 +- .../kafka/controller/KafkaController.scala | 101 ++++++---- .../scala/kafka/server/BrokerFeatures.scala | 24 ++- .../kafka/server/FinalizedFeatureCache.scala | 10 +- .../main/scala/kafka/server/KafkaApis.scala | 173 +++++++++--------- ...resTest.scala => UpdateFeaturesTest.scala} | 100 +++++----- 23 files changed, 385 insertions(+), 361 deletions(-) rename clients/src/main/java/org/apache/kafka/clients/admin/{FinalizedFeatureUpdate.java => FeatureUpdate.java} (62%) rename clients/src/main/java/org/apache/kafka/clients/admin/{UpdateFinalizedFeaturesOptions.java => UpdateFeaturesOptions.java} (86%) rename clients/src/main/java/org/apache/kafka/clients/admin/{UpdateFinalizedFeaturesResult.java => UpdateFeaturesResult.java} (90%) rename clients/src/main/java/org/apache/kafka/common/errors/{FinalizedFeatureUpdateFailedException.java => FeatureUpdateFailedException.java} (81%) rename clients/src/main/java/org/apache/kafka/common/requests/{UpdateFinalizedFeaturesRequest.java => UpdateFeaturesRequest.java} (53%) rename clients/src/main/java/org/apache/kafka/common/requests/{UpdateFinalizedFeaturesResponse.java => UpdateFeaturesResponse.java} (59%) rename clients/src/main/resources/common/message/{UpdateFinalizedFeaturesRequest.json => UpdateFeaturesRequest.json} (87%) rename clients/src/main/resources/common/message/{UpdateFinalizedFeaturesResponse.json => UpdateFeaturesResponse.json} (96%) rename core/src/test/scala/unit/kafka/server/{UpdateFinalizedFeaturesTest.scala => UpdateFeaturesTest.scala} (83%) 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 091ea30910044..45bae8538bc15 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; @@ -1308,8 +1309,9 @@ AlterUserScramCredentialsResult alterUserScramCredentials(List * The following exceptions can be anticipated when calling {@code get()} on the future from the * returned {@link DescribeFeaturesResult}: @@ -1330,22 +1332,22 @@ AlterUserScramCredentialsResult alterUserScramCredentials(List - * The API takes as input a set of FinalizedFeatureUpdate that need to be applied. Each such - * update specifies the finalized feature to be added or updated or deleted, along with the new - * max feature version level value. + * The API takes in a set of feature updates that need to be applied. Each such update specifies + * the finalized feature to be added or updated or deleted, along with the new max feature + * version level value. *

*

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

@@ -1365,10 +1367,9 @@ AlterUserScramCredentialsResult alterUserScramCredentials(List featureUpdates, UpdateFinalizedFeaturesOptions options); + UpdateFeaturesResult updateFeatures(Set featureUpdates, UpdateFeaturesOptions options); /** * Get the metrics kept by the adminClient 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 index 149df5f99b203..8cdfcd75323f9 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/FeatureMetadata.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/FeatureMetadata.java @@ -17,6 +17,7 @@ package org.apache.kafka.clients.admin; import java.util.Objects; +import java.util.Optional; import org.apache.kafka.common.feature.Features; import org.apache.kafka.common.feature.FinalizedVersionRange; import org.apache.kafka.common.feature.SupportedVersionRange; @@ -29,7 +30,7 @@ public class FeatureMetadata { private final Features finalizedFeatures; - private final int finalizedFeaturesEpoch; + private final Optional finalizedFeaturesEpoch; private final Features supportedFeatures; @@ -41,7 +42,11 @@ public FeatureMetadata( Objects.requireNonNull(finalizedFeatures, "Provided finalizedFeatures can not be null."); Objects.requireNonNull(supportedFeatures, "Provided supportedFeatures can not be null."); this.finalizedFeatures = finalizedFeatures; - this.finalizedFeaturesEpoch = finalizedFeaturesEpoch; + if (finalizedFeaturesEpoch >= 0) { + this.finalizedFeaturesEpoch = Optional.of(finalizedFeaturesEpoch); + } else { + this.finalizedFeaturesEpoch = Optional.empty(); + } this.supportedFeatures = supportedFeatures; } @@ -55,9 +60,9 @@ public Features finalizedFeatures() { /** * The epoch for the finalized features. - * Valid values are >= 0. A value < 0 means the finalized features are absent/unavailable. + * If the returned value is empty, it means the finalized features are absent/unavailable. */ - public int finalizedFeaturesEpoch() { + public Optional finalizedFeaturesEpoch() { return finalizedFeaturesEpoch; } diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/FinalizedFeatureUpdate.java b/clients/src/main/java/org/apache/kafka/clients/admin/FeatureUpdate.java similarity index 62% rename from clients/src/main/java/org/apache/kafka/clients/admin/FinalizedFeatureUpdate.java rename to clients/src/main/java/org/apache/kafka/clients/admin/FeatureUpdate.java index fcd4599a4f04a..00dfea96b88da 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/FinalizedFeatureUpdate.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/FeatureUpdate.java @@ -18,14 +18,13 @@ import java.util.Objects; import java.util.Set; -import org.apache.kafka.common.message.UpdateFinalizedFeaturesRequestData; +import org.apache.kafka.common.message.UpdateFeaturesRequestData; /** * Encapsulates details about an update to a finalized feature. This is particularly useful to - * define each feature update in the - * {@link Admin#updateFinalizedFeatures(Set, UpdateFinalizedFeaturesOptions)} API request. + * define each feature update in the {@link Admin#updateFeatures(Set, UpdateFeaturesOptions)} API. */ -public class FinalizedFeatureUpdate { +public class FeatureUpdate { private final String featureName; private final short maxVersionLevel; private final boolean allowDowngrade; @@ -40,8 +39,7 @@ public class FinalizedFeatureUpdate { * maximum version level of the finalized feature. * - false, otherwise. */ - public FinalizedFeatureUpdate( - final String featureName, final short maxVersionLevel, final boolean allowDowngrade) { + public FeatureUpdate(final String featureName, final short maxVersionLevel, final boolean allowDowngrade) { Objects.requireNonNull(featureName, "Provided feature name can not be null."); if (maxVersionLevel < 1 && !allowDowngrade) { throw new IllegalArgumentException( @@ -54,50 +52,36 @@ public FinalizedFeatureUpdate( this.allowDowngrade = allowDowngrade; } - /** - * @return the name of the finalized feature to be updated. - */ public String featureName() { return featureName; } - /** - * @return the new maximum version level for the finalized feature. - */ public short maxVersionLevel() { return maxVersionLevel; } - /** - * @return - true, if this feature update was meant to downgrade the maximum version level of - * the finalized feature. - * - false, otherwise. - */ public boolean allowDowngrade() { return allowDowngrade; } /** - * Helper function that creates {@link UpdateFinalizedFeaturesRequestData} from a set of - * {@link FinalizedFeatureUpdate}. + * Helper function that creates {@link UpdateFeaturesRequestData} from a set of {@link FeatureUpdate}. * - * @param updates the set of {@link FinalizedFeatureUpdate} + * @param updates the set of {@link FeatureUpdate} * - * @return a newly constructed UpdateFinalizedFeaturesRequestData object + * @return a newly constructed UpdateFeaturesRequestData object */ - public static UpdateFinalizedFeaturesRequestData createRequest(Set updates) { - final UpdateFinalizedFeaturesRequestData.FinalizedFeatureUpdateKeyCollection items - = new UpdateFinalizedFeaturesRequestData.FinalizedFeatureUpdateKeyCollection(); - for (FinalizedFeatureUpdate update : updates) { - final UpdateFinalizedFeaturesRequestData.FinalizedFeatureUpdateKey item = - new UpdateFinalizedFeaturesRequestData.FinalizedFeatureUpdateKey(); + public static UpdateFeaturesRequestData createRequest(Set updates) { + final UpdateFeaturesRequestData.FeatureUpdateKeyCollection items + = new UpdateFeaturesRequestData.FeatureUpdateKeyCollection(); + for (FeatureUpdate update : updates) { + final UpdateFeaturesRequestData.FeatureUpdateKey item = + new UpdateFeaturesRequestData.FeatureUpdateKey(); item.setName(update.featureName()); item.setMaxVersionLevel(update.maxVersionLevel()); item.setAllowDowngrade(update.allowDowngrade()); items.add(item); } - final UpdateFinalizedFeaturesRequestData data = new UpdateFinalizedFeaturesRequestData(); - data.setFinalizedFeatureUpdates(items); - return data; + return new UpdateFeaturesRequestData().setFeatureUpdates(items); } } 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 4c0bcabf9e9dd..1f3d78366bda9 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 @@ -228,8 +228,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.UpdateFinalizedFeaturesRequest; -import org.apache.kafka.common.requests.UpdateFinalizedFeaturesResponse; +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; @@ -4342,6 +4342,7 @@ private static byte[] getSaltedPasword(ScramMechanism publicScramMechanism, byte public DescribeFeaturesResult describeFeatures(final DescribeFeaturesOptions options) { final KafkaFutureImpl future = new KafkaFutureImpl<>(); final long now = time.milliseconds(); + Call callViaLeastLoadedNode = new Call("describeFeatures", calcDeadlineMs(now, options.timeoutMs()), new LeastLoadedNodeProvider()) { @@ -4371,39 +4372,50 @@ void handleFailure(Throwable throwable) { } }; - Call call = callViaLeastLoadedNode; - if (options.sendRequestToController()) { - call = new Call("describeFeatures", calcDeadlineMs(now, options.timeoutMs()), - new ControllerNodeProvider()) { + Call callViaControllerNode = new Call("describeFeatures", calcDeadlineMs(now, options.timeoutMs()), + new ControllerNodeProvider()) { - @Override - ApiVersionsRequest.Builder createRequest(int timeoutMs) { - return (ApiVersionsRequest.Builder) callViaLeastLoadedNode.createRequest(timeoutMs); - } + @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.NOT_CONTROLLER.code()) { - handleNotControllerError(Errors.NOT_CONTROLLER); - } else { - callViaLeastLoadedNode.handleResponse(response); - } + @Override + void handleResponse(AbstractResponse response) { + final ApiVersionsResponse apiVersionsResponse = (ApiVersionsResponse) response; + if (apiVersionsResponse.data.errorCode() == Errors.NONE.code()) { + future.complete( + new FeatureMetadata( + apiVersionsResponse.finalizedFeatures(), + apiVersionsResponse.finalizedFeaturesEpoch(), + apiVersionsResponse.supportedFeatures())); + } else if (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) { - callViaLeastLoadedNode.handleFailure(throwable); - } - }; + @Override + void handleFailure(Throwable throwable) { + completeAllExceptionally(Collections.singletonList(future), throwable); + } + }; + + Call call; + if (options.sendRequestToController()) { + call = callViaControllerNode; + } else { + call = callViaLeastLoadedNode; } runnable.call(call, now); return new DescribeFeaturesResult(future); } @Override - public UpdateFinalizedFeaturesResult updateFinalizedFeatures( - final Set featureUpdates, final UpdateFinalizedFeaturesOptions options) { + public UpdateFeaturesResult updateFeatures( + final Set featureUpdates, final UpdateFeaturesOptions options) { final KafkaFutureImpl future = new KafkaFutureImpl<>(); final long now = time.milliseconds(); @@ -4411,14 +4423,14 @@ public UpdateFinalizedFeaturesResult updateFinalizedFeatures( new ControllerNodeProvider()) { @Override - UpdateFinalizedFeaturesRequest.Builder createRequest(int timeoutMs) { - return new UpdateFinalizedFeaturesRequest.Builder(FinalizedFeatureUpdate.createRequest(featureUpdates)); + UpdateFeaturesRequest.Builder createRequest(int timeoutMs) { + return new UpdateFeaturesRequest.Builder(FeatureUpdate.createRequest(featureUpdates)); } @Override void handleResponse(AbstractResponse response) { - final UpdateFinalizedFeaturesResponse featuresResponse = - (UpdateFinalizedFeaturesResponse) response; + final UpdateFeaturesResponse featuresResponse = + (UpdateFeaturesResponse) response; final Errors error = Errors.forCode(featuresResponse.data().errorCode()); if (error == Errors.NONE) { future.complete(null); @@ -4436,7 +4448,7 @@ void handleFailure(Throwable throwable) { } }; runnable.call(call, now); - return new UpdateFinalizedFeaturesResult(future); + return new UpdateFeaturesResult(future); } /** diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/UpdateFinalizedFeaturesOptions.java b/clients/src/main/java/org/apache/kafka/clients/admin/UpdateFeaturesOptions.java similarity index 86% rename from clients/src/main/java/org/apache/kafka/clients/admin/UpdateFinalizedFeaturesOptions.java rename to clients/src/main/java/org/apache/kafka/clients/admin/UpdateFeaturesOptions.java index 6f615665ee0f7..912918d6391da 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/UpdateFinalizedFeaturesOptions.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/UpdateFeaturesOptions.java @@ -19,12 +19,12 @@ import org.apache.kafka.common.annotation.InterfaceStability; @InterfaceStability.Evolving -public class UpdateFinalizedFeaturesOptions extends AbstractOptions { +public class UpdateFeaturesOptions extends AbstractOptions { /** * Sets the timeout in milliseconds for this operation or {@code null} if the default API * timeout for the AdminClient should be used. */ - public UpdateFinalizedFeaturesOptions timeoutMs(Integer timeoutMs) { + public UpdateFeaturesOptions timeoutMs(Integer timeoutMs) { this.timeoutMs = timeoutMs; return this; } diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/UpdateFinalizedFeaturesResult.java b/clients/src/main/java/org/apache/kafka/clients/admin/UpdateFeaturesResult.java similarity index 90% rename from clients/src/main/java/org/apache/kafka/clients/admin/UpdateFinalizedFeaturesResult.java rename to clients/src/main/java/org/apache/kafka/clients/admin/UpdateFeaturesResult.java index e9dbffe9520cb..0fec1ded816b4 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/UpdateFinalizedFeaturesResult.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/UpdateFeaturesResult.java @@ -18,10 +18,10 @@ import org.apache.kafka.common.KafkaFuture; -public class UpdateFinalizedFeaturesResult { +public class UpdateFeaturesResult { private final KafkaFuture future; - public UpdateFinalizedFeaturesResult(KafkaFuture future) { + public UpdateFeaturesResult(KafkaFuture future) { this.future = future; } diff --git a/clients/src/main/java/org/apache/kafka/common/errors/FinalizedFeatureUpdateFailedException.java b/clients/src/main/java/org/apache/kafka/common/errors/FeatureUpdateFailedException.java similarity index 81% rename from clients/src/main/java/org/apache/kafka/common/errors/FinalizedFeatureUpdateFailedException.java rename to clients/src/main/java/org/apache/kafka/common/errors/FeatureUpdateFailedException.java index 174f8bc39a21c..8ad5da2dbf564 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/FinalizedFeatureUpdateFailedException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/FeatureUpdateFailedException.java @@ -16,14 +16,14 @@ */ package org.apache.kafka.common.errors; -public class FinalizedFeatureUpdateFailedException extends ApiException { +public class FeatureUpdateFailedException extends ApiException { private static final long serialVersionUID = 1L; - public FinalizedFeatureUpdateFailedException(String message) { + public FeatureUpdateFailedException(String message) { super(message); } - public FinalizedFeatureUpdateFailedException(String message, Throwable cause) { + public FeatureUpdateFailedException(String message, Throwable cause) { super(message, cause); } } 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 37ac45a1baa7b..183ea59da4d47 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,8 +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.UpdateFinalizedFeaturesRequestData; -import org.apache.kafka.common.message.UpdateFinalizedFeaturesResponseData; +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; @@ -249,10 +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); - DescribeQuorumRequestData.SCHEMAS, DescribeQuorumResponseData.SCHEMAS), - UPDATE_FINALIZED_FEATURES(57, "UpdateFinalizedFeatures", - UpdateFinalizedFeaturesRequestData.SCHEMAS, UpdateFinalizedFeaturesResponseData.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 36a84d41a2405..54eb17bab754a 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,9 +35,9 @@ 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.FinalizedFeatureUpdateFailedException; import org.apache.kafka.common.errors.FetchSessionIdNotFoundException; import org.apache.kafka.common.errors.GroupAuthorizationException; import org.apache.kafka.common.errors.GroupIdNotFoundException; @@ -338,7 +338,7 @@ public enum Errors { 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), - FINALIZED_FEATURE_UPDATE_FAILED(96, "Unable to update finalized features.", FinalizedFeatureUpdateFailedException::new); + FINALIZED_FEATURE_UPDATE_FAILED(96, "Unable to update finalized features.", 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 d29fe46da80f4..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,8 +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_FINALIZED_FEATURES: - return new UpdateFinalizedFeaturesRequest(struct, 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 15d724e521b98..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,8 +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_FINALIZED_FEATURES: - return new UpdateFinalizedFeaturesResponse(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/UpdateFinalizedFeaturesRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesRequest.java similarity index 53% rename from clients/src/main/java/org/apache/kafka/common/requests/UpdateFinalizedFeaturesRequest.java rename to clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesRequest.java index 19810507911f5..14da1961a5235 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/UpdateFinalizedFeaturesRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesRequest.java @@ -17,25 +17,25 @@ package org.apache.kafka.common.requests; import java.nio.ByteBuffer; -import org.apache.kafka.common.message.UpdateFinalizedFeaturesResponseData; -import org.apache.kafka.common.message.UpdateFinalizedFeaturesRequestData; +import org.apache.kafka.common.message.UpdateFeaturesResponseData; +import org.apache.kafka.common.message.UpdateFeaturesRequestData; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.types.Struct; -public class UpdateFinalizedFeaturesRequest extends AbstractRequest { +public class UpdateFeaturesRequest extends AbstractRequest { - public static class Builder extends AbstractRequest.Builder { + public static class Builder extends AbstractRequest.Builder { - private final UpdateFinalizedFeaturesRequestData data; + private final UpdateFeaturesRequestData data; - public Builder(UpdateFinalizedFeaturesRequestData data) { - super(ApiKeys.UPDATE_FINALIZED_FEATURES); + public Builder(UpdateFeaturesRequestData data) { + super(ApiKeys.UPDATE_FEATURES); this.data = data; } @Override - public UpdateFinalizedFeaturesRequest build(short version) { - return new UpdateFinalizedFeaturesRequest(data, version); + public UpdateFeaturesRequest build(short version) { + return new UpdateFeaturesRequest(data, version); } @Override @@ -44,23 +44,23 @@ public String toString() { } } - public final UpdateFinalizedFeaturesRequestData data; + public final UpdateFeaturesRequestData data; - public UpdateFinalizedFeaturesRequest(UpdateFinalizedFeaturesRequestData data, short version) { - super(ApiKeys.UPDATE_FINALIZED_FEATURES, version); + public UpdateFeaturesRequest(UpdateFeaturesRequestData data, short version) { + super(ApiKeys.UPDATE_FEATURES, version); this.data = data; } - public UpdateFinalizedFeaturesRequest(Struct struct, short version) { - super(ApiKeys.UPDATE_FINALIZED_FEATURES, version); - this.data = new UpdateFinalizedFeaturesRequestData(struct, version); + public UpdateFeaturesRequest(Struct struct, short version) { + super(ApiKeys.UPDATE_FEATURES, version); + this.data = new UpdateFeaturesRequestData(struct, version); } @Override public AbstractResponse getErrorResponse(int throttleTimeMsIgnored, Throwable e) { final ApiError apiError = ApiError.fromThrowable(e); - return new UpdateFinalizedFeaturesResponse( - new UpdateFinalizedFeaturesResponseData() + return new UpdateFeaturesResponse( + new UpdateFeaturesResponseData() .setErrorCode(apiError.error().code()) .setErrorMessage(apiError.message())); } @@ -70,16 +70,16 @@ protected Struct toStruct() { return data.toStruct(version()); } - public UpdateFinalizedFeaturesRequestData data() { + public UpdateFeaturesRequestData data() { return data; } - public static UpdateFinalizedFeaturesRequest parse(ByteBuffer buffer, short version) { - return new UpdateFinalizedFeaturesRequest( - ApiKeys.UPDATE_FINALIZED_FEATURES.parseRequest(version, buffer), version); + public static UpdateFeaturesRequest parse(ByteBuffer buffer, short version) { + return new UpdateFeaturesRequest( + ApiKeys.UPDATE_FEATURES.parseRequest(version, buffer), version); } - public static boolean isDeleteRequest(UpdateFinalizedFeaturesRequestData.FinalizedFeatureUpdateKey update) { + public static boolean isDeleteRequest(UpdateFeaturesRequestData.FeatureUpdateKey update) { return update.maxVersionLevel() < 1; } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/UpdateFinalizedFeaturesResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesResponse.java similarity index 59% rename from clients/src/main/java/org/apache/kafka/common/requests/UpdateFinalizedFeaturesResponse.java rename to clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesResponse.java index b00377701a674..d41bc809a40f9 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/UpdateFinalizedFeaturesResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesResponse.java @@ -18,26 +18,35 @@ import java.nio.ByteBuffer; import java.util.Map; -import org.apache.kafka.common.message.UpdateFinalizedFeaturesResponseData; +import org.apache.kafka.common.message.UpdateFeaturesResponseData; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.types.Struct; -public class UpdateFinalizedFeaturesResponse extends AbstractResponse { - public final UpdateFinalizedFeaturesResponseData data; +/** + * 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 { + + public final UpdateFeaturesResponseData data; - public UpdateFinalizedFeaturesResponse(UpdateFinalizedFeaturesResponseData data) { + public UpdateFeaturesResponse(UpdateFeaturesResponseData data) { this.data = data; } - public UpdateFinalizedFeaturesResponse(Struct struct) { - final short latestVersion = (short) (UpdateFinalizedFeaturesResponseData.SCHEMAS.length - 1); - this.data = new UpdateFinalizedFeaturesResponseData(struct, latestVersion); + public UpdateFeaturesResponse(Struct struct) { + final short latestVersion = (short) (UpdateFeaturesResponseData.SCHEMAS.length - 1); + this.data = new UpdateFeaturesResponseData(struct, latestVersion); } - public UpdateFinalizedFeaturesResponse(Struct struct, short version) { - this.data = new UpdateFinalizedFeaturesResponseData(struct, version); + public UpdateFeaturesResponse(Struct struct, short version) { + this.data = new UpdateFeaturesResponseData(struct, version); } public Errors error() { @@ -59,11 +68,11 @@ public String toString() { return data.toString(); } - public UpdateFinalizedFeaturesResponseData data() { + public UpdateFeaturesResponseData data() { return data; } - public static UpdateFinalizedFeaturesResponse parse(ByteBuffer buffer, short version) { - return new UpdateFinalizedFeaturesResponse(ApiKeys.UPDATE_FINALIZED_FEATURES.parseResponse(version, buffer), version); + public static UpdateFeaturesResponse parse(ByteBuffer buffer, short version) { + return new UpdateFeaturesResponse(ApiKeys.UPDATE_FEATURES.parseResponse(version, buffer), version); } } diff --git a/clients/src/main/resources/common/message/UpdateFinalizedFeaturesRequest.json b/clients/src/main/resources/common/message/UpdateFeaturesRequest.json similarity index 87% rename from clients/src/main/resources/common/message/UpdateFinalizedFeaturesRequest.json rename to clients/src/main/resources/common/message/UpdateFeaturesRequest.json index c7ed42642e6a0..6a134e8aab2a1 100644 --- a/clients/src/main/resources/common/message/UpdateFinalizedFeaturesRequest.json +++ b/clients/src/main/resources/common/message/UpdateFeaturesRequest.json @@ -16,15 +16,15 @@ { "apiKey": 50, "type": "request", - "name": "UpdateFinalizedFeaturesRequest", + "name": "UpdateFeaturesRequest", "validVersions": "0", "flexibleVersions": "0+", "fields": [ - { "name": "FinalizedFeatureUpdates", "type": "[]FinalizedFeatureUpdateKey", "versions": "0+", + { "name": "FeatureUpdates", "type": "[]FeatureUpdateKey", "versions": "0+", "about": "The list of updates to finalized features.", "fields": [ {"name": "Name", "type": "string", "versions": "0+", "mapKey": true, "about": "The name of the finalized feature to be updated."}, - {"name": "MaxVersionLevel", "type": "int16", "versions": "0+", + {"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."} diff --git a/clients/src/main/resources/common/message/UpdateFinalizedFeaturesResponse.json b/clients/src/main/resources/common/message/UpdateFeaturesResponse.json similarity index 96% rename from clients/src/main/resources/common/message/UpdateFinalizedFeaturesResponse.json rename to clients/src/main/resources/common/message/UpdateFeaturesResponse.json index a018d03cb7076..05965db920f66 100644 --- a/clients/src/main/resources/common/message/UpdateFinalizedFeaturesResponse.json +++ b/clients/src/main/resources/common/message/UpdateFeaturesResponse.json @@ -16,7 +16,7 @@ { "apiKey": 50, "type": "response", - "name": "UpdateFinalizedFeaturesResponse", + "name": "UpdateFeaturesResponse", "validVersions": "0", "flexibleVersions": "0+", "fields": [ 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 e28bb0683024a..15f1f73844d43 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 @@ -115,7 +115,7 @@ import org.apache.kafka.common.message.OffsetDeleteResponseData.OffsetDeleteResponsePartitionCollection; import org.apache.kafka.common.message.OffsetDeleteResponseData.OffsetDeleteResponseTopic; import org.apache.kafka.common.message.OffsetDeleteResponseData.OffsetDeleteResponseTopicCollection; -import org.apache.kafka.common.message.UpdateFinalizedFeaturesResponseData; +import org.apache.kafka.common.message.UpdateFeaturesResponseData; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.quota.ClientQuotaAlteration; @@ -159,8 +159,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.UpdateFinalizedFeaturesRequest; -import org.apache.kafka.common.requests.UpdateFinalizedFeaturesResponse; +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; @@ -484,10 +484,10 @@ private static DescribeGroupsResponseData prepareDescribeGroupsResponseData(Stri return data; } - private static UpdateFinalizedFeaturesResponse prepareUpdateFinalizedFeaturesResponse(Errors error) { - final UpdateFinalizedFeaturesResponseData data = new UpdateFinalizedFeaturesResponseData(); + private static UpdateFeaturesResponse prepareUpdateFeaturesResponse(Errors error) { + final UpdateFeaturesResponseData data = new UpdateFeaturesResponseData(); data.setErrorCode(error.code()); - return new UpdateFinalizedFeaturesResponse(data); + return new UpdateFeaturesResponse(data); } private static FeatureMetadata getDefaultFeatureMetadata() { @@ -514,7 +514,7 @@ private static ApiVersionsResponse prepareApiVersionsResponseForDescribeFeatures ApiVersionsResponse.DEFAULT_API_VERSIONS_RESPONSE.data().apiKeys(), getDefaultFeatureMetadata().supportedFeatures(), getDefaultFeatureMetadata().finalizedFeatures(), - getDefaultFeatureMetadata().finalizedFeaturesEpoch())); + getDefaultFeatureMetadata().finalizedFeaturesEpoch().orElse(-1))); } final ApiVersionsResponseData data = new ApiVersionsResponseData(); data.setErrorCode(error.code()); @@ -3932,33 +3932,33 @@ public void testListOffsetsNonRetriableErrors() throws Exception { } @Test - public void testUpdateFinalizedFeaturesDuringSuccess() throws Exception { - testUpdateFinalizedFeaturesDuringError(Errors.NONE); + public void testUpdateFeaturesDuringSuccess() throws Exception { + testUpdateFeaturesDuringError(Errors.NONE); } @Test - public void testUpdateFinalizedFeaturesInvalidRequestError() throws Exception { - testUpdateFinalizedFeaturesDuringError(Errors.INVALID_REQUEST); + public void testUpdateFeaturesInvalidRequestError() throws Exception { + testUpdateFeaturesDuringError(Errors.INVALID_REQUEST); } @Test - public void testUpdateFinalizedFeaturesUpdateFailedError() throws Exception { - testUpdateFinalizedFeaturesDuringError(Errors.FINALIZED_FEATURE_UPDATE_FAILED); + public void testUpdateFeaturesUpdateFailedError() throws Exception { + testUpdateFeaturesDuringError(Errors.FEATURE_UPDATE_FAILED); } - private void testUpdateFinalizedFeaturesDuringError(Errors error) throws Exception { + private void testUpdateFeaturesDuringError(Errors error) throws Exception { try (final AdminClientUnitTestEnv env = mockClientEnv()) { env.kafkaClient().prepareResponse( - body -> body instanceof UpdateFinalizedFeaturesRequest, - prepareUpdateFinalizedFeaturesResponse(error)); - final KafkaFuture future = env.adminClient().updateFinalizedFeatures( + body -> body instanceof UpdateFeaturesRequest, + prepareUpdateFeaturesResponse(error)); + final KafkaFuture future = env.adminClient().updateFeatures( new HashSet<>( Arrays.asList( - new FinalizedFeatureUpdate( + new FeatureUpdate( "test_feature_1", (short) 2, false), - new FinalizedFeatureUpdate( + new FeatureUpdate( "test_feature_2", (short) 3, true))), - new UpdateFinalizedFeaturesOptions().timeoutMs(10000)).result(); + new UpdateFeaturesOptions().timeoutMs(10000)).result(); if (error.exception() == null) { future.get(); } else { @@ -3970,26 +3970,26 @@ private void testUpdateFinalizedFeaturesDuringError(Errors error) throws Excepti } @Test - public void testUpdateFinalizedFeaturesHandleNotControllerException() throws Exception { + public void testUpdateFeaturesHandleNotControllerException() throws Exception { try (final AdminClientUnitTestEnv env = mockClientEnv()) { env.kafkaClient().prepareResponseFrom( - prepareUpdateFinalizedFeaturesResponse(Errors.NOT_CONTROLLER), + prepareUpdateFeaturesResponse(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( - prepareUpdateFinalizedFeaturesResponse(Errors.NONE), + prepareUpdateFeaturesResponse(Errors.NONE), env.cluster().nodeById(1)); - final KafkaFuture future = env.adminClient().updateFinalizedFeatures( + final KafkaFuture future = env.adminClient().updateFeatures( new HashSet<>( Arrays.asList( - new FinalizedFeatureUpdate( + new FeatureUpdate( "test_feature_1", (short) 2, false), - new FinalizedFeatureUpdate( + new FeatureUpdate( "test_feature_2", (short) 3, true))), - new UpdateFinalizedFeaturesOptions().timeoutMs(10000)).result(); + new UpdateFeaturesOptions().timeoutMs(10000)).result(); future.get(); } } 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 0582a15c7204a..86e160ebc9323 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 @@ -842,7 +842,7 @@ public DescribeFeaturesResult describeFeatures(DescribeFeaturesOptions options) } @Override - public UpdateFinalizedFeaturesResult updateFinalizedFeatures(Set featureUpdates, UpdateFinalizedFeaturesOptions options) { + public UpdateFeaturesResult updateFeatures(Set featureUpdates, UpdateFeaturesOptions options) { throw new UnsupportedOperationException("Not implemented yet"); } diff --git a/core/src/main/scala/kafka/controller/ControllerState.scala b/core/src/main/scala/kafka/controller/ControllerState.scala index 8412fa375c8e9..f84240536dd85 100644 --- a/core/src/main/scala/kafka/controller/ControllerState.scala +++ b/core/src/main/scala/kafka/controller/ControllerState.scala @@ -110,7 +110,7 @@ object ControllerState { override protected def hasRateAndTimeMetric: Boolean = false } - case object UpdateFinalizedFeatures extends ControllerState { + case object UpdateFeatures extends ControllerState { def value = 17 } @@ -118,5 +118,5 @@ object ControllerState { AlterPartitionReassignment, AutoLeaderBalance, ManualLeaderBalance, ControlledShutdown, IsrChange, LeaderAndIsrResponseReceived, LogDirChange, ControllerShutdown, UncleanLeaderElectionEnable, TopicUncleanLeaderElectionEnable, ListPartitionReassignment, UpdateMetadataResponseReceived, - UpdateFinalizedFeatures) + UpdateFeatures) } diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index d750cb4661ad0..6009c775dba0a 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -24,7 +24,7 @@ import kafka.api._ import kafka.common._ import kafka.controller.KafkaController.{AlterIsrCallback, AlterReassignmentsCallback, ElectLeadersCallback, ListReassignmentsCallback} import kafka.cluster.Broker -import kafka.controller.KafkaController.{AlterReassignmentsCallback, ElectLeadersCallback, ListReassignmentsCallback, UpdateFinalizedFeaturesCallback} +import kafka.controller.KafkaController.{AlterReassignmentsCallback, ElectLeadersCallback, ListReassignmentsCallback, UpdateFeaturesCallback} import kafka.metrics.{KafkaMetricsGroup, KafkaTimer} import kafka.server._ import kafka.utils._ @@ -63,7 +63,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 UpdateFinalizedFeaturesCallback = (Errors, Option[String]) => Unit + type UpdateFeaturesCallback = (Errors, Option[String]) => Unit } class KafkaController(val config: KafkaConfig, @@ -293,22 +293,42 @@ class KafkaController(val config: KafkaConfig, } /** - * Enables the feature versioning system (KIP-584). + * This method enables the feature versioning system (KIP-584). * - * Sets up the FeatureZNode with enabled status. This status means the feature versioning system - * (KIP-584) is enabled, and, the finalized features stored in the FeatureZNode are active. This - * 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. + * 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 + * {@link SupportedVersionRange}). It refers to a feature that a particular broker advertises + * support for. Each broker advertises the version ranges of it’s 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 is represented by a name (String) and a range of version levels (defined + * by a {@link FinalizedVersionRange}). Whenever the feature versioning system (KIP-584) is + * enabled, the finalized features are stored in ZK 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 one and only entity modifying + * the information about finalized features and their version levels. + * + * This method sets up the FeatureZNode with enabled status. This status means the feature + * versioning system (KIP-584) is enabled, and, the finalized features stored in the FeatureZNode + * are active. This 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: - * 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. Assuming this is the - * case, then here is how we it: 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 default supported features as its finalized features. + * 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 + * default supported features as its finalized features. * * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0: * Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the @@ -366,23 +386,24 @@ class KafkaController(val config: KafkaConfig, if (existingFeatureZNode.status.equals(FeatureZNodeStatus.Enabled)) { newFeatures = Features.finalizedFeatures(existingFeatureZNode.features.features().asScala.map { case (featureName, existingVersionRange) => { - val updatedVersionRange = defaultFinalizedFeatures.get(featureName) - if (updatedVersionRange == null) { + val brokerDefaultVersionRange = defaultFinalizedFeatures.get(featureName) + if (brokerDefaultVersionRange == null) { warn(s"Existing finalized feature: $featureName with $existingVersionRange" + s" is absent in default finalized $defaultFinalizedFeatures") (featureName, existingVersionRange) - } else if (existingVersionRange.max() >= updatedVersionRange.min()) { + } else if (existingVersionRange.max() >= brokerDefaultVersionRange.min() && + brokerDefaultVersionRange.max() >= existingVersionRange.max()) { // Through this change, we deprecate all version levels in the closed range: - // [existingVersionRange.min(), updatedVersionRange.min() - 1] - (featureName, new FinalizedVersionRange(updatedVersionRange.min(), existingVersionRange.max())) + // [existingVersionRange.min(), brokerDefaultVersionRange.min() - 1] + (featureName, new FinalizedVersionRange(brokerDefaultVersionRange.min(), existingVersionRange.max())) } else { - // This is a special case: If the existing version levels fall completely outside the - // range of the default finalized version levels (i.e. no intersection), then, this - // case is not eligible for deprecation. This requires that the max version level be - // upgraded first to a value that's equal to the the default minimum version level. - info(s"Can not update minimum version level in finalized feature: $featureName," - + s" since the existing $existingVersionRange does not intersect with the default" - + s" $updatedVersionRange.") + // If the existing version levels fall completely outside the + // range of the default finalized version levels (i.e. no intersection), or, if the + // existing version levels are ineligible for a modification since they are + // incompatible with default finalized version levels, then we skip the update. + warn(s"Can not update minimum version level in finalized feature: $featureName," + + s" since the existing $existingVersionRange is not eligible for a change" + + s" based on the default $brokerDefaultVersionRange.") (featureName, existingVersionRange) } } @@ -1150,7 +1171,9 @@ class KafkaController(val config: KafkaConfig, /** * Send the leader information for selected partitions to selected brokers so that they can correctly respond to - * metadata requests + * metadata requests. Particularly, when feature versioning is enabled, we filter out brokers with incompatible + * features from receiving the metadata requests. This is because we do not want to activate incompatible brokers, + * as these may have harmful consequences to the cluster. * * @param brokers The brokers that the update metadata request should be sent to */ @@ -1833,13 +1856,13 @@ class KafkaController(val config: KafkaConfig, } } - private def processUpdateFinalizedFeatures(newFeatures: Features[FinalizedVersionRange], - callback: UpdateFinalizedFeaturesCallback): Unit = { + private def processUpdateFeatures(newFeatures: Features[FinalizedVersionRange], + callback: UpdateFeaturesCallback): Unit = { if (isActive) { val incompatibleBrokers = controllerContext.liveOrShuttingDownBrokers.filter(broker => { BrokerFeatures.hasIncompatibleFeatures(broker.features, newFeatures) }) - if (incompatibleBrokers.size > 0) { + if (incompatibleBrokers.nonEmpty) { callback( Errors.INVALID_REQUEST, Some( @@ -1854,8 +1877,8 @@ class KafkaController(val config: KafkaConfig, callback(Errors.NONE, Option.empty) } catch { case e: Exception => callback( - Errors.FINALIZED_FEATURE_UPDATE_FAILED, - Some(Errors.FINALIZED_FEATURE_UPDATE_FAILED.message() + " Error: " + e)) + Errors.FEATURE_UPDATE_FAILED, + Some(Errors.FEATURE_UPDATE_FAILED.message() + " Error: " + e)) } } } else { @@ -1897,9 +1920,9 @@ class KafkaController(val config: KafkaConfig, eventManager.put(ListPartitionReassignments(partitions, callback)) } - def updateFinalizedFeatures(newFeatures: Features[FinalizedVersionRange], - callback: UpdateFinalizedFeaturesCallback): Unit = { - eventManager.put(UpdateFinalizedFeatures(newFeatures, callback)) + def updateFeatures(newFeatures: Features[FinalizedVersionRange], + callback: UpdateFeaturesCallback): Unit = { + eventManager.put(UpdateFeatures(newFeatures, callback)) } def alterPartitionReassignments(partitions: Map[TopicPartition, Option[Seq[Int]]], @@ -2191,8 +2214,8 @@ class KafkaController(val config: KafkaConfig, processZkPartitionReassignment() case ListPartitionReassignments(partitions, callback) => processListPartitionReassignments(partitions, callback) - case UpdateFinalizedFeatures(request, callback) => - processUpdateFinalizedFeatures(request, callback) + case UpdateFeatures(request, callback) => + processUpdateFeatures(request, callback) case PartitionReassignmentIsrChange(partition) => processPartitionReassignmentIsrChange(partition) case IsrChangeNotification => @@ -2486,9 +2509,9 @@ case class ListPartitionReassignments(partitionsOpt: Option[Set[TopicPartition]] override def preempt(): Unit = callback(Right(new ApiError(Errors.NOT_CONTROLLER, null))) } -case class UpdateFinalizedFeatures(newFeatures: Features[FinalizedVersionRange], - callback: UpdateFinalizedFeaturesCallback) extends ControllerEvent { - override def state: ControllerState = ControllerState.UpdateFinalizedFeatures +case class UpdateFeatures(newFeatures: Features[FinalizedVersionRange], + callback: UpdateFeaturesCallback) extends ControllerEvent { + override def state: ControllerState = ControllerState.UpdateFeatures } diff --git a/core/src/main/scala/kafka/server/BrokerFeatures.scala b/core/src/main/scala/kafka/server/BrokerFeatures.scala index 6ec977e7c181f..734cf15249e3e 100644 --- a/core/src/main/scala/kafka/server/BrokerFeatures.scala +++ b/core/src/main/scala/kafka/server/BrokerFeatures.scala @@ -35,12 +35,12 @@ import scala.jdk.CollectionConverters._ * in the closed range: [1, latest_min_version_level - 1] get deprecated by the controller logic * that applies this map to persistent finalized feature state in ZK (this mutation happens * during controller election and during finalized feature updates via the - * APIKeys.UPDATE_FINALIZED_FEATURES api). + * ApiKeys.UPDATE_FINALIZED_FEATURES api). This will automatically mean clients have to stop + * using the finalized min version levels that have been deprecated. * * This class also provides APIs to check for incompatibilities between the features supported by - * the Broker and finalized features. - * - * NOTE: the update*() and clear*() APIs of this class should be used only for testing purposes. + * the Broker and finalized features. The class is generally immutable. It provides few APIs to + * mutate state only for the purpose of testing. */ class BrokerFeatures private (@volatile var supportedFeatures: Features[SupportedVersionRange], @volatile var defaultFeatureMinVersionLevels: Map[String, Short]) { @@ -101,7 +101,7 @@ class BrokerFeatures private (@volatile var supportedFeatures: Features[Supporte * is empty, it means there were no feature incompatibilities found. */ def incompatibleFeatures(finalized: Features[FinalizedVersionRange]): Features[FinalizedVersionRange] = { - BrokerFeatures.incompatibleFeatures(supportedFeatures, finalized, true) + BrokerFeatures.incompatibleFeatures(supportedFeatures, finalized, logIncompatibilities = true) } } @@ -131,8 +131,8 @@ object BrokerFeatures extends Logging { private def incompatibleFeatures(supportedFeatures: Features[SupportedVersionRange], finalizedFeatures: Features[FinalizedVersionRange], logIncompatibilities: Boolean): Features[FinalizedVersionRange] = { - val incompatibilities = finalizedFeatures.features.asScala.map { - case (feature, versionLevels) => { + 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)) @@ -142,15 +142,14 @@ object BrokerFeatures extends Logging { } else { (feature, versionLevels, null) } - } }.filter{ case(_, _, errorReason) => errorReason != null}.toList - if (logIncompatibilities && incompatibilities.nonEmpty) { + if (logIncompatibilities && incompatibleFeaturesInfo.nonEmpty) { warn( - "Feature incompatibilities seen: " + incompatibilities.map{ + "Feature incompatibilities seen: " + incompatibleFeaturesInfo.map { case(_, _, errorReason) => errorReason }) } - Features.finalizedFeatures(incompatibilities.map{ + Features.finalizedFeatures(incompatibleFeaturesInfo.map { case(feature, versionLevels, _) => (feature, versionLevels) }.toMap.asJava) } @@ -169,12 +168,11 @@ object BrokerFeatures extends Logging { featureMinVersionLevels: Map[String, Short] ): Boolean = { featureMinVersionLevels.forall { - case(featureName, minVersionLevel) => { + case(featureName, minVersionLevel) => val supportedFeature = supportedFeatures.get(featureName) (supportedFeature != null) && new FinalizedVersionRange(minVersionLevel, supportedFeature.max()) .isCompatibleWith(supportedFeature) - } } } } diff --git a/core/src/main/scala/kafka/server/FinalizedFeatureCache.scala b/core/src/main/scala/kafka/server/FinalizedFeatureCache.scala index 8e5cac3d8b5ed..774d3636b05e5 100644 --- a/core/src/main/scala/kafka/server/FinalizedFeatureCache.scala +++ b/core/src/main/scala/kafka/server/FinalizedFeatureCache.scala @@ -144,18 +144,16 @@ class FinalizedFeatureCache(private val brokerFeatures: BrokerFeatures) extends if(timeoutMs < 0L) { throw new IllegalArgumentException(s"Expected timeoutMs >= 0, but $timeoutMs was provided.") } + val waitEndTimeNanos = System.nanoTime() + (timeoutMs * 1_000_000) synchronized { - var sleptTimeMs = 0L while (!waitCondition()) { - val timeoutLeftMs = timeoutMs - sleptTimeMs - if (timeoutLeftMs <= 0) { + if (System.nanoTime() > 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 timeBeforeNanos = System.nanoTime - wait(timeoutLeftMs) - sleptTimeMs += max(1L, (System.nanoTime - timeBeforeNanos) / 1_000_000) + val sleepTimeMs = max(1L, (waitEndTimeNanos - System.nanoTime()) / 1_000_000) + wait(sleepTimeMs) } } } diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index ec82d1a73f571..7b222231e0c82 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -53,7 +53,7 @@ import org.apache.kafka.common.internals.Topic.{GROUP_METADATA_TOPIC_NAME, TRANS import org.apache.kafka.common.message.AlterConfigsResponseData.AlterConfigsResourceResponse import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic import org.apache.kafka.common.message.CreatePartitionsResponseData.CreatePartitionsTopicResult -import org.apache.kafka.common.message.{AddOffsetsToTxnResponseData, AlterConfigsResponseData, AlterPartitionReassignmentsResponseData, AlterReplicaLogDirsResponseData, CreateAclsResponseData, CreatePartitionsResponseData, CreateTopicsResponseData, DeleteAclsResponseData, DeleteGroupsResponseData, DeleteRecordsResponseData, DeleteTopicsResponseData, DescribeAclsResponseData, DescribeConfigsResponseData, DescribeGroupsResponseData, DescribeLogDirsResponseData, EndTxnResponseData, ExpireDelegationTokenResponseData, FindCoordinatorResponseData, HeartbeatResponseData, InitProducerIdResponseData, JoinGroupResponseData, LeaveGroupResponseData, ListGroupsResponseData, ListPartitionReassignmentsResponseData, MetadataResponseData, OffsetCommitRequestData, OffsetCommitResponseData, OffsetDeleteResponseData, RenewDelegationTokenResponseData, SaslAuthenticateResponseData, SaslHandshakeResponseData, StopReplicaResponseData, SyncGroupResponseData, UpdateMetadataResponseData} +import org.apache.kafka.common.message.{AddOffsetsToTxnResponseData, AlterConfigsResponseData, AlterPartitionReassignmentsResponseData, AlterReplicaLogDirsResponseData, CreateAclsResponseData, CreatePartitionsResponseData, CreateTopicsResponseData, DeleteAclsResponseData, DeleteGroupsResponseData, DeleteRecordsResponseData, DeleteTopicsResponseData, DescribeAclsResponseData, DescribeConfigsResponseData, DescribeGroupsResponseData, DescribeLogDirsResponseData, EndTxnResponseData, ExpireDelegationTokenResponseData, FindCoordinatorResponseData, HeartbeatResponseData, InitProducerIdResponseData, JoinGroupResponseData, LeaveGroupResponseData, ListGroupsResponseData, ListPartitionReassignmentsResponseData, MetadataResponseData, OffsetCommitRequestData, OffsetCommitResponseData, OffsetDeleteResponseData, RenewDelegationTokenResponseData, SaslAuthenticateResponseData, SaslHandshakeResponseData, StopReplicaResponseData, SyncGroupResponseData, UpdateFeaturesRequestData, UpdateFeaturesResponseData, UpdateMetadataResponseData} import org.apache.kafka.common.message.CreateTopicsResponseData.{CreatableTopicResult, CreatableTopicResultCollection} import org.apache.kafka.common.message.DeleteGroupsResponseData.{DeletableGroupResult, DeletableGroupResultCollection} import org.apache.kafka.common.message.AlterPartitionReassignmentsResponseData.{ReassignablePartitionResponse, ReassignableTopicResponse} @@ -66,7 +66,6 @@ import org.apache.kafka.common.message.LeaveGroupResponseData.MemberResponse import org.apache.kafka.common.message.ListOffsetRequestData.ListOffsetPartition import org.apache.kafka.common.message.ListOffsetResponseData import org.apache.kafka.common.message.ListOffsetResponseData.{ListOffsetPartitionResponse, ListOffsetTopicResponse} -import org.apache.kafka.common.message.UpdateFinalizedFeaturesResponseData import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.network.{ListenerName, Send} import org.apache.kafka.common.protocol.{ApiKeys, Errors} @@ -189,7 +188,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_FINALIZED_FEATURES => handleUpdateFinalizedFeatures(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()) @@ -3112,13 +3111,12 @@ class KafkaApis(val requestChannel: RequestChannel, } } - def handleUpdateFinalizedFeatures(request: RequestChannel.Request): Unit = { - val updateFinalizedFeaturesRequest = request.body[UpdateFinalizedFeaturesRequest] + def handleUpdateFeatures(request: RequestChannel.Request): Unit = { + val updateFeaturesRequest = request.body[UpdateFeaturesRequest] def sendResponseCallback(error: Errors, msgOverride: Option[String]): Unit = { - sendResponseExemptThrottle(request, new UpdateFinalizedFeaturesResponse( - new UpdateFinalizedFeaturesResponseData() - .setErrorCode(error.code()) - .setErrorMessage(msgOverride.getOrElse(error.message())))) + val data = new UpdateFeaturesResponseData().setErrorCode(error.code()) + msgOverride.map(msg => data.setErrorMessage(msg)) + sendResponseExemptThrottle(request, new UpdateFeaturesResponse(data)) } if (!authorize(request.context, ALTER, CLUSTER, CLUSTER_NAME)) { @@ -3127,11 +3125,13 @@ class KafkaApis(val requestChannel: RequestChannel, sendResponseCallback(Errors.NOT_CONTROLLER, Option.empty) } else if (!config.isFeatureVersioningEnabled) { sendResponseCallback(Errors.INVALID_REQUEST, Some("Feature versioning system is disabled.")) + } else if (updateFeaturesRequest.data.featureUpdates.isEmpty) { + sendResponseCallback(Errors.INVALID_REQUEST, Some("Can not provide empty FinalizedFeatureUpdates in the request.")) } else { - val targetFeaturesOrError = getTargetFinalizedFeaturesOrError(updateFinalizedFeaturesRequest) + val targetFeaturesOrError = getTargetFinalizedFeaturesOrError(updateFeaturesRequest) targetFeaturesOrError match { case Left(targetFeatures) => - controller.updateFinalizedFeatures(targetFeatures, sendResponseCallback) + controller.updateFeatures(targetFeatures, sendResponseCallback) case Right(error) => sendResponseCallback(error.error, Some(error.message)) } @@ -3148,99 +3148,94 @@ class KafkaApis(val requestChannel: RequestChannel, * @return - the target finalized features, if request validation is successful * - an ApiError if request validation fails */ - def getTargetFinalizedFeaturesOrError(request: UpdateFinalizedFeaturesRequest): - Either[Features[FinalizedVersionRange], ApiError] = { - val updates = request.data.finalizedFeatureUpdates - if (updates.isEmpty) { - return Right(new ApiError(Errors.INVALID_REQUEST, - "Can not provide empty FinalizedFeatureUpdates in the request.")) + private def getTargetFinalizedFeaturesOrError(request: UpdateFeaturesRequest): Either[Features[FinalizedVersionRange], ApiError] = { + val updates = request.data.featureUpdates + val newFeatures = scala.collection.mutable.Map[String, FinalizedVersionRange]() + + def addFeature(update: UpdateFeaturesRequestData.FeatureUpdateKey): Unit = { + // NOTE: Below we set the finalized min version level to be the default minimum version + // level. If the finalized feature already exists, then, this can cause deprecation of all + // version levels in the closed range: + // [existingVersionRange.min(), defaultMinVersionLevel - 1]. + val defaultMinVersionLevel = brokerFeatures.defaultMinVersionLevel(update.name) + newFeatures += ( + update.name -> new FinalizedVersionRange( + defaultMinVersionLevel, + update.maxVersionLevel)) } val latestFeatures = featureCache.get - val newFeatures = scala.collection.mutable.Map[String, FinalizedVersionRange]() - updates.asScala.foreach( + updates.asScala.iterator.map( update => { - // Rule #1) Check that the feature name is not empty. if (update.name.isEmpty) { - return Right( + // Rule #1) Check that the feature name is not empty. + Some( new ApiError(Errors.INVALID_REQUEST, "Can not contain empty feature name in the request.")) - } - - val cacheEntry = latestFeatures.map(lf => lf.features.get(update.name)).orNull - - // We handle deletion requests separately from non-deletion requests. - if (UpdateFinalizedFeaturesRequest.isDeleteRequest(update)) { // Deletion request - // Rule #2) Disallow deletion of a finalized feature without allowDowngrade flag set. - if (!update.allowDowngrade) { - return Right( - new ApiError(Errors.INVALID_REQUEST, - s"Can not delete feature: '${update.name}' without setting the" + - " allowDowngrade flag in the request.")) - } - - // Rule #3) Disallow deletion of a non-existing finalized feature. - if (cacheEntry == null) { - return Right( - new ApiError(Errors.INVALID_REQUEST, - s"Can not delete non-existing finalized feature: '${update.name}'")) - } - } else { // Non-deletion request - if (cacheEntry != null) { - // Rule 4) Disallow a case where target maxVersionLevel matches - // existing maxVersionLevel. - if (update.maxVersionLevel == cacheEntry.max()) { - return Right( - new ApiError(Errors.INVALID_REQUEST, - s"Can not upgrade a finalized feature: '${update.name}' from" + - s" existing maxVersionLevel:${cacheEntry.max} to the same value.")) - } + } else { + val cacheEntry = latestFeatures.map(lf => lf.features.get(update.name)).orNull - // Rule #5) Disallow downgrade of a finalized feature without the - // allowDowngrade flag set. - if (update.maxVersionLevel < cacheEntry.max && !update.allowDowngrade) { - return Right( + // We handle deletion requests separately from non-deletion requests. + if (UpdateFeaturesRequest.isDeleteRequest(update)) { + if (!update.allowDowngrade) { + // Rule #2) Disallow deletion of a finalized feature without allowDowngrade flag set. + Some( new ApiError(Errors.INVALID_REQUEST, - s"Can not downgrade finalized feature: '${update.name}' from" + - s" existing maxVersionLevel:${cacheEntry.max} to provided" + - s" maxVersionLevel:${update.maxVersionLevel} without setting the" + - " allowDowngrade flag in the request.")) - } - - // Rule #6) Disallow a request that sets allowDowngrade flag without specifying a - // maxVersionLevel that's lower than the existing maxVersionLevel. - if (update.allowDowngrade && update.maxVersionLevel > cacheEntry.max) { - return Right( + s"Can not delete feature: '${update.name}' without setting the" + + " allowDowngrade flag to true in the request.")) + } else if (cacheEntry == null) { + // Rule #3) Disallow deletion of a non-existing finalized feature. + Some( new ApiError(Errors.INVALID_REQUEST, - s"When finalized feature: '${update.name}' has the allowDowngrade flag" + - s" set in the request, the provided maxVersionLevel:${update.maxVersionLevel}" + - s" can not be greater than existing maxVersionLevel:${cacheEntry.max}.")) + s"Can not delete non-existing finalized feature: '${update.name}'")) } - - // Rule #7) Disallow downgrade of a finalized feature below the existing finalized - // minVersionLevel. - if (update.maxVersionLevel() < cacheEntry.min()) { - return Right( - new ApiError(Errors.INVALID_REQUEST, - s"Can not downgrade finalized feature: '${update.name}' to" + - s" maxVersionLevel:${update.maxVersionLevel} because it's lower than the" + - s" existing minVersionLevel:${cacheEntry.min}.")) + } else { + if (cacheEntry == null) { + addFeature(update) + } else { + if (update.maxVersionLevel == cacheEntry.max()) { + // Rule 4) Disallow a case where target maxVersionLevel matches + // existing maxVersionLevel. + Some( + new ApiError(Errors.INVALID_REQUEST, + s"Can not ${if (update.allowDowngrade) "downgrade" else "upgrade"}" + + s" a finalized feature: '${update.name}' from existing" + + s" maxVersionLevel:${cacheEntry.max} to the same value.")) + } else if (update.maxVersionLevel < cacheEntry.max && !update.allowDowngrade) { + // Rule #5) Disallow downgrade of a finalized feature without the + // allowDowngrade flag set. + Some( + new ApiError(Errors.INVALID_REQUEST, + s"Can not downgrade finalized feature: '${update.name}' from" + + s" existing maxVersionLevel:${cacheEntry.max} to provided" + + s" maxVersionLevel:${update.maxVersionLevel} without setting the" + + " allowDowngrade flag in the request.")) + } else if (update.allowDowngrade && update.maxVersionLevel > cacheEntry.max) { + // Rule #6) Disallow a request that sets allowDowngrade flag without specifying a + // maxVersionLevel that's lower than the existing maxVersionLevel. + Some( + new ApiError(Errors.INVALID_REQUEST, + s"When finalized feature: '${update.name}' has the allowDowngrade flag" + + s" set in the request, the provided maxVersionLevel:${update.maxVersionLevel}" + + s" can not be greater than existing maxVersionLevel:${cacheEntry.max}.")) + } else if (update.maxVersionLevel() < cacheEntry.min()) { + // Rule #7) Disallow downgrade of a finalized feature below the existing finalized + // minVersionLevel. + Some( + new ApiError(Errors.INVALID_REQUEST, + s"Can not downgrade finalized feature: '${update.name}' to" + + s" maxVersionLevel:${update.maxVersionLevel} because it's lower than the" + + s" existing minVersionLevel:${cacheEntry.min}.")) + } else { + addFeature(update) + } } } - - // NOTE: if the finalized feature already exists, then, below we set the default minimum - // version level. This can cause deprecation of all version levels in the closed range: - // [existingVersionRange.min(), defaultMinVersionLevel - 1], unless it was previously - // deprecated. - val defaultMinVersionLevel = brokerFeatures.defaultMinVersionLevel(update.name) - newFeatures += ( - update.name -> new FinalizedVersionRange( - defaultMinVersionLevel, - update.maxVersionLevel)) } } - ) - Left(Features.finalizedFeatures(newFeatures.asJava)) + ).collectFirst { + case Some(error) => error.asInstanceOf[ApiError] + }.map(error => Right(error)).getOrElse(Left(Features.finalizedFeatures(newFeatures.asJava))) } // private package for testing diff --git a/core/src/test/scala/unit/kafka/server/UpdateFinalizedFeaturesTest.scala b/core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala similarity index 83% rename from core/src/test/scala/unit/kafka/server/UpdateFinalizedFeaturesTest.scala rename to core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala index fe699c34030f2..48c199ba801d0 100644 --- a/core/src/test/scala/unit/kafka/server/UpdateFinalizedFeaturesTest.scala +++ b/core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala @@ -11,13 +11,13 @@ 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, FeatureMetadata, FinalizedFeatureUpdate, UpdateFinalizedFeaturesOptions, UpdateFinalizedFeaturesResult} +import org.apache.kafka.clients.admin.{Admin, DescribeFeaturesOptions, FeatureMetadata, 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.UpdateFinalizedFeaturesRequestData +import org.apache.kafka.common.message.UpdateFeaturesRequestData import org.apache.kafka.common.protocol.Errors -import org.apache.kafka.common.requests.{UpdateFinalizedFeaturesRequest, UpdateFinalizedFeaturesResponse} +import org.apache.kafka.common.requests.{UpdateFeaturesRequest, UpdateFeaturesResponse} import org.apache.kafka.common.utils.Utils import org.junit.Test import org.junit.Assert.{assertEquals, assertNotEquals, assertNotNull, assertTrue} @@ -26,7 +26,7 @@ import org.scalatest.Assertions.{assertThrows, intercept} import scala.reflect.ClassTag import scala.util.matching.Regex -class UpdateFinalizedFeaturesTest extends BaseRequestTest { +class UpdateFeaturesTest extends BaseRequestTest { override def brokerCount = 3 @@ -100,8 +100,8 @@ class UpdateFinalizedFeaturesTest extends BaseRequestTest { } private def checkException[ExceptionType <: Throwable]( - result: UpdateFinalizedFeaturesResult, - exceptionMsgPattern: Regex + result: UpdateFeaturesResult, + exceptionMsgPattern: Regex )(implicit tag: ClassTag[ExceptionType]): Unit = { val exception = intercept[ExecutionException] { result.result().get() @@ -116,12 +116,12 @@ class UpdateFinalizedFeaturesTest extends BaseRequestTest { * and raises the ExceptionType on the client side as expected. * * @param invalidUpdate the invalid feature update to be sent in the - * updateFinalizedFeatures request to the server + * updateFeatures request to the server * @param exceptionMsgPattern a pattern for the expected exception message */ private def testWithInvalidFeatureUpdate[ExceptionType <: Throwable]( - invalidUpdate: FinalizedFeatureUpdate, - exceptionMsgPattern: Regex + invalidUpdate: FeatureUpdate, + exceptionMsgPattern: Regex )(implicit tag: ClassTag[ExceptionType]): Unit = { TestUtils.waitUntilControllerElected(zkClient) @@ -131,8 +131,8 @@ class UpdateFinalizedFeaturesTest extends BaseRequestTest { val adminClient = createAdminClient() val nodeBefore = getFeatureZNode() - val result = adminClient.updateFinalizedFeatures( - new HashSet[FinalizedFeatureUpdate](Collections.singletonList(invalidUpdate)), new UpdateFinalizedFeaturesOptions()) + val result = adminClient.updateFeatures( + new HashSet[FeatureUpdate](Collections.singletonList(invalidUpdate)), new UpdateFeaturesOptions()) checkException[ExceptionType](result, exceptionMsgPattern) checkFeatures( @@ -149,14 +149,14 @@ class UpdateFinalizedFeaturesTest extends BaseRequestTest { val versionBefore = updateFeatureZNode(defaultFinalizedFeatures()) val nodeBefore = getFeatureZNode() - val requestData = FinalizedFeatureUpdate.createRequest( - new util.HashSet[FinalizedFeatureUpdate]( - Collections.singletonList(new FinalizedFeatureUpdate("feature_1", + val requestData = FeatureUpdate.createRequest( + new util.HashSet[FeatureUpdate]( + Collections.singletonList(new FeatureUpdate("feature_1", defaultSupportedFeatures().get("feature_1").max(), false)))) - val response = connectAndReceive[UpdateFinalizedFeaturesResponse]( - new UpdateFinalizedFeaturesRequest.Builder(requestData).build(), notControllerSocketServer) + val response = connectAndReceive[UpdateFeaturesResponse]( + new UpdateFeaturesRequest.Builder(requestData).build(), notControllerSocketServer) assertEquals(Errors.NOT_CONTROLLER, response.error()) checkFeatures( @@ -167,7 +167,7 @@ class UpdateFinalizedFeaturesTest extends BaseRequestTest { @Test def testShouldFailRequestForInvalidFeatureName(): Unit = { - testWithInvalidFeatureUpdate[InvalidRequestException](new FinalizedFeatureUpdate( + testWithInvalidFeatureUpdate[InvalidRequestException](new FeatureUpdate( "", defaultSupportedFeatures().get("feature_1").max(), false), @@ -176,7 +176,7 @@ class UpdateFinalizedFeaturesTest extends BaseRequestTest { @Test def testShouldFailRequestWhenDowngradeFlagIsNotSetDuringDowngrade(): Unit = { - testWithInvalidFeatureUpdate[InvalidRequestException](new FinalizedFeatureUpdate( + testWithInvalidFeatureUpdate[InvalidRequestException](new FeatureUpdate( "feature_1", (defaultFinalizedFeatures().get("feature_1").max() - 1).asInstanceOf[Short], false), @@ -186,7 +186,7 @@ class UpdateFinalizedFeaturesTest extends BaseRequestTest { @Test def testShouldFailRequestWhenDowngradeToHigherVersionLevelIsAttempted(): Unit = { testWithInvalidFeatureUpdate[InvalidRequestException]( - new FinalizedFeatureUpdate( + new FeatureUpdate( "feature_1", defaultSupportedFeatures().get("feature_1").max(), true), @@ -196,7 +196,7 @@ class UpdateFinalizedFeaturesTest extends BaseRequestTest { @Test def testShouldFailRequestInClientWhenDowngradeFlagIsNotSetDuringDeletion(): Unit = { assertThrows[IllegalArgumentException] { - new FinalizedFeatureUpdate("feature_1", 0, false) + new FeatureUpdate("feature_1", 0, false) } } @@ -211,17 +211,17 @@ class UpdateFinalizedFeaturesTest extends BaseRequestTest { val nodeBefore = getFeatureZNode() val featureUpdates - = new UpdateFinalizedFeaturesRequestData.FinalizedFeatureUpdateKeyCollection(); - val featureUpdate = new UpdateFinalizedFeaturesRequestData.FinalizedFeatureUpdateKey(); + = new UpdateFeaturesRequestData.FeatureUpdateKeyCollection(); + val featureUpdate = new UpdateFeaturesRequestData.FeatureUpdateKey(); featureUpdate.setName("feature_1") featureUpdate.setMaxVersionLevel(0) featureUpdate.setAllowDowngrade(false) featureUpdates.add(featureUpdate); - val requestData = new UpdateFinalizedFeaturesRequestData() - requestData.setFinalizedFeatureUpdates(featureUpdates); + val requestData = new UpdateFeaturesRequestData() + requestData.setFeatureUpdates(featureUpdates); - val response = connectAndReceive[UpdateFinalizedFeaturesResponse]( - new UpdateFinalizedFeaturesRequest.Builder(requestData).build(), controllerSocketServer) + val response = connectAndReceive[UpdateFeaturesResponse]( + new UpdateFeaturesRequest.Builder(requestData).build(), controllerSocketServer) assertEquals(Errors.INVALID_REQUEST, response.error) val exceptionMsgPattern = ".*Can not delete feature: 'feature_1'.*allowDowngrade.*".r @@ -235,14 +235,14 @@ class UpdateFinalizedFeaturesTest extends BaseRequestTest { @Test def testShouldFailRequestDuringDeletionOfNonExistingFeature(): Unit = { testWithInvalidFeatureUpdate[InvalidRequestException]( - new FinalizedFeatureUpdate("feature_non_existing", 0, true), + new FeatureUpdate("feature_non_existing", 0, true), ".*Can not delete non-existing finalized feature: 'feature_non_existing'.*".r) } @Test def testShouldFailRequestWhenUpgradingToSameVersionLevel(): Unit = { testWithInvalidFeatureUpdate[InvalidRequestException]( - new FinalizedFeatureUpdate( + new FeatureUpdate( "feature_1", defaultFinalizedFeatures().get("feature_1").max(), false), ".*Can not upgrade a finalized feature: 'feature_1'.*to the same value.*".r) } @@ -258,13 +258,13 @@ class UpdateFinalizedFeaturesTest extends BaseRequestTest { Utils.mkMap(Utils.mkEntry("feature_1", new FinalizedVersionRange(minVersionLevel, 2)))) val versionBefore = updateFeatureZNode(initialFinalizedFeatures) - val update = new FinalizedFeatureUpdate( + val update = new FeatureUpdate( "feature_1", (minVersionLevel - 1).asInstanceOf[Short], true) val adminClient = createAdminClient() val nodeBefore = getFeatureZNode() - val result = adminClient.updateFinalizedFeatures( - new HashSet[FinalizedFeatureUpdate](Collections.singletonList(update)), new UpdateFinalizedFeaturesOptions()) + val result = adminClient.updateFeatures( + new HashSet[FeatureUpdate](Collections.singletonList(update)), new UpdateFeaturesOptions()) checkException[InvalidRequestException]( result, ".*Can not downgrade finalized feature: 'feature_1' to maxVersionLevel:1.*existing minVersionLevel:2.*".r) @@ -298,13 +298,13 @@ class UpdateFinalizedFeaturesTest extends BaseRequestTest { val versionBefore = updateFeatureZNode(defaultFinalizedFeatures()) - val invalidUpdate = new FinalizedFeatureUpdate( + val invalidUpdate = new FeatureUpdate( "feature_1", defaultSupportedFeatures().get("feature_1").max(), false) val nodeBefore = getFeatureZNode() val adminClient = createAdminClient() - val result = adminClient.updateFinalizedFeatures( - new HashSet[FinalizedFeatureUpdate](Collections.singletonList(invalidUpdate)), - new UpdateFinalizedFeaturesOptions()) + val result = adminClient.updateFeatures( + new HashSet[FeatureUpdate](Collections.singletonList(invalidUpdate)), + new UpdateFeaturesOptions()) checkException[InvalidRequestException]( result, ".*1 broker.*incompatible.*".r) @@ -334,9 +334,9 @@ class UpdateFinalizedFeaturesTest extends BaseRequestTest { Utils.mkMap( Utils.mkEntry("feature_1", new FinalizedVersionRange(1, 3)), Utils.mkEntry("feature_2", new FinalizedVersionRange(2, 3)))) - val update1 = new FinalizedFeatureUpdate( + val update1 = new FeatureUpdate( "feature_1", targetFinalizedFeatures.get("feature_1").max(), false) - val update2 = new FinalizedFeatureUpdate( + val update2 = new FeatureUpdate( "feature_2", targetFinalizedFeatures.get("feature_2").max(), true) val expected = new FeatureMetadata( @@ -348,9 +348,9 @@ class UpdateFinalizedFeaturesTest extends BaseRequestTest { Utils.mkEntry("feature_2", new SupportedVersionRange(2, 5))))) val adminClient = createAdminClient() - adminClient.updateFinalizedFeatures( - new HashSet[FinalizedFeatureUpdate](Arrays.asList(update1, update2)), - new UpdateFinalizedFeaturesOptions()).result().get() + adminClient.updateFeatures( + new HashSet[FeatureUpdate](Arrays.asList(update1, update2)), + new UpdateFeaturesOptions()).result().get() checkFeatures( adminClient, @@ -378,16 +378,16 @@ class UpdateFinalizedFeaturesTest extends BaseRequestTest { Utils.mkMap( Utils.mkEntry("feature_1", new FinalizedVersionRange(1, 3)), Utils.mkEntry("feature_2", new FinalizedVersionRange(2, 3)))) - val validUpdate = new FinalizedFeatureUpdate( + val validUpdate = new FeatureUpdate( "feature_1", targetFinalizedFeatures.get("feature_1").max(), false) - val invalidUpdate = new FinalizedFeatureUpdate( + val invalidUpdate = new FeatureUpdate( "feature_2", targetFinalizedFeatures.get("feature_2").max(), false) val nodeBefore = getFeatureZNode() val adminClient = createAdminClient() - val result = adminClient.updateFinalizedFeatures( - new HashSet[FinalizedFeatureUpdate](Arrays.asList(validUpdate, invalidUpdate)), - new UpdateFinalizedFeaturesOptions()) + val result = adminClient.updateFeatures( + new HashSet[FeatureUpdate](Arrays.asList(validUpdate, invalidUpdate)), + new UpdateFeaturesOptions()) checkException[InvalidRequestException]( result, ".*Can not downgrade finalized feature: 'feature_2'.*allowDowngrade.*".r) @@ -430,16 +430,16 @@ class UpdateFinalizedFeaturesTest extends BaseRequestTest { Utils.mkMap( Utils.mkEntry("feature_1", new FinalizedVersionRange(1, 3)), Utils.mkEntry("feature_2", new FinalizedVersionRange(2, 3)))) - val invalidUpdate = new FinalizedFeatureUpdate( + val invalidUpdate = new FeatureUpdate( "feature_1", targetFinalizedFeatures.get("feature_1").max(), false) - val validUpdate = new FinalizedFeatureUpdate( + val validUpdate = new FeatureUpdate( "feature_2", targetFinalizedFeatures.get("feature_2").max(), true) val nodeBefore = getFeatureZNode() val adminClient = createAdminClient() - val result = adminClient.updateFinalizedFeatures( - new HashSet[FinalizedFeatureUpdate](Arrays.asList(invalidUpdate, validUpdate)), - new UpdateFinalizedFeaturesOptions()) + val result = adminClient.updateFeatures( + new HashSet[FeatureUpdate](Arrays.asList(invalidUpdate, validUpdate)), + new UpdateFeaturesOptions()) checkException[InvalidRequestException](result, ".*1 broker.*incompatible.*".r) checkFeatures( From 08f064b6cb42e3cfed06285f10de4428ba1a6091 Mon Sep 17 00:00:00 2001 From: Kowshik Prakasam Date: Tue, 21 Jul 2020 03:11:18 -0700 Subject: [PATCH 08/41] Minor: add missing header in UpdateFeaturesTest.scala --- .../unit/kafka/server/UpdateFeaturesTest.scala | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala b/core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala index 48c199ba801d0..6d52829112958 100644 --- a/core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala +++ b/core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala @@ -1,3 +1,20 @@ +/** + * 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 From 6b9e237c5b753fac69d2989ea5ade6c8ce759cf1 Mon Sep 17 00:00:00 2001 From: Kowshik Prakasam Date: Tue, 21 Jul 2020 18:26:50 -0700 Subject: [PATCH 09/41] Minor cosmetic changes --- .../clients/admin/UpdateFeaturesOptions.java | 2 +- .../clients/admin/UpdateFeaturesResult.java | 2 +- .../errors/FeatureUpdateFailedException.java | 4 ++-- .../clients/admin/KafkaAdminClientTest.java | 4 +--- .../src/main/scala/kafka/server/KafkaApis.scala | 17 +++++++---------- 5 files changed, 12 insertions(+), 17 deletions(-) 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 index 912918d6391da..15de027c4d754 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/UpdateFeaturesOptions.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/UpdateFeaturesOptions.java @@ -24,7 +24,7 @@ public class UpdateFeaturesOptions extends AbstractOptions future; - public UpdateFeaturesResult(KafkaFuture future) { + public UpdateFeaturesResult(final KafkaFuture future) { this.future = future; } 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 index 8ad5da2dbf564..9f5e23d3104da 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/FeatureUpdateFailedException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/FeatureUpdateFailedException.java @@ -19,11 +19,11 @@ public class FeatureUpdateFailedException extends ApiException { private static final long serialVersionUID = 1L; - public FeatureUpdateFailedException(String message) { + public FeatureUpdateFailedException(final String message) { super(message); } - public FeatureUpdateFailedException(String message, Throwable cause) { + public FeatureUpdateFailedException(final String message, final Throwable cause) { super(message, cause); } } 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 15f1f73844d43..091f3c0125f88 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 @@ -516,9 +516,7 @@ private static ApiVersionsResponse prepareApiVersionsResponseForDescribeFeatures getDefaultFeatureMetadata().finalizedFeatures(), getDefaultFeatureMetadata().finalizedFeaturesEpoch().orElse(-1))); } - final ApiVersionsResponseData data = new ApiVersionsResponseData(); - data.setErrorCode(error.code()); - return new ApiVersionsResponse(data); + return new ApiVersionsResponse(new ApiVersionsResponseData().setErrorCode(error.code())); } /** diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 7b222231e0c82..4d45a3b36e0ae 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -3169,9 +3169,8 @@ class KafkaApis(val requestChannel: RequestChannel, update => { if (update.name.isEmpty) { // Rule #1) Check that the feature name is not empty. - Some( - new ApiError(Errors.INVALID_REQUEST, - "Can not contain empty feature name in the request.")) + Some(new ApiError(Errors.INVALID_REQUEST, + "Can not contain empty feature name in the request.")) } else { val cacheEntry = latestFeatures.map(lf => lf.features.get(update.name)).orNull @@ -3179,15 +3178,13 @@ class KafkaApis(val requestChannel: RequestChannel, if (UpdateFeaturesRequest.isDeleteRequest(update)) { if (!update.allowDowngrade) { // Rule #2) Disallow deletion of a finalized feature without allowDowngrade flag set. - Some( - new ApiError(Errors.INVALID_REQUEST, - s"Can not delete feature: '${update.name}' without setting the" + - " allowDowngrade flag to true in the request.")) + Some(new ApiError(Errors.INVALID_REQUEST, + s"Can not delete feature: '${update.name}' without setting the" + + " allowDowngrade flag to true in the request.")) } else if (cacheEntry == null) { // Rule #3) Disallow deletion of a non-existing finalized feature. - Some( - new ApiError(Errors.INVALID_REQUEST, - s"Can not delete non-existing finalized feature: '${update.name}'")) + Some(new ApiError(Errors.INVALID_REQUEST, + s"Can not delete non-existing finalized feature: '${update.name}'")) } } else { if (cacheEntry == null) { From c2772a1196e0beb18c1581b43573e23b7a22a284 Mon Sep 17 00:00:00 2001 From: Kowshik Prakasam Date: Tue, 28 Jul 2020 02:19:08 -0700 Subject: [PATCH 10/41] Address comments from Boyang --- .../org/apache/kafka/clients/admin/Admin.java | 32 +- .../admin/DescribeFeaturesOptions.java | 24 +- .../kafka/clients/admin/FeatureUpdate.java | 46 +-- .../kafka/clients/admin/KafkaAdminClient.java | 122 +++---- .../clients/admin/UpdateFeaturesOptions.java | 14 +- .../clients/admin/UpdateFeaturesResult.java | 22 +- .../common/feature/FinalizedVersionRange.java | 4 - .../apache/kafka/common/protocol/Errors.java | 2 +- .../common/requests/ApiVersionsResponse.java | 4 +- .../requests/UpdateFeaturesRequest.java | 18 +- .../requests/UpdateFeaturesResponse.java | 14 +- .../common/message/UpdateFeaturesRequest.json | 6 +- .../message/UpdateFeaturesResponse.json | 13 +- .../clients/admin/KafkaAdminClientTest.java | 141 +++++---- .../kafka/clients/admin/MockAdminClient.java | 2 +- .../kafka/controller/KafkaController.scala | 271 ++++++++++++---- .../scala/kafka/server/BrokerFeatures.scala | 14 +- .../kafka/server/FinalizedFeatureCache.scala | 5 +- .../main/scala/kafka/server/KafkaApis.scala | 139 ++------ .../main/scala/kafka/server/KafkaConfig.scala | 1 + .../ControllerIntegrationTest.scala | 2 +- .../kafka/server/BrokerFeaturesTest.scala | 9 +- .../FinalizedFeatureChangeListenerTest.scala | 5 + .../kafka/server/UpdateFeaturesTest.scala | 297 +++++++++++------- 24 files changed, 696 insertions(+), 511 deletions(-) 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 45bae8538bc15..5a0751d4a292f 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 @@ -1322,28 +1322,28 @@ AlterUserScramCredentialsResult alterUserScramCredentials(List * @param options the options to use * - * @return the DescribeFeaturesResult containing the result + * @return the {@link DescribeFeaturesResult} containing the result */ DescribeFeaturesResult describeFeatures(DescribeFeaturesOptions options); /** - * Applies specified updates to finalized features. The API is atomic, meaning that if a single - * feature update in the request can't succeed on the controller, then none of the feature - * updates are carried out. This request is issued only to the controller since the API is - * only served by the controller. + * Applies specified updates to finalized features. This operation is not transactional so it + * may succeed for some features while fail for others. *

- * The API takes in a set of feature updates that need to be applied. Each such update specifies - * the finalized feature to be added or updated or deleted, along with the new max feature - * version level value. + * The API takes in a map of finalized feature name to {@link FeatureUpdate} that need 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 feature. *

    *
  • Downgrade of feature version level is not a regular operation/intent. It is only allowed - * in the controller if the feature update 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.
  • + * 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 feature update, and, setting the - * max version level to be less than 1.
  • + * done by setting the allowDowngrade flag to true in the {@link FeatureUpdate}, and, setting + * the max version level to be less than 1. *
*

* The following exceptions can be anticipated when calling {@code get()} on the futures @@ -1364,12 +1364,12 @@ AlterUserScramCredentialsResult alterUserScramCredentials(List * This operation is supported by brokers with version 2.7.0 or higher. - * @param featureUpdates the set of finalized feature updates + * @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(Set featureUpdates, UpdateFeaturesOptions options); + 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 index 45bd656cb85bf..d7043af43c5d8 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/DescribeFeaturesOptions.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/DescribeFeaturesOptions.java @@ -16,36 +16,34 @@ */ package org.apache.kafka.clients.admin; +import java.util.Set; import org.apache.kafka.common.annotation.InterfaceStability; +/** + * Options for {@link AdminClient#describeFeatures(DescribeFeaturesOptions)} + * + * The API of this class is evolving. See {@link AdminClient} for details. + */ @InterfaceStability.Evolving public class DescribeFeaturesOptions extends AbstractOptions { + /** * - True means the {@link Admin#describeFeatures(DescribeFeaturesOptions)} request can be * issued only to the controller. * - False means the {@link Admin#describeFeatures(DescribeFeaturesOptions)} request can be * issued to any random broker. */ - private boolean shouldSendRequestToController = false; + private boolean sendRequestToController = false; /** * Sets a flag indicating that the describe features request should be issued to the controller. */ - public DescribeFeaturesOptions sendRequestToController(boolean shouldSendRequestToController) { - this.shouldSendRequestToController = shouldSendRequestToController; + public DescribeFeaturesOptions sendRequestToController(boolean sendRequestToController) { + this.sendRequestToController = sendRequestToController; return this; } public boolean sendRequestToController() { - return shouldSendRequestToController; - } - - /** - * Sets the timeout in milliseconds for this operation or {@code null} if the default API - * timeout for the AdminClient should be used. - */ - public DescribeFeaturesOptions timeoutMs(Integer timeoutMs) { - this.timeoutMs = timeoutMs; - return this; + return sendRequestToController; } } 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 index 00dfea96b88da..749522b9f9723 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/FeatureUpdate.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/FeatureUpdate.java @@ -16,21 +16,17 @@ */ package org.apache.kafka.clients.admin; -import java.util.Objects; -import java.util.Set; -import org.apache.kafka.common.message.UpdateFeaturesRequestData; +import java.util.Map; /** * Encapsulates details about an update to a finalized feature. This is particularly useful to - * define each feature update in the {@link Admin#updateFeatures(Set, UpdateFeaturesOptions)} API. + * define each feature update in the {@link Admin#updateFeatures(Map, UpdateFeaturesOptions)} API. */ public class FeatureUpdate { - private final String featureName; private final short maxVersionLevel; private final boolean allowDowngrade; /** - * @param featureName the name of the finalized feature to be updated. * @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 @@ -39,23 +35,16 @@ public class FeatureUpdate { * maximum version level of the finalized feature. * - false, otherwise. */ - public FeatureUpdate(final String featureName, final short maxVersionLevel, final boolean allowDowngrade) { - Objects.requireNonNull(featureName, "Provided feature name can not be null."); + public FeatureUpdate(final short maxVersionLevel, final boolean allowDowngrade) { if (maxVersionLevel < 1 && !allowDowngrade) { - throw new IllegalArgumentException( - String.format( - "For featureName: %s, the allowDowngrade flag is not set when the" + - " provided maxVersionLevel:%d is < 1.", featureName, maxVersionLevel)); + throw new IllegalArgumentException(String.format( + "The allowDowngrade flag is not set when the provided maxVersionLevel:%d is < 1.", + maxVersionLevel)); } - this.featureName = featureName; this.maxVersionLevel = maxVersionLevel; this.allowDowngrade = allowDowngrade; } - public String featureName() { - return featureName; - } - public short maxVersionLevel() { return maxVersionLevel; } @@ -63,25 +52,4 @@ public short maxVersionLevel() { public boolean allowDowngrade() { return allowDowngrade; } - - /** - * Helper function that creates {@link UpdateFeaturesRequestData} from a set of {@link FeatureUpdate}. - * - * @param updates the set of {@link FeatureUpdate} - * - * @return a newly constructed UpdateFeaturesRequestData object - */ - public static UpdateFeaturesRequestData createRequest(Set updates) { - final UpdateFeaturesRequestData.FeatureUpdateKeyCollection items - = new UpdateFeaturesRequestData.FeatureUpdateKeyCollection(); - for (FeatureUpdate update : updates) { - final UpdateFeaturesRequestData.FeatureUpdateKey item = - new UpdateFeaturesRequestData.FeatureUpdateKey(); - item.setName(update.featureName()); - item.setMaxVersionLevel(update.maxVersionLevel()); - item.setAllowDowngrade(update.allowDowngrade()); - items.add(item); - } - return new UpdateFeaturesRequestData().setFeatureUpdates(items); - } -} +} \ No newline at end of file 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 1f3d78366bda9..849855a0eb91e 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 @@ -143,6 +143,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; @@ -4342,9 +4344,11 @@ private static byte[] getSaltedPasword(ScramMechanism publicScramMechanism, byte 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(); - Call callViaLeastLoadedNode = new Call("describeFeatures", calcDeadlineMs(now, options.timeoutMs()), - new LeastLoadedNodeProvider()) { + Call call = new Call( + "describeFeatures", calcDeadlineMs(now, options.timeoutMs()), provider) { @Override ApiVersionsRequest.Builder createRequest(int timeoutMs) { @@ -4360,36 +4364,7 @@ void handleResponse(AbstractResponse response) { apiVersionsResponse.finalizedFeatures(), apiVersionsResponse.finalizedFeaturesEpoch(), apiVersionsResponse.supportedFeatures())); - } else { - future.completeExceptionally( - Errors.forCode(apiVersionsResponse.data.errorCode()).exception()); - } - } - - @Override - void handleFailure(Throwable throwable) { - completeAllExceptionally(Collections.singletonList(future), throwable); - } - }; - - Call callViaControllerNode = new Call("describeFeatures", calcDeadlineMs(now, options.timeoutMs()), - new ControllerNodeProvider()) { - - @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( - new FeatureMetadata( - apiVersionsResponse.finalizedFeatures(), - apiVersionsResponse.finalizedFeaturesEpoch(), - apiVersionsResponse.supportedFeatures())); - } else if (apiVersionsResponse.data.errorCode() == Errors.NOT_CONTROLLER.code()) { + } else if (options.sendRequestToController() && apiVersionsResponse.data.errorCode() == Errors.NOT_CONTROLLER.code()) { handleNotControllerError(Errors.NOT_CONTROLLER); } else { future.completeExceptionally( @@ -4403,52 +4378,87 @@ void handleFailure(Throwable throwable) { } }; - Call call; - if (options.sendRequestToController()) { - call = callViaControllerNode; - } else { - call = callViaLeastLoadedNode; - } runnable.call(call, now); return new DescribeFeaturesResult(future); } @Override public UpdateFeaturesResult updateFeatures( - final Set featureUpdates, final UpdateFeaturesOptions options) { - final KafkaFutureImpl future = new KafkaFutureImpl<>(); - final long now = time.milliseconds(); + final Map featureUpdates, final UpdateFeaturesOptions options) { + if (featureUpdates == null || featureUpdates.isEmpty()) { + throw new IllegalArgumentException("Feature updates can not be null or empty."); + } + Objects.requireNonNull(options, "UpdateFeaturesOptions can not be null"); + + final Map> updateFutures = new HashMap<>(); + final UpdateFeaturesRequestData.FeatureUpdateKeyCollection featureUpdatesRequestData + = new UpdateFeaturesRequestData.FeatureUpdateKeyCollection(); + for (Map.Entry entry : featureUpdates.entrySet()) { + String feature = entry.getKey(); + FeatureUpdate update = entry.getValue(); + if (feature.trim().isEmpty()) { + throw new IllegalArgumentException("Provided feature can not be null or empty."); + } + + updateFutures.put(feature, new KafkaFutureImpl<>()); + final UpdateFeaturesRequestData.FeatureUpdateKey requestItem = + new UpdateFeaturesRequestData.FeatureUpdateKey(); + requestItem.setFeature(feature); + requestItem.setMaxVersionLevel(update.maxVersionLevel()); + requestItem.setAllowDowngrade(update.allowDowngrade()); + featureUpdatesRequestData.add(requestItem); + } + final UpdateFeaturesRequestData request = new UpdateFeaturesRequestData().setFeatureUpdates(featureUpdatesRequestData); - final Call call = new Call("updateFinalizedFeatures", calcDeadlineMs(now, options.timeoutMs()), + final long now = time.milliseconds(); + final Call call = new Call("updateFeatures", calcDeadlineMs(now, options.timeoutMs()), new ControllerNodeProvider()) { @Override UpdateFeaturesRequest.Builder createRequest(int timeoutMs) { - return new UpdateFeaturesRequest.Builder(FeatureUpdate.createRequest(featureUpdates)); + return new UpdateFeaturesRequest.Builder(request); } @Override - void handleResponse(AbstractResponse response) { - final UpdateFeaturesResponse featuresResponse = - (UpdateFeaturesResponse) response; - final Errors error = Errors.forCode(featuresResponse.data().errorCode()); - if (error == Errors.NONE) { - future.complete(null); - } else if (error == Errors.NOT_CONTROLLER) { - handleNotControllerError(error); - } else { - future.completeExceptionally( - error.exception(featuresResponse.data.errorMessage())); + void handleResponse(AbstractResponse abstractResponse) { + final UpdateFeaturesResponse response = + (UpdateFeaturesResponse) abstractResponse; + + // Check for controller change. + for (UpdatableFeatureResult result : response.data().results()) { + Errors error = Errors.forCode(result.errorCode()); + if (error == Errors.NOT_CONTROLLER) { + handleNotControllerError(error); + throw error.exception(); + } } + + for (UpdatableFeatureResult result : response.data().results()) { + KafkaFutureImpl future = updateFutures.get(result.feature()); + if (future == null) { + log.warn("Server response mentioned unknown feature {}", result.feature()); + } else { + 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); } @Override void handleFailure(Throwable throwable) { - completeAllExceptionally(Collections.singletonList(future), throwable); + completeAllExceptionally(updateFutures.values(), throwable); } }; + runnable.call(call, now); - return new UpdateFeaturesResult(future); + return new UpdateFeaturesResult(new HashMap<>(updateFutures)); } /** 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 index 15de027c4d754..e936f09ca1955 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/UpdateFeaturesOptions.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/UpdateFeaturesOptions.java @@ -16,16 +16,14 @@ */ package org.apache.kafka.clients.admin; +import java.util.Set; import org.apache.kafka.common.annotation.InterfaceStability; +/** + * Options for {@link AdminClient#updateFeatures(Set, UpdateFeaturesOptions)} + * + * The API of this class is evolving. See {@link AdminClient} for details. + */ @InterfaceStability.Evolving public class UpdateFeaturesOptions extends AbstractOptions { - /** - * Sets the timeout in milliseconds for this operation or {@code null} if the default API - * timeout for the AdminClient should be used. - */ - public UpdateFeaturesOptions timeoutMs(final Integer timeoutMs) { - this.timeoutMs = timeoutMs; - return this; - } } 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 index c20cfe9a42a23..3c4e52f1ba82b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/UpdateFeaturesResult.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/UpdateFeaturesResult.java @@ -16,16 +16,28 @@ */ package org.apache.kafka.clients.admin; +import java.util.Map; import org.apache.kafka.common.KafkaFuture; public class UpdateFeaturesResult { - private final KafkaFuture future; + private final Map> futures; - public UpdateFeaturesResult(final KafkaFuture future) { - this.future = future; + /** + * @param futures a map from feature names to future, which can be used to check the status of + * individual feature updates. + */ + public UpdateFeaturesResult(final Map> futures) { + this.futures = futures; } - public KafkaFuture result() { - return future; + 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/feature/FinalizedVersionRange.java b/clients/src/main/java/org/apache/kafka/common/feature/FinalizedVersionRange.java index 35c08f19b520d..811712df4fbc4 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 @@ -50,8 +50,4 @@ public static FinalizedVersionRange fromMap(Map versionRangeMap) public boolean isIncompatibleWith(SupportedVersionRange supportedVersionRange) { return min() < supportedVersionRange.min() || max() > supportedVersionRange.max(); } - - public boolean isCompatibleWith(SupportedVersionRange supportedVersionRange) { - return !isIncompatibleWith(supportedVersionRange); - } } 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 54eb17bab754a..74142c4fa5a57 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 @@ -338,7 +338,7 @@ public enum Errors { 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), - FINALIZED_FEATURE_UPDATE_FAILED(96, "Unable to update finalized features.", FeatureUpdateFailedException::new); + FEATURE_UPDATE_FAILED(96, "Unable to update finalized features.", FeatureUpdateFailedException::new); private static final Logger log = LoggerFactory.getLogger(Errors.class); 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 a357e9ad78978..35dc00a9e7764 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 @@ -86,9 +86,7 @@ public Features finalizedFeatures() { final Map features = new HashMap<>(); for (FinalizedFeatureKey key : data.finalizedFeatures().valuesSet()) { - features.put( - key.name(), - new FinalizedVersionRange(key.minVersionLevel(), key.maxVersionLevel())); + features.put(key.name(), new FinalizedVersionRange(key.minVersionLevel(), key.maxVersionLevel())); } return Features.finalizedFeatures(features); 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 index 14da1961a5235..e779c723aa34e 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesRequest.java @@ -17,8 +17,11 @@ 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; @@ -44,7 +47,7 @@ public String toString() { } } - public final UpdateFeaturesRequestData data; + private final UpdateFeaturesRequestData data; public UpdateFeaturesRequest(UpdateFeaturesRequestData data, short version) { super(ApiKeys.UPDATE_FEATURES, version); @@ -59,10 +62,15 @@ public UpdateFeaturesRequest(Struct struct, short version) { @Override public AbstractResponse getErrorResponse(int throttleTimeMsIgnored, Throwable e) { final ApiError apiError = ApiError.fromThrowable(e); - return new UpdateFeaturesResponse( - new UpdateFeaturesResponseData() + UpdatableFeatureResultCollection results = new UpdatableFeatureResultCollection(); + for (FeatureUpdateKey update : this.data.featureUpdates().valuesSet()) { + UpdatableFeatureResult result = new UpdatableFeatureResult() + .setFeature(update.feature()) .setErrorCode(apiError.error().code()) - .setErrorMessage(apiError.message())); + .setErrorMessage(apiError.message()); + results.add(result); + } + return new UpdateFeaturesResponse(new UpdateFeaturesResponseData().setResults(results)); } @Override @@ -80,6 +88,6 @@ public static UpdateFeaturesRequest parse(ByteBuffer buffer, short version) { } public static boolean isDeleteRequest(UpdateFeaturesRequestData.FeatureUpdateKey update) { - return update.maxVersionLevel() < 1; + 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 index d41bc809a40f9..ef98ebad5f423 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesResponse.java @@ -17,7 +17,10 @@ package org.apache.kafka.common.requests; import java.nio.ByteBuffer; +import java.util.HashMap; import java.util.Map; +import java.util.stream.Collectors; +import org.apache.kafka.common.config.ConfigResource; import org.apache.kafka.common.message.UpdateFeaturesResponseData; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; @@ -34,7 +37,7 @@ */ public class UpdateFeaturesResponse extends AbstractResponse { - public final UpdateFeaturesResponseData data; + private final UpdateFeaturesResponseData data; public UpdateFeaturesResponse(UpdateFeaturesResponseData data) { this.data = data; @@ -49,13 +52,16 @@ public UpdateFeaturesResponse(Struct struct, short version) { this.data = new UpdateFeaturesResponseData(struct, version); } - public Errors error() { - return Errors.forCode(data.errorCode()); + 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 errorCounts(Errors.forCode(data.errorCode())); + return apiErrorCounts(errors()); } @Override diff --git a/clients/src/main/resources/common/message/UpdateFeaturesRequest.json b/clients/src/main/resources/common/message/UpdateFeaturesRequest.json index 6a134e8aab2a1..d2bf14b1e9d53 100644 --- a/clients/src/main/resources/common/message/UpdateFeaturesRequest.json +++ b/clients/src/main/resources/common/message/UpdateFeaturesRequest.json @@ -22,11 +22,11 @@ "fields": [ { "name": "FeatureUpdates", "type": "[]FeatureUpdateKey", "versions": "0+", "about": "The list of updates to finalized features.", "fields": [ - {"name": "Name", "type": "string", "versions": "0+", "mapKey": true, + {"name": "Feature", "type": "string", "versions": "0+", "mapKey": true, "about": "The name of the finalized feature to be updated."}, - {"name": "MaxVersionLevel", "type": "int16", "versions": "0+", + {"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+", + {"name": "AllowDowngrade", "type": "bool", "versions": "0+", "about": "When set to true, the finalized feature version level is allowed to be downgraded/deleted."} ]} ] diff --git a/clients/src/main/resources/common/message/UpdateFeaturesResponse.json b/clients/src/main/resources/common/message/UpdateFeaturesResponse.json index 05965db920f66..1aaf4d8c8b914 100644 --- a/clients/src/main/resources/common/message/UpdateFeaturesResponse.json +++ b/clients/src/main/resources/common/message/UpdateFeaturesResponse.json @@ -20,9 +20,14 @@ "validVersions": "0", "flexibleVersions": "0+", "fields": [ - { "name": "ErrorCode", "type": "int16", "versions": "0+", - "about": "The error code or `0` if there was no error." }, - { "name": "ErrorMessage", "type": "string", "versions": "0+", "nullableVersions": "0+", - "about": "The error message or `null` if there was no 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 091f3c0125f88..f7cdb7c0d278b 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 @@ -116,6 +116,8 @@ import org.apache.kafka.common.message.OffsetDeleteResponseData.OffsetDeleteResponseTopic; import org.apache.kafka.common.message.OffsetDeleteResponseData.OffsetDeleteResponseTopicCollection; 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.quota.ClientQuotaAlteration; @@ -484,26 +486,26 @@ private static DescribeGroupsResponseData prepareDescribeGroupsResponseData(Stri return data; } - private static UpdateFeaturesResponse prepareUpdateFeaturesResponse(Errors error) { - final UpdateFeaturesResponseData data = new UpdateFeaturesResponseData(); - data.setErrorCode(error.code()); - return new UpdateFeaturesResponse(data); + private static UpdateFeaturesResponse prepareUpdateFeaturesResponse(Map featureUpdateErrors) { + final UpdatableFeatureResultCollection results = new UpdatableFeatureResultCollection(); + for (Map.Entry entry : featureUpdateErrors.entrySet()) { + UpdatableFeatureResult result = new UpdatableFeatureResult(); + result.setFeature(entry.getKey()); + Errors error = entry.getValue(); + result.setErrorCode(error.code()); + result.setErrorMessage(error.message()); + results.add(result); + } + return new UpdateFeaturesResponse( new UpdateFeaturesResponseData().setResults(results)); } - private static FeatureMetadata getDefaultFeatureMetadata() { + private static FeatureMetadata defaultFeatureMetadata() { return new FeatureMetadata( - Features.finalizedFeatures(new HashMap() { - { - put("test_feature_1", new FinalizedVersionRange((short) 2, (short) 3)); - } - }), + Features.finalizedFeatures( + Utils.mkMap(Utils.mkEntry("test_feature_1", new FinalizedVersionRange((short) 2, (short) 3)))), 1, - Features.supportedFeatures(new HashMap() { - { - put("test_feature_1", new SupportedVersionRange((short) 1, (short) 5)); - } - }) - ); + Features.supportedFeatures( + Utils.mkMap(Utils.mkEntry("test_feature_1", new SupportedVersionRange((short) 1, (short) 5))))); } private static ApiVersionsResponse prepareApiVersionsResponseForDescribeFeatures(Errors error) { @@ -512,9 +514,9 @@ private static ApiVersionsResponse prepareApiVersionsResponseForDescribeFeatures ApiVersionsResponse.DEFAULT_API_VERSIONS_RESPONSE.throttleTimeMs(), error, ApiVersionsResponse.DEFAULT_API_VERSIONS_RESPONSE.data().apiKeys(), - getDefaultFeatureMetadata().supportedFeatures(), - getDefaultFeatureMetadata().finalizedFeatures(), - getDefaultFeatureMetadata().finalizedFeaturesEpoch().orElse(-1))); + defaultFeatureMetadata().supportedFeatures(), + defaultFeatureMetadata().finalizedFeatures(), + defaultFeatureMetadata().finalizedFeaturesEpoch().orElse(-1))); } return new ApiVersionsResponse(new ApiVersionsResponseData().setErrorCode(error.code())); } @@ -3931,38 +3933,66 @@ public void testListOffsetsNonRetriableErrors() throws Exception { @Test public void testUpdateFeaturesDuringSuccess() throws Exception { - testUpdateFeaturesDuringError(Errors.NONE); + testUpdateFeatures( + makeTestFeatureUpdates(), + makeTestFeatureUpdateErrors(Errors.NONE)); } @Test public void testUpdateFeaturesInvalidRequestError() throws Exception { - testUpdateFeaturesDuringError(Errors.INVALID_REQUEST); + testUpdateFeatures( + makeTestFeatureUpdates(), + makeTestFeatureUpdateErrors(Errors.INVALID_REQUEST)); } @Test public void testUpdateFeaturesUpdateFailedError() throws Exception { - testUpdateFeaturesDuringError(Errors.FEATURE_UPDATE_FAILED); + testUpdateFeatures( + makeTestFeatureUpdates(), + makeTestFeatureUpdateErrors(Errors.FEATURE_UPDATE_FAILED)); + } + + @Test + public void testUpdateFeaturesPartialSuccess() throws Exception { + final Map errors = makeTestFeatureUpdateErrors(Errors.NONE); + errors.put("test_feature_2", Errors.INVALID_REQUEST); + testUpdateFeatures(makeTestFeatureUpdates(), errors); + } + + 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 Errors error) { + final Map updates = makeTestFeatureUpdates(); + final Map errors = new HashMap<>(); + for (Map.Entry entry : updates.entrySet()) { + errors.put(entry.getKey(), error); + } + return errors; } - private void testUpdateFeaturesDuringError(Errors error) throws Exception { + private void testUpdateFeatures(Map featureUpdates, + Map featureUpdateErrors) throws Exception { try (final AdminClientUnitTestEnv env = mockClientEnv()) { env.kafkaClient().prepareResponse( body -> body instanceof UpdateFeaturesRequest, - prepareUpdateFeaturesResponse(error)); - final KafkaFuture future = env.adminClient().updateFeatures( - new HashSet<>( - Arrays.asList( - new FeatureUpdate( - "test_feature_1", (short) 2, false), - new FeatureUpdate( - "test_feature_2", (short) 3, true))), - new UpdateFeaturesOptions().timeoutMs(10000)).result(); - if (error.exception() == null) { - future.get(); - } else { - final ExecutionException e = assertThrows(ExecutionException.class, - () -> future.get()); - assertEquals(e.getCause().getClass(), error.exception().getClass()); + prepareUpdateFeaturesResponse(featureUpdateErrors)); + final Map> futures = env.adminClient().updateFeatures( + featureUpdates, + new UpdateFeaturesOptions().timeoutMs(10000)).values(); + for (Map.Entry> entry : futures.entrySet()) { + final KafkaFuture future = entry.getValue(); + final Errors error = featureUpdateErrors.get(entry.getKey()); + if (error == Errors.NONE) { + future.get(); + } else { + final ExecutionException e = assertThrows(ExecutionException.class, + () -> future.get()); + assertEquals(e.getCause().getClass(), error.exception().getClass()); + } } } } @@ -3971,23 +4001,28 @@ private void testUpdateFeaturesDuringError(Errors error) throws Exception { public void testUpdateFeaturesHandleNotControllerException() throws Exception { try (final AdminClientUnitTestEnv env = mockClientEnv()) { env.kafkaClient().prepareResponseFrom( - prepareUpdateFeaturesResponse(Errors.NOT_CONTROLLER), + request -> request instanceof UpdateFeaturesRequest, + prepareUpdateFeaturesResponse(Utils.mkMap( + Utils.mkEntry("test_feature_1", Errors.NOT_CONTROLLER), + Utils.mkEntry("test_feature_2", Errors.NOT_CONTROLLER))), env.cluster().nodeById(0)); + final int controllerId = 1; env.kafkaClient().prepareResponse(MetadataResponse.prepareResponse(env.cluster().nodes(), env.cluster().clusterResource().clusterId(), - 1, - Collections.emptyList())); + controllerId, + Collections.emptyList())); env.kafkaClient().prepareResponseFrom( - prepareUpdateFeaturesResponse(Errors.NONE), - env.cluster().nodeById(1)); + request -> request instanceof UpdateFeaturesRequest, + prepareUpdateFeaturesResponse(Utils.mkMap( + Utils.mkEntry("test_feature_1", Errors.NONE), + Utils.mkEntry("test_feature_2", Errors.NONE))), + env.cluster().nodeById(controllerId)); final KafkaFuture future = env.adminClient().updateFeatures( - new HashSet<>( - Arrays.asList( - new FeatureUpdate( - "test_feature_1", (short) 2, false), - new FeatureUpdate( - "test_feature_2", (short) 3, true))), - new UpdateFeaturesOptions().timeoutMs(10000)).result(); + 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(); } } @@ -4000,8 +4035,8 @@ public void testDescribeFeaturesSuccess() throws Exception { prepareApiVersionsResponseForDescribeFeatures(Errors.NONE)); final KafkaFuture future = env.adminClient().describeFeatures( new DescribeFeaturesOptions().timeoutMs(10000)).featureMetadata(); - FeatureMetadata metadata = future.get(); - assertEquals(getDefaultFeatureMetadata(), metadata); + final FeatureMetadata metadata = future.get(); + assertEquals(defaultFeatureMetadata(), metadata); } } @@ -4014,7 +4049,7 @@ public void testDescribeFeaturesHandleNotControllerException() throws Exception env.kafkaClient().prepareResponse(MetadataResponse.prepareResponse(env.cluster().nodes(), env.cluster().clusterResource().clusterId(), 1, - Collections.emptyList())); + Collections.emptyList())); env.kafkaClient().prepareResponseFrom( prepareApiVersionsResponseForDescribeFeatures(Errors.NONE), env.cluster().nodeById(1)); 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 86e160ebc9323..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 @@ -842,7 +842,7 @@ public DescribeFeaturesResult describeFeatures(DescribeFeaturesOptions options) } @Override - public UpdateFeaturesResult updateFeatures(Set featureUpdates, UpdateFeaturesOptions options) { + public UpdateFeaturesResult updateFeatures(Map featureUpdates, UpdateFeaturesOptions options) { throw new UnsupportedOperationException("Not implemented yet"); } diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 6009c775dba0a..4a1f76761319f 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -38,10 +38,11 @@ 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 @@ -63,7 +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 = (Errors, Option[String]) => Unit + type UpdateFeaturesCallback = (Map[String, ApiError]) => Unit } class KafkaController(val config: KafkaConfig, @@ -301,24 +302,24 @@ class KafkaController(val config: KafkaConfig, * 1. Supported feature: * A supported feature is represented by a name (String) and a range of versions (defined by a * {@link SupportedVersionRange}). It refers to a feature that a particular broker advertises - * support for. Each broker advertises the version ranges of it’s own supported features in its - * own BrokerIdZnode. The contents of the advertisement are specific to the particular broker and + * 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 is represented by a name (String) and a range of version levels (defined + * A finalized feature is represented by a name (String) and a range of version levels (defined * by a {@link FinalizedVersionRange}). Whenever the feature versioning system (KIP-584) is - * enabled, the finalized features are stored in ZK in the cluster-wide common FeatureZNode. + * 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 one and only entity modifying - * the information about finalized features and their version levels. + * 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. This status means the feature - * versioning system (KIP-584) is enabled, and, the finalized features stored in the FeatureZNode - * are active. This 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. + * 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: * @@ -332,17 +333,17 @@ class KafkaController(val config: KafkaConfig, * * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0: * Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the - * Broker binary has been upgraded to a newer version that supports the feature versioning - * system (KIP-584). This means the user is upgrading from an earlier version of the Broker + * broker binary has been upgraded to a newer version that 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 * 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. The reason to do this is that enabling all the possible features immediately after - * an upgrade could be harmful to the cluster. + * 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 absent, then it - * will react by creating a FeatureZNode with disabled status and empty finalized features. + * controller will start up and check if the FeatureZNode is absent. If 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 @@ -359,7 +360,7 @@ class KafkaController(val config: KafkaConfig, * existing finalized features and mutate them for the purpose of version level deprecation * (if needed). * This is how we handle this case: If an existing finalized feature is present in the default - * finalized features, then, it's existing minimum version level is updated to the default + * finalized features, then, its existing minimum version level is updated to the default * minimum version level maintained in the BrokerFeatures object. The goal of this mutation is * to permanently deprecate one or more feature version levels. The range of feature version * levels deprecated are from the closed range: [existing_min_version_level, default_min_version_level]. @@ -385,14 +386,14 @@ class KafkaController(val config: KafkaConfig, var newFeatures: Features[FinalizedVersionRange] = Features.emptyFinalizedFeatures() if (existingFeatureZNode.status.equals(FeatureZNodeStatus.Enabled)) { newFeatures = Features.finalizedFeatures(existingFeatureZNode.features.features().asScala.map { - case (featureName, existingVersionRange) => { + case (featureName, existingVersionRange) => val brokerDefaultVersionRange = defaultFinalizedFeatures.get(featureName) if (brokerDefaultVersionRange == null) { warn(s"Existing finalized feature: $featureName with $existingVersionRange" + s" is absent in default finalized $defaultFinalizedFeatures") (featureName, existingVersionRange) - } else if (existingVersionRange.max() >= brokerDefaultVersionRange.min() && - brokerDefaultVersionRange.max() >= existingVersionRange.max()) { + } else if (brokerDefaultVersionRange.max() >= existingVersionRange.max() && + existingVersionRange.max() >= brokerDefaultVersionRange.min()) { // Through this change, we deprecate all version levels in the closed range: // [existingVersionRange.min(), brokerDefaultVersionRange.min() - 1] (featureName, new FinalizedVersionRange(brokerDefaultVersionRange.min(), existingVersionRange.max())) @@ -406,7 +407,6 @@ class KafkaController(val config: KafkaConfig, + s" based on the default $brokerDefaultVersionRange.") (featureName, existingVersionRange) } - } }.asJava) } val newFeatureZNode = new FeatureZNode(FeatureZNodeStatus.Enabled, newFeatures) @@ -429,7 +429,7 @@ class KafkaController(val config: KafkaConfig, * 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 caceh (via FinalizedFeatureChangeListener) + * 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 = { @@ -1182,16 +1182,13 @@ class KafkaController(val config: KafkaConfig, val filteredBrokers = scala.collection.mutable.Set[Int]() ++ brokers if (config.isFeatureVersioningEnabled) { def hasIncompatibleFeatures(broker: Broker): Boolean = { - val latestFinalizedFeatures = featureCache.get - if (latestFinalizedFeatures.isDefined) { - BrokerFeatures.hasIncompatibleFeatures(broker.features, latestFinalizedFeatures.get.features) - } else { - false - } + featureCache.get.exists( + latestFinalizedFeatures => + BrokerFeatures.hasIncompatibleFeatures(broker.features, latestFinalizedFeatures.features)) } controllerContext.liveOrShuttingDownBrokers.foreach(broker => { if (filteredBrokers.contains(broker.id) && hasIncompatibleFeatures(broker)) { - warn(s"Ignoring UpdateMetadataRequest to broker: ${broker.id} due to incompatible features") + warn(s"No UpdateMetadataRequest will be sent to broker: ${broker.id} due to incompatible features") filteredBrokers -= broker.id } }) @@ -1856,33 +1853,189 @@ class KafkaController(val config: KafkaConfig, } } - private def processUpdateFeatures(newFeatures: Features[FinalizedVersionRange], - callback: UpdateFeaturesCallback): Unit = { - if (isActive) { - val incompatibleBrokers = controllerContext.liveOrShuttingDownBrokers.filter(broker => { - BrokerFeatures.hasIncompatibleFeatures(broker.features, newFeatures) - }) - if (incompatibleBrokers.nonEmpty) { - callback( - Errors.INVALID_REQUEST, - Some( - s"Could not apply finalized feature updates because ${incompatibleBrokers.size} brokers" - + s" were found to have incompatible features. newFeatures: $newFeatures" - + s", incompatibleBrokers: $incompatibleBrokers.")) + /** + * Returns the new FinalizedVersionRange for the feature, if there are no feature + * incompatibilities with all known brokers. Otherwise returns a suitable error. + * + * @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 newFinalizedVersionRangeOrError(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") + } + // NOTE: Below we set the finalized min version level to be the default minimum version + // level. If the finalized feature already exists, then, this can cause deprecation of all + // version levels in the closed range: + // [existingVersionRange.min(), defaultMinVersionLevel - 1]. + val defaultMinVersionLevel = brokerFeatures.defaultMinVersionLevel(update.feature) + val newVersionRange = new FinalizedVersionRange(defaultMinVersionLevel, update.maxVersionLevel) + val numIncompatibleBrokers = controllerContext.liveOrShuttingDownBrokers.count(broker => { + val singleFinalizedFeature = + Features.finalizedFeatures(Utils.mkMap(Utils.mkEntry(update.feature, newVersionRange))) + BrokerFeatures.hasIncompatibleFeatures(broker.features, singleFinalizedFeature) + }) + if (numIncompatibleBrokers == 0) { + Left(newVersionRange) + } else { + Right( + new ApiError(Errors.INVALID_REQUEST, + s"Could not apply finalized feature update because $numIncompatibleBrokers" + + " brokers were found to have incompatible features.")) + } + } + + /** + * Validate and process a finalized feature update. + * + * If processed successfully, 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 processing failed, then returned value contains a suitable ApiError. + * + * @param update the feature update to be processed. + * + * @return the new FinalizedVersionRange or error, as described above. + */ + private def processFeatureUpdate(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[Option[FinalizedVersionRange], ApiError] = { + val existingFeatures = featureCache.get + .map(featuresAndEpoch => featuresAndEpoch.features.features().asScala) + .getOrElse(Map[String, FinalizedVersionRange]()) + + def newVersionRangeOrError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[Option[FinalizedVersionRange], ApiError] = { + newFinalizedVersionRangeOrError(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 { + val cacheEntry = existingFeatures.get(update.feature).orNull + + // We handle deletion requests separately from non-deletion requests. + if (UpdateFeaturesRequest.isDeleteRequest(update)) { + if (cacheEntry == null) { + // Disallow deletion of a non-existing finalized feature. + Right( + new ApiError(Errors.INVALID_REQUEST, + s"Can not delete non-existing finalized feature: '${update.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 for feature: '${update.feature}' without setting the" + + s" allowDowngrade flag to true in the request.")) } else { - try { - val newVersion = zkClient.updateFeatureZNode( - new FeatureZNode(FeatureZNodeStatus.Enabled, newFeatures)) - featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs) - callback(Errors.NONE, Option.empty) - } catch { - case e: Exception => callback( - Errors.FEATURE_UPDATE_FAILED, - Some(Errors.FEATURE_UPDATE_FAILED.message() + " Error: " + e)) + if (cacheEntry == null) { + newVersionRangeOrError(update) + } else { + if (update.maxVersionLevel == cacheEntry.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: '${update.feature}' from existing" + + s" maxVersionLevel:${cacheEntry.max} to the same value.")) + } else if (update.maxVersionLevel < cacheEntry.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: '${update.feature}' from" + + s" existing maxVersionLevel:${cacheEntry.max} to provided" + + s" maxVersionLevel:${update.maxVersionLevel} without setting the" + + " allowDowngrade flag in the request.")) + } else if (update.allowDowngrade && update.maxVersionLevel > cacheEntry.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 finalized feature: '${update.feature}' has the allowDowngrade" + + s" flag set in the request, the provided" + + s" maxVersionLevel:${update.maxVersionLevel} can not be greater than" + + s" existing maxVersionLevel:${cacheEntry.max}.")) + } else if (update.maxVersionLevel() < cacheEntry.min()) { + // Disallow downgrade of a finalized feature below the existing finalized + // minVersionLevel. + Right( + new ApiError(Errors.INVALID_REQUEST, + s"Can not downgrade finalized feature: '${update.feature}' to" + + s" maxVersionLevel:${update.maxVersionLevel} because it's lower than" + + s" the existing minVersionLevel:${cacheEntry.min}.")) + } else { + newVersionRangeOrError(update) + } } } + } + } + + private def processFeatureUpdates(request: UpdateFeaturesRequest, + callback: UpdateFeaturesCallback): Unit = { + if (isActive) { + processFeatureUpdatesWithActiveController(request, callback) } else { - callback(Errors.NOT_CONTROLLER, Option.empty) + val results = request.data().featureUpdates().asScala.map { + update => update.feature() -> new ApiError(Errors.NOT_CONTROLLER) + }.toMap + callback(results) + } + } + + 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]()) + // Map of feature to FinalizedVersionRange. This contains the target features to be eventually + // written to FeatureZNode. + val targetFeatures = scala.collection.mutable.Map[String, FinalizedVersionRange]() ++ existingFeatures + // Map of feature to error. + var errors = scala.collection.mutable.Map[String, ApiError]() + + // Process each FeatureUpdate, insert an entry into errors and mutate targetFeatures suitably. + // If a FeatureUpdate is found to be valid, then the corresponding entry in errors contains + // Errors.NONE. Otherwise the entry contains the appropriate error. + updates.asScala.iterator.foreach { update => + processFeatureUpdate(update) match { + case Left(newVersionRangeOrNone) => + newVersionRangeOrNone + .map(newVersionRange => targetFeatures += (update.feature() -> newVersionRange)) + .getOrElse(targetFeatures -= update.feature()) + errors += (update.feature() -> new ApiError(Errors.NONE)) + case Right(featureUpdateFailureReason) => errors += (update.feature() -> featureUpdateFailureReason) + } + } + + if (existingFeatures.equals(targetFeatures)) { + callback(errors) + } else { + try { + val newNode = new FeatureZNode(FeatureZNodeStatus.Enabled, Features.finalizedFeatures(targetFeatures.asJava)) + val newVersion = zkClient.updateFeatureZNode(newNode) + featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs) + } 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 => + errors = errors.map { case (feature, apiError) => + if (apiError.error() == Errors.NONE) { + (feature, new ApiError(Errors.FEATURE_UPDATE_FAILED, + Errors.FEATURE_UPDATE_FAILED.message() + " Error: " + e)) + } else { + (feature, apiError) + } + } + } finally { + callback(errors) + } } } @@ -1920,9 +2073,9 @@ class KafkaController(val config: KafkaConfig, eventManager.put(ListPartitionReassignments(partitions, callback)) } - def updateFeatures(newFeatures: Features[FinalizedVersionRange], + def updateFeatures(request: UpdateFeaturesRequest, callback: UpdateFeaturesCallback): Unit = { - eventManager.put(UpdateFeatures(newFeatures, callback)) + eventManager.put(UpdateFeatures(request, callback)) } def alterPartitionReassignments(partitions: Map[TopicPartition, Option[Seq[Int]]], @@ -2215,7 +2368,7 @@ class KafkaController(val config: KafkaConfig, case ListPartitionReassignments(partitions, callback) => processListPartitionReassignments(partitions, callback) case UpdateFeatures(request, callback) => - processUpdateFeatures(request, callback) + processFeatureUpdates(request, callback) case PartitionReassignmentIsrChange(partition) => processPartitionReassignmentIsrChange(partition) case IsrChangeNotification => @@ -2509,7 +2662,7 @@ case class ListPartitionReassignments(partitionsOpt: Option[Set[TopicPartition]] override def preempt(): Unit = callback(Right(new ApiError(Errors.NOT_CONTROLLER, null))) } -case class UpdateFeatures(newFeatures: Features[FinalizedVersionRange], +case class UpdateFeatures(request: UpdateFeaturesRequest, callback: UpdateFeaturesCallback) extends ControllerEvent { override def state: ControllerState = ControllerState.UpdateFeatures } diff --git a/core/src/main/scala/kafka/server/BrokerFeatures.scala b/core/src/main/scala/kafka/server/BrokerFeatures.scala index 734cf15249e3e..70861495cd368 100644 --- a/core/src/main/scala/kafka/server/BrokerFeatures.scala +++ b/core/src/main/scala/kafka/server/BrokerFeatures.scala @@ -31,15 +31,15 @@ import scala.jdk.CollectionConverters._ * 2. The default minimum version levels for specific features. This map enables feature * version level deprecation. This is how it works: in order to deprecate feature version levels, * in this map the default minimum version level of a feature can be set to a new value that's - * higher than 1 (let's call this new_min_version_level). In doing so, the feature version levels + * higher than 1 (let's call this latest_min_version_level). In doing so, the feature version levels * in the closed range: [1, latest_min_version_level - 1] get deprecated by the controller logic * that applies this map to persistent finalized feature state in ZK (this mutation happens * during controller election and during finalized feature updates via the - * ApiKeys.UPDATE_FINALIZED_FEATURES api). This will automatically mean clients have to stop - * using the finalized min version levels that have been deprecated. + * ApiKeys.UPDATE_FINALIZED_FEATURES api). This will automatically mean external clients of Kafka + * would need to stop using the finalized min version levels that have been deprecated. * * This class also provides APIs to check for incompatibilities between the features supported by - * the Broker and finalized features. The class is generally immutable. It provides few APIs to + * 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], @@ -86,7 +86,7 @@ class BrokerFeatures private (@volatile var supportedFeatures: Features[Supporte } /** - * Returns the set of feature names found to be 'incompatible'. + * 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: @@ -171,8 +171,8 @@ object BrokerFeatures extends Logging { case(featureName, minVersionLevel) => val supportedFeature = supportedFeatures.get(featureName) (supportedFeature != null) && - new FinalizedVersionRange(minVersionLevel, supportedFeature.max()) - .isCompatibleWith(supportedFeature) + !new FinalizedVersionRange(minVersionLevel, supportedFeature.max()) + .isIncompatibleWith(supportedFeature) } } } diff --git a/core/src/main/scala/kafka/server/FinalizedFeatureCache.scala b/core/src/main/scala/kafka/server/FinalizedFeatureCache.scala index 774d3636b05e5..ddd7f9854a63b 100644 --- a/core/src/main/scala/kafka/server/FinalizedFeatureCache.scala +++ b/core/src/main/scala/kafka/server/FinalizedFeatureCache.scala @@ -147,12 +147,13 @@ class FinalizedFeatureCache(private val brokerFeatures: BrokerFeatures) extends val waitEndTimeNanos = System.nanoTime() + (timeoutMs * 1_000_000) synchronized { while (!waitCondition()) { - if (System.nanoTime() > waitEndTimeNanos) { + 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 - System.nanoTime()) / 1_000_000) + val sleepTimeMs = max(1L, (waitEndTimeNanos - nowNanos) / 1_000_000) wait(sleepTimeMs) } } diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 4d45a3b36e0ae..a27c7b603ae40 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -47,13 +47,12 @@ import org.apache.kafka.common.acl.{AclBinding, AclOperation} import org.apache.kafka.common.acl.AclOperation._ import org.apache.kafka.common.config.ConfigResource import org.apache.kafka.common.errors._ -import org.apache.kafka.common.feature.{Features, FinalizedVersionRange} import org.apache.kafka.common.internals.{FatalExitError, Topic} import org.apache.kafka.common.internals.Topic.{GROUP_METADATA_TOPIC_NAME, TRANSACTION_STATE_TOPIC_NAME, isInternal} import org.apache.kafka.common.message.AlterConfigsResponseData.AlterConfigsResourceResponse import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic import org.apache.kafka.common.message.CreatePartitionsResponseData.CreatePartitionsTopicResult -import org.apache.kafka.common.message.{AddOffsetsToTxnResponseData, AlterConfigsResponseData, AlterPartitionReassignmentsResponseData, AlterReplicaLogDirsResponseData, CreateAclsResponseData, CreatePartitionsResponseData, CreateTopicsResponseData, DeleteAclsResponseData, DeleteGroupsResponseData, DeleteRecordsResponseData, DeleteTopicsResponseData, DescribeAclsResponseData, DescribeConfigsResponseData, DescribeGroupsResponseData, DescribeLogDirsResponseData, EndTxnResponseData, ExpireDelegationTokenResponseData, FindCoordinatorResponseData, HeartbeatResponseData, InitProducerIdResponseData, JoinGroupResponseData, LeaveGroupResponseData, ListGroupsResponseData, ListPartitionReassignmentsResponseData, MetadataResponseData, OffsetCommitRequestData, OffsetCommitResponseData, OffsetDeleteResponseData, RenewDelegationTokenResponseData, SaslAuthenticateResponseData, SaslHandshakeResponseData, StopReplicaResponseData, SyncGroupResponseData, UpdateFeaturesRequestData, UpdateFeaturesResponseData, UpdateMetadataResponseData} +import org.apache.kafka.common.message.{AddOffsetsToTxnResponseData, AlterConfigsResponseData, AlterPartitionReassignmentsResponseData, AlterReplicaLogDirsResponseData, CreateAclsResponseData, CreatePartitionsResponseData, CreateTopicsResponseData, DeleteAclsResponseData, DeleteGroupsResponseData, DeleteRecordsResponseData, DeleteTopicsResponseData, DescribeAclsResponseData, DescribeConfigsResponseData, DescribeGroupsResponseData, DescribeLogDirsResponseData, EndTxnResponseData, ExpireDelegationTokenResponseData, FindCoordinatorResponseData, HeartbeatResponseData, InitProducerIdResponseData, JoinGroupResponseData, LeaveGroupResponseData, ListGroupsResponseData, ListPartitionReassignmentsResponseData, MetadataResponseData, OffsetCommitRequestData, OffsetCommitResponseData, OffsetDeleteResponseData, RenewDelegationTokenResponseData, SaslAuthenticateResponseData, SaslHandshakeResponseData, StopReplicaResponseData, SyncGroupResponseData, UpdateFeaturesResponseData, UpdateMetadataResponseData} import org.apache.kafka.common.message.CreateTopicsResponseData.{CreatableTopicResult, CreatableTopicResultCollection} import org.apache.kafka.common.message.DeleteGroupsResponseData.{DeletableGroupResult, DeletableGroupResultCollection} import org.apache.kafka.common.message.AlterPartitionReassignmentsResponseData.{ReassignablePartitionResponse, ReassignableTopicResponse} @@ -91,6 +90,7 @@ import scala.collection.mutable.ArrayBuffer import scala.collection.{Map, Seq, Set, immutable, mutable} import scala.util.{Failure, Success, Try} import kafka.coordinator.group.GroupOverview +import org.apache.kafka.common.message.UpdateFeaturesResponseData.{UpdatableFeatureResult, UpdatableFeatureResultCollection} /** @@ -3113,128 +3113,35 @@ class KafkaApis(val requestChannel: RequestChannel, def handleUpdateFeatures(request: RequestChannel.Request): Unit = { val updateFeaturesRequest = request.body[UpdateFeaturesRequest] - def sendResponseCallback(error: Errors, msgOverride: Option[String]): Unit = { - val data = new UpdateFeaturesResponseData().setErrorCode(error.code()) - msgOverride.map(msg => data.setErrorMessage(msg)) - sendResponseExemptThrottle(request, new UpdateFeaturesResponse(data)) + def featureUpdateErrors(error: Errors, msgOverride: Option[String]): Map[String, ApiError] = { + updateFeaturesRequest.data().featureUpdates().asScala.map( + update => update.feature() -> new ApiError(error, msgOverride.getOrElse(error.message())) + ).toMap + } + + def sendResponseCallback(updateErrors: Map[String, ApiError]): Unit = { + val results = new UpdatableFeatureResultCollection() + updateErrors.foreach { + case (feature, error) => + val result = new UpdatableFeatureResult() + .setFeature(feature) + .setErrorCode(error.error().code()) + .setErrorMessage(error.message()) + results.add(result) + } + val responseData = new UpdateFeaturesResponseData().setResults(results) + sendResponseExemptThrottle(request, new UpdateFeaturesResponse(responseData)) } if (!authorize(request.context, ALTER, CLUSTER, CLUSTER_NAME)) { - sendResponseCallback(Errors.CLUSTER_AUTHORIZATION_FAILED, Option.empty) - } else if (!controller.isActive) { - sendResponseCallback(Errors.NOT_CONTROLLER, Option.empty) + sendResponseCallback(featureUpdateErrors(Errors.CLUSTER_AUTHORIZATION_FAILED, Option.empty)) } else if (!config.isFeatureVersioningEnabled) { - sendResponseCallback(Errors.INVALID_REQUEST, Some("Feature versioning system is disabled.")) - } else if (updateFeaturesRequest.data.featureUpdates.isEmpty) { - sendResponseCallback(Errors.INVALID_REQUEST, Some("Can not provide empty FinalizedFeatureUpdates in the request.")) + sendResponseCallback(featureUpdateErrors(Errors.INVALID_REQUEST, Some("Feature versioning system is disabled."))) } else { - val targetFeaturesOrError = getTargetFinalizedFeaturesOrError(updateFeaturesRequest) - targetFeaturesOrError match { - case Left(targetFeatures) => - controller.updateFeatures(targetFeatures, sendResponseCallback) - case Right(error) => - sendResponseCallback(error.error, Some(error.message)) - } + controller.updateFeatures(updateFeaturesRequest, sendResponseCallback) } } - /** - * Validates the provided UpdateFinalizedFeaturesRequest, checking for various error cases. - * If the validation is successful, returns the target finalized features constructed from the - * request. - * - * @param request the request to be validated - * - * @return - the target finalized features, if request validation is successful - * - an ApiError if request validation fails - */ - private def getTargetFinalizedFeaturesOrError(request: UpdateFeaturesRequest): Either[Features[FinalizedVersionRange], ApiError] = { - val updates = request.data.featureUpdates - val newFeatures = scala.collection.mutable.Map[String, FinalizedVersionRange]() - - def addFeature(update: UpdateFeaturesRequestData.FeatureUpdateKey): Unit = { - // NOTE: Below we set the finalized min version level to be the default minimum version - // level. If the finalized feature already exists, then, this can cause deprecation of all - // version levels in the closed range: - // [existingVersionRange.min(), defaultMinVersionLevel - 1]. - val defaultMinVersionLevel = brokerFeatures.defaultMinVersionLevel(update.name) - newFeatures += ( - update.name -> new FinalizedVersionRange( - defaultMinVersionLevel, - update.maxVersionLevel)) - } - - val latestFeatures = featureCache.get - updates.asScala.iterator.map( - update => { - if (update.name.isEmpty) { - // Rule #1) Check that the feature name is not empty. - Some(new ApiError(Errors.INVALID_REQUEST, - "Can not contain empty feature name in the request.")) - } else { - val cacheEntry = latestFeatures.map(lf => lf.features.get(update.name)).orNull - - // We handle deletion requests separately from non-deletion requests. - if (UpdateFeaturesRequest.isDeleteRequest(update)) { - if (!update.allowDowngrade) { - // Rule #2) Disallow deletion of a finalized feature without allowDowngrade flag set. - Some(new ApiError(Errors.INVALID_REQUEST, - s"Can not delete feature: '${update.name}' without setting the" + - " allowDowngrade flag to true in the request.")) - } else if (cacheEntry == null) { - // Rule #3) Disallow deletion of a non-existing finalized feature. - Some(new ApiError(Errors.INVALID_REQUEST, - s"Can not delete non-existing finalized feature: '${update.name}'")) - } - } else { - if (cacheEntry == null) { - addFeature(update) - } else { - if (update.maxVersionLevel == cacheEntry.max()) { - // Rule 4) Disallow a case where target maxVersionLevel matches - // existing maxVersionLevel. - Some( - new ApiError(Errors.INVALID_REQUEST, - s"Can not ${if (update.allowDowngrade) "downgrade" else "upgrade"}" + - s" a finalized feature: '${update.name}' from existing" + - s" maxVersionLevel:${cacheEntry.max} to the same value.")) - } else if (update.maxVersionLevel < cacheEntry.max && !update.allowDowngrade) { - // Rule #5) Disallow downgrade of a finalized feature without the - // allowDowngrade flag set. - Some( - new ApiError(Errors.INVALID_REQUEST, - s"Can not downgrade finalized feature: '${update.name}' from" + - s" existing maxVersionLevel:${cacheEntry.max} to provided" + - s" maxVersionLevel:${update.maxVersionLevel} without setting the" + - " allowDowngrade flag in the request.")) - } else if (update.allowDowngrade && update.maxVersionLevel > cacheEntry.max) { - // Rule #6) Disallow a request that sets allowDowngrade flag without specifying a - // maxVersionLevel that's lower than the existing maxVersionLevel. - Some( - new ApiError(Errors.INVALID_REQUEST, - s"When finalized feature: '${update.name}' has the allowDowngrade flag" + - s" set in the request, the provided maxVersionLevel:${update.maxVersionLevel}" + - s" can not be greater than existing maxVersionLevel:${cacheEntry.max}.")) - } else if (update.maxVersionLevel() < cacheEntry.min()) { - // Rule #7) Disallow downgrade of a finalized feature below the existing finalized - // minVersionLevel. - Some( - new ApiError(Errors.INVALID_REQUEST, - s"Can not downgrade finalized feature: '${update.name}' to" + - s" maxVersionLevel:${update.maxVersionLevel} because it's lower than the" + - s" existing minVersionLevel:${cacheEntry.min}.")) - } else { - addFeature(update) - } - } - } - } - } - ).collectFirst { - case Some(error) => error.asInstanceOf[ApiError] - }.map(error => Right(error)).getOrElse(Left(Features.finalizedFeatures(newFeatures.asJava))) - } - // 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..da0fec82d50e3 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -266,6 +266,7 @@ object Defaults { val PasswordEncoderIterations = 4096 } + object KafkaConfig { private val LogConfigPrefix = "log." diff --git a/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala b/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala index 524c19e16b7c6..c5e1e9a36801a 100644 --- a/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala @@ -737,7 +737,7 @@ class ControllerIntegrationTest extends ZooKeeperTestHarness { TestUtils.waitUntilControllerElected(zkClient) val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path) - assertNotEquals(version, ZkVersion.UnknownVersion) + assertEquals(0, version) val featureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get) if (interBrokerProtocolVersion >= KAFKA_2_7_IV0) { assertEquals(FeatureZNodeStatus.Enabled, featureZNode.status) diff --git a/core/src/test/scala/unit/kafka/server/BrokerFeaturesTest.scala b/core/src/test/scala/unit/kafka/server/BrokerFeaturesTest.scala index 642ecdd2a5f75..0f937b3b1462a 100644 --- a/core/src/test/scala/unit/kafka/server/BrokerFeaturesTest.scala +++ b/core/src/test/scala/unit/kafka/server/BrokerFeaturesTest.scala @@ -24,6 +24,7 @@ import org.junit.Test import scala.jdk.CollectionConverters._ class BrokerFeaturesTest { + @Test def testEmpty(): Unit = { assertTrue(BrokerFeatures.createDefault().supportedFeatures.empty) @@ -66,18 +67,18 @@ class BrokerFeaturesTest { classOf[IllegalArgumentException], () => brokerFeatures.setDefaultMinVersionLevels(defaultMinVersionLevelsWithNonExistingFeature)) - val defaultMinVersionLevelsWithInvalidValue1 = Map[String, Short]( + val defaultMinVersionLevelsWithInvalidSmallValue = Map[String, Short]( "test_feature_1" -> 2, "test_feature_2" -> (supportedFeatures.get("test_feature_2").min() - 1).asInstanceOf[Short]) assertThrows( classOf[IllegalArgumentException], - () => brokerFeatures.setDefaultMinVersionLevels(defaultMinVersionLevelsWithInvalidValue1)) + () => brokerFeatures.setDefaultMinVersionLevels(defaultMinVersionLevelsWithInvalidSmallValue)) - val defaultMinVersionLevelsWithInvalidValue2 = Map[String, Short]( + val defaultMinVersionLevelsWithInvalidLargeValue = Map[String, Short]( "test_feature_1" -> 2, "test_feature_2" -> (supportedFeatures.get("test_feature_2").max() + 1).asInstanceOf[Short]) assertThrows( classOf[IllegalArgumentException], - () => brokerFeatures.setDefaultMinVersionLevels(defaultMinVersionLevelsWithInvalidValue2)) + () => brokerFeatures.setDefaultMinVersionLevels(defaultMinVersionLevelsWithInvalidLargeValue)) } } diff --git a/core/src/test/scala/unit/kafka/server/FinalizedFeatureChangeListenerTest.scala b/core/src/test/scala/unit/kafka/server/FinalizedFeatureChangeListenerTest.scala index 1d4bee4c902dd..4d853892903e2 100644 --- a/core/src/test/scala/unit/kafka/server/FinalizedFeatureChangeListenerTest.scala +++ b/core/src/test/scala/unit/kafka/server/FinalizedFeatureChangeListenerTest.scala @@ -98,10 +98,15 @@ class FinalizedFeatureChangeListenerTest extends ZooKeeperTestHarness { 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]( diff --git a/core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala b/core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala index 6d52829112958..038eee112f790 100644 --- a/core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala +++ b/core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala @@ -18,9 +18,6 @@ package kafka.server import java.util -import java.util.Arrays -import java.util.Collections -import java.util.HashSet import java.util.Properties import java.util.concurrent.ExecutionException @@ -33,13 +30,15 @@ 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, assertNotEquals, assertNotNull, assertTrue} +import org.junit.Assert.{assertEquals, assertFalse, assertNotEquals, assertNotNull, assertTrue} import org.scalatest.Assertions.{assertThrows, intercept} +import scala.jdk.CollectionConverters._ import scala.reflect.ClassTag import scala.util.matching.Regex @@ -83,11 +82,11 @@ class UpdateFeaturesTest extends BaseRequestTest { "Controller did not get broker updates") } - private def updateSupportedFeatures(features: Features[SupportedVersionRange]): Unit = { + private def updateSupportedFeaturesInAllBrokers(features: Features[SupportedVersionRange]): Unit = { updateSupportedFeatures(features, Set[KafkaServer]() ++ servers) } - private def updateDefaultMinVersionLevels(newMinVersionLevels: Map[String, Short]): Unit = { + private def updateDefaultMinVersionLevelsInAllBrokers(newMinVersionLevels: Map[String, Short]): Unit = { servers.foreach(s => { s.brokerFeatures.setDefaultMinVersionLevels(newMinVersionLevels) }) @@ -116,16 +115,20 @@ class UpdateFeaturesTest extends BaseRequestTest { assertEquals(expectedMetadata, featureMetadata) } - private def checkException[ExceptionType <: Throwable]( - result: UpdateFeaturesResult, - exceptionMsgPattern: Regex - )(implicit tag: ClassTag[ExceptionType]): Unit = { - val exception = intercept[ExecutionException] { - result.result().get() + 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(cause.getMessage, exceptionMsgPattern.findFirstIn(cause.getMessage).isDefined) } - assertNotNull(exception.getCause) - assertEquals(exception.getCause.getClass, tag.runtimeClass) - assertTrue(exceptionMsgPattern.findFirstIn(exception.getCause.getMessage).isDefined) + } /** @@ -136,22 +139,20 @@ class UpdateFeaturesTest extends BaseRequestTest { * updateFeatures request to the server * @param exceptionMsgPattern a pattern for the expected exception message */ - private def testWithInvalidFeatureUpdate[ExceptionType <: Throwable]( - invalidUpdate: FeatureUpdate, - exceptionMsgPattern: Regex - )(implicit tag: ClassTag[ExceptionType]): Unit = { + private def testWithInvalidFeatureUpdate[ExceptionType <: Throwable](feature: String, + invalidUpdate: FeatureUpdate, + exceptionMsgPattern: Regex) + (implicit tag: ClassTag[ExceptionType]): Unit = { TestUtils.waitUntilControllerElected(zkClient) - updateSupportedFeatures(defaultSupportedFeatures()) + updateSupportedFeaturesInAllBrokers(defaultSupportedFeatures()) val versionBefore = updateFeatureZNode(defaultFinalizedFeatures()) - val adminClient = createAdminClient() val nodeBefore = getFeatureZNode() - val result = adminClient.updateFeatures( - new HashSet[FeatureUpdate](Collections.singletonList(invalidUpdate)), new UpdateFeaturesOptions()) + val result = adminClient.updateFeatures(Utils.mkMap(Utils.mkEntry(feature, invalidUpdate)), new UpdateFeaturesOptions()) - checkException[ExceptionType](result, exceptionMsgPattern) + checkException[ExceptionType](result, Map(feature -> exceptionMsgPattern)) checkFeatures( adminClient, nodeBefore, @@ -162,58 +163,89 @@ class UpdateFeaturesTest extends BaseRequestTest { def testShouldFailRequestIfNotController(): Unit = { TestUtils.waitUntilControllerElected(zkClient) - updateSupportedFeatures(defaultSupportedFeatures()) + updateSupportedFeaturesInAllBrokers(defaultSupportedFeatures()) val versionBefore = updateFeatureZNode(defaultFinalizedFeatures()) val nodeBefore = getFeatureZNode() - val requestData = FeatureUpdate.createRequest( - new util.HashSet[FeatureUpdate]( - Collections.singletonList(new FeatureUpdate("feature_1", - defaultSupportedFeatures().get("feature_1").max(), - false)))) + val updates = new FeatureUpdateKeyCollection() + val update = new UpdateFeaturesRequestData.FeatureUpdateKey(); + update.setFeature("feature_1"); + update.setMaxVersionLevel(defaultSupportedFeatures().get("feature_1").max()) + update.setAllowDowngrade(false) + updates.add(update) val response = connectAndReceive[UpdateFeaturesResponse]( - new UpdateFeaturesRequest.Builder(requestData).build(), notControllerSocketServer) - - assertEquals(Errors.NOT_CONTROLLER, response.error()) + new UpdateFeaturesRequest.Builder(new UpdateFeaturesRequestData().setFeatureUpdates(updates)).build(), + notControllerSocketServer) + + assertEquals(1, response.data.results.size) + val result = response.data.results.asScala.head + assertEquals("feature_1", result.feature) + assertEquals(Errors.NOT_CONTROLLER, Errors.forCode(result.errorCode)) + assertNotNull(result.errorMessage) + assertFalse(result.errorMessage.isEmpty) checkFeatures( createAdminClient(), nodeBefore, new FeatureMetadata(defaultFinalizedFeatures(), versionBefore, defaultSupportedFeatures())) } + @Test + def testShouldFailRequestForEmptyUpdates(): Unit = { + val nullMap: util.Map[String, FeatureUpdate] = null + val emptyMap: util.Map[String, FeatureUpdate] = Utils.mkMap() + Set(nullMap, emptyMap).foreach { updates => + val client = createAdminClient() + val exception = intercept[IllegalArgumentException] { + client.updateFeatures(updates, new UpdateFeaturesOptions()) + } + assertNotNull(exception) + assertEquals("Feature updates can not be null or empty.", exception.getMessage) + } + } + + @Test + def testShouldFailRequestForNullUpdateFeaturesOptions(): Unit = { + val client = createAdminClient() + val update = new FeatureUpdate(defaultSupportedFeatures().get("feature_1").max(), false) + val exception = intercept[NullPointerException] { + client.updateFeatures(Utils.mkMap(Utils.mkEntry("feature_1", update)), null) + } + assertNotNull(exception) + assertEquals("UpdateFeaturesOptions can not be null", exception.getMessage) + } + @Test def testShouldFailRequestForInvalidFeatureName(): Unit = { - testWithInvalidFeatureUpdate[InvalidRequestException](new FeatureUpdate( - "", - defaultSupportedFeatures().get("feature_1").max(), - false), - ".*empty feature name.*".r) + val client = createAdminClient() + val update = new FeatureUpdate(defaultSupportedFeatures().get("feature_1").max(), false) + val exception = intercept[IllegalArgumentException] { + client.updateFeatures(Utils.mkMap(Utils.mkEntry("", update)), new UpdateFeaturesOptions()) + } + assertNotNull(exception) + assertTrue((".*Provided feature can not be null or empty.*"r).findFirstIn(exception.getMessage).isDefined) } @Test def testShouldFailRequestWhenDowngradeFlagIsNotSetDuringDowngrade(): Unit = { - testWithInvalidFeatureUpdate[InvalidRequestException](new FeatureUpdate( + testWithInvalidFeatureUpdate[InvalidRequestException]( "feature_1", - (defaultFinalizedFeatures().get("feature_1").max() - 1).asInstanceOf[Short], - false), + new FeatureUpdate((defaultFinalizedFeatures().get("feature_1").max() - 1).asInstanceOf[Short],false), ".*Can not downgrade finalized feature: 'feature_1'.*allowDowngrade.*".r) } @Test def testShouldFailRequestWhenDowngradeToHigherVersionLevelIsAttempted(): Unit = { testWithInvalidFeatureUpdate[InvalidRequestException]( - new FeatureUpdate( - "feature_1", - defaultSupportedFeatures().get("feature_1").max(), - true), + "feature_1", + new FeatureUpdate(defaultSupportedFeatures().get("feature_1").max(), true), ".*finalized feature: 'feature_1'.*allowDowngrade.* provided maxVersionLevel:3.*existing maxVersionLevel:2.*".r) } @Test def testShouldFailRequestInClientWhenDowngradeFlagIsNotSetDuringDeletion(): Unit = { assertThrows[IllegalArgumentException] { - new FeatureUpdate("feature_1", 0, false) + new FeatureUpdate(0, false) } } @@ -221,28 +253,34 @@ class UpdateFeaturesTest extends BaseRequestTest { def testShouldFailRequestInServerWhenDowngradeFlagIsNotSetDuringDeletion(): Unit = { TestUtils.waitUntilControllerElected(zkClient) - updateSupportedFeatures(defaultSupportedFeatures()) + updateSupportedFeaturesInAllBrokers(defaultSupportedFeatures()) val versionBefore = updateFeatureZNode(defaultFinalizedFeatures()) val adminClient = createAdminClient() val nodeBefore = getFeatureZNode() - val featureUpdates + val updates = new UpdateFeaturesRequestData.FeatureUpdateKeyCollection(); - val featureUpdate = new UpdateFeaturesRequestData.FeatureUpdateKey(); - featureUpdate.setName("feature_1") - featureUpdate.setMaxVersionLevel(0) - featureUpdate.setAllowDowngrade(false) - featureUpdates.add(featureUpdate); + val update = new UpdateFeaturesRequestData.FeatureUpdateKey(); + update.setFeature("feature_1") + update.setMaxVersionLevel(0) + update.setAllowDowngrade(false) + updates.add(update); val requestData = new UpdateFeaturesRequestData() - requestData.setFeatureUpdates(featureUpdates); + requestData.setFeatureUpdates(updates); val response = connectAndReceive[UpdateFeaturesResponse]( - new UpdateFeaturesRequest.Builder(requestData).build(), controllerSocketServer) - - assertEquals(Errors.INVALID_REQUEST, response.error) - val exceptionMsgPattern = ".*Can not delete feature: 'feature_1'.*allowDowngrade.*".r - assertTrue(exceptionMsgPattern.findFirstIn(response.data.errorMessage).isDefined) + new UpdateFeaturesRequest.Builder(new UpdateFeaturesRequestData().setFeatureUpdates(updates)).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 for feature: 'feature_1'.*allowDowngrade.*".r + assertTrue(exceptionMsgPattern.findFirstIn(result.errorMessage).isDefined) checkFeatures( adminClient, nodeBefore, @@ -252,15 +290,16 @@ class UpdateFeaturesTest extends BaseRequestTest { @Test def testShouldFailRequestDuringDeletionOfNonExistingFeature(): Unit = { testWithInvalidFeatureUpdate[InvalidRequestException]( - new FeatureUpdate("feature_non_existing", 0, true), + "feature_non_existing", + new FeatureUpdate(0, true), ".*Can not delete non-existing finalized feature: 'feature_non_existing'.*".r) } @Test def testShouldFailRequestWhenUpgradingToSameVersionLevel(): Unit = { testWithInvalidFeatureUpdate[InvalidRequestException]( - new FeatureUpdate( - "feature_1", defaultFinalizedFeatures().get("feature_1").max(), false), + "feature_1", + new FeatureUpdate(defaultFinalizedFeatures().get("feature_1").max(), false), ".*Can not upgrade a finalized feature: 'feature_1'.*to the same value.*".r) } @@ -268,23 +307,23 @@ class UpdateFeaturesTest extends BaseRequestTest { def testShouldFailRequestWhenDowngradingBelowMinVersionLevel(): Unit = { TestUtils.waitUntilControllerElected(zkClient) - updateSupportedFeatures(defaultSupportedFeatures()) + updateSupportedFeaturesInAllBrokers(defaultSupportedFeatures()) val minVersionLevel = 2.asInstanceOf[Short] - updateDefaultMinVersionLevels(Map[String, Short]("feature_1" -> minVersionLevel)) + updateDefaultMinVersionLevelsInAllBrokers(Map[String, Short]("feature_1" -> minVersionLevel)) val initialFinalizedFeatures = Features.finalizedFeatures( Utils.mkMap(Utils.mkEntry("feature_1", new FinalizedVersionRange(minVersionLevel, 2)))) val versionBefore = updateFeatureZNode(initialFinalizedFeatures) - val update = new FeatureUpdate( - "feature_1", (minVersionLevel - 1).asInstanceOf[Short], true) + val update = new FeatureUpdate((minVersionLevel - 1).asInstanceOf[Short], true) val adminClient = createAdminClient() val nodeBefore = getFeatureZNode() val result = adminClient.updateFeatures( - new HashSet[FeatureUpdate](Collections.singletonList(update)), new UpdateFeaturesOptions()) + Utils.mkMap(Utils.mkEntry("feature_1", update)), new UpdateFeaturesOptions()) checkException[InvalidRequestException]( - result, ".*Can not downgrade finalized feature: 'feature_1' to maxVersionLevel:1.*existing minVersionLevel:2.*".r) + result, + Map("feature_1" -> ".*Can not downgrade finalized feature: 'feature_1' to maxVersionLevel:1.*existing minVersionLevel:2.*".r)) checkFeatures( adminClient, nodeBefore, @@ -297,7 +336,7 @@ class UpdateFeaturesTest extends BaseRequestTest { val controller = servers.filter { server => server.kafkaController.isActive}.head val nonControllerServers = servers.filter { server => !server.kafkaController.isActive} - val unsupportedBrokers = Set[KafkaServer](nonControllerServers(0)) + val unsupportedBrokers = Set[KafkaServer](nonControllerServers.head) val supportedBrokers = Set[KafkaServer](nonControllerServers(1), controller) updateSupportedFeatures(defaultSupportedFeatures(), supportedBrokers) @@ -315,16 +354,14 @@ class UpdateFeaturesTest extends BaseRequestTest { val versionBefore = updateFeatureZNode(defaultFinalizedFeatures()) - val invalidUpdate = new FeatureUpdate( - "feature_1", defaultSupportedFeatures().get("feature_1").max(), false) + val invalidUpdate = new FeatureUpdate(defaultSupportedFeatures().get("feature_1").max(), false) val nodeBefore = getFeatureZNode() val adminClient = createAdminClient() val result = adminClient.updateFeatures( - new HashSet[FeatureUpdate](Collections.singletonList(invalidUpdate)), + Utils.mkMap(Utils.mkEntry("feature_1", invalidUpdate)), new UpdateFeaturesOptions()) - checkException[InvalidRequestException]( - result, ".*1 broker.*incompatible.*".r) + checkException[InvalidRequestException](result, Map("feature_1" -> ".*1 broker.*incompatible.*".r)) checkFeatures( adminClient, nodeBefore, @@ -332,15 +369,54 @@ class UpdateFeaturesTest extends BaseRequestTest { } @Test - def testSuccessFeatureUpgradeAndDowngrade(): Unit = { + def testSuccessfulFeatureUpgradeAndWithNoExistingFinalizedFeatures(): Unit = { TestUtils.waitUntilControllerElected(zkClient) - updateSupportedFeatures( + updateSupportedFeaturesInAllBrokers( Features.supportedFeatures( Utils.mkMap( Utils.mkEntry("feature_1", new SupportedVersionRange(1, 3)), Utils.mkEntry("feature_2", new SupportedVersionRange(2, 5))))) - updateDefaultMinVersionLevels(Map[String, Short]("feature_1" -> 1, "feature_2" -> 2)) + updateDefaultMinVersionLevelsInAllBrokers(Map[String, Short]("feature_1" -> 1, "feature_2" -> 2)) + 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 expected = new FeatureMetadata( + targetFinalizedFeatures, + versionBefore + 1, + Features.supportedFeatures( + Utils.mkMap( + Utils.mkEntry("feature_1", new SupportedVersionRange(1, 3)), + Utils.mkEntry("feature_2", new SupportedVersionRange(2, 5))))) + + 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), + expected) + } + + @Test + def testSuccessfulFeatureUpgradeAndDowngrade(): Unit = { + TestUtils.waitUntilControllerElected(zkClient) + + updateSupportedFeaturesInAllBrokers( + Features.supportedFeatures( + Utils.mkMap( + Utils.mkEntry("feature_1", new SupportedVersionRange(1, 3)), + Utils.mkEntry("feature_2", new SupportedVersionRange(2, 5))))) + updateDefaultMinVersionLevelsInAllBrokers(Map[String, Short]("feature_1" -> 1, "feature_2" -> 2)) val versionBefore = updateFeatureZNode( Features.finalizedFeatures( Utils.mkMap( @@ -351,10 +427,8 @@ class UpdateFeaturesTest extends BaseRequestTest { Utils.mkMap( Utils.mkEntry("feature_1", new FinalizedVersionRange(1, 3)), Utils.mkEntry("feature_2", new FinalizedVersionRange(2, 3)))) - val update1 = new FeatureUpdate( - "feature_1", targetFinalizedFeatures.get("feature_1").max(), false) - val update2 = new FeatureUpdate( - "feature_2", targetFinalizedFeatures.get("feature_2").max(), true) + val update1 = new FeatureUpdate(targetFinalizedFeatures.get("feature_1").max(), false) + val update2 = new FeatureUpdate(targetFinalizedFeatures.get("feature_2").max(), true) val expected = new FeatureMetadata( targetFinalizedFeatures, @@ -366,8 +440,9 @@ class UpdateFeaturesTest extends BaseRequestTest { val adminClient = createAdminClient() adminClient.updateFeatures( - new HashSet[FeatureUpdate](Arrays.asList(update1, update2)), - new UpdateFeaturesOptions()).result().get() + Utils.mkMap(Utils.mkEntry("feature_1", update1), Utils.mkEntry("feature_2", update2)), + new UpdateFeaturesOptions() + ).all().get() checkFeatures( adminClient, @@ -376,15 +451,15 @@ class UpdateFeaturesTest extends BaseRequestTest { } @Test - def testShouldFailRequestDuringValidFeatureUpgradeAndInvalidDowngrade(): Unit = { + def testPartialSuccessDuringValidFeatureUpgradeAndInvalidDowngrade(): Unit = { TestUtils.waitUntilControllerElected(zkClient) val initialSupportedFeatures = Features.supportedFeatures( Utils.mkMap( Utils.mkEntry("feature_1", new SupportedVersionRange(1, 3)), Utils.mkEntry("feature_2", new SupportedVersionRange(2, 5)))) - updateSupportedFeatures(initialSupportedFeatures) - updateDefaultMinVersionLevels(Map[String, Short]("feature_1" -> 1, "feature_2" -> 2)) + updateSupportedFeaturesInAllBrokers(initialSupportedFeatures) + updateDefaultMinVersionLevelsInAllBrokers(Map[String, Short]("feature_1" -> 1, "feature_2" -> 2)) val initialFinalizedFeatures = Features.finalizedFeatures( Utils.mkMap( Utils.mkEntry("feature_1", new FinalizedVersionRange(1, 2)), @@ -395,32 +470,36 @@ class UpdateFeaturesTest extends BaseRequestTest { Utils.mkMap( Utils.mkEntry("feature_1", new FinalizedVersionRange(1, 3)), Utils.mkEntry("feature_2", new FinalizedVersionRange(2, 3)))) - val validUpdate = new FeatureUpdate( - "feature_1", targetFinalizedFeatures.get("feature_1").max(), false) - val invalidUpdate = new FeatureUpdate( - "feature_2", targetFinalizedFeatures.get("feature_2").max(), false) + val validUpdate = new FeatureUpdate(targetFinalizedFeatures.get("feature_1").max(), false) + val invalidUpdate = new FeatureUpdate(targetFinalizedFeatures.get("feature_2").max(), false) - val nodeBefore = getFeatureZNode() val adminClient = createAdminClient() val result = adminClient.updateFeatures( - new HashSet[FeatureUpdate](Arrays.asList(validUpdate, invalidUpdate)), + 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, ".*Can not downgrade finalized feature: 'feature_2'.*allowDowngrade.*".r) + result, Map("feature_2" -> ".*Can not downgrade finalized feature: 'feature_2'.*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, - nodeBefore, - new FeatureMetadata(initialFinalizedFeatures, versionBefore, initialSupportedFeatures)) + FeatureZNode(FeatureZNodeStatus.Enabled, expectedFeatures), + new FeatureMetadata(expectedFeatures, versionBefore + 1, initialSupportedFeatures)) } @Test - def testShouldFailRequestDuringInvalidFeatureUpgradeAndValidDowngrade(): Unit = { + def testPartialSuccessDuringInvalidFeatureUpgradeAndValidDowngrade(): Unit = { TestUtils.waitUntilControllerElected(zkClient) val controller = servers.filter { server => server.kafkaController.isActive}.head val nonControllerServers = servers.filter { server => !server.kafkaController.isActive} - val unsupportedBrokers = Set[KafkaServer](nonControllerServers(0)) + val unsupportedBrokers = Set[KafkaServer](nonControllerServers.head) val supportedBrokers = Set[KafkaServer](nonControllerServers(1), controller) val initialSupportedFeatures = Features.supportedFeatures( @@ -435,7 +514,7 @@ class UpdateFeaturesTest extends BaseRequestTest { Utils.mkEntry("feature_2", new SupportedVersionRange(2, 5)))) updateSupportedFeatures(badSupportedFeatures, unsupportedBrokers) - updateDefaultMinVersionLevels(Map[String, Short]("feature_1" -> 1, "feature_2" -> 2)) + updateDefaultMinVersionLevelsInAllBrokers(Map[String, Short]("feature_1" -> 1, "feature_2" -> 2)) val initialFinalizedFeatures = Features.finalizedFeatures( Utils.mkMap( @@ -447,21 +526,25 @@ class UpdateFeaturesTest extends BaseRequestTest { Utils.mkMap( Utils.mkEntry("feature_1", new FinalizedVersionRange(1, 3)), Utils.mkEntry("feature_2", new FinalizedVersionRange(2, 3)))) - val invalidUpdate = new FeatureUpdate( - "feature_1", targetFinalizedFeatures.get("feature_1").max(), false) - val validUpdate = new FeatureUpdate( - "feature_2", targetFinalizedFeatures.get("feature_2").max(), true) + val invalidUpdate = new FeatureUpdate(targetFinalizedFeatures.get("feature_1").max(), false) + val validUpdate = new FeatureUpdate(targetFinalizedFeatures.get("feature_2").max(), true) - val nodeBefore = getFeatureZNode() val adminClient = createAdminClient() val result = adminClient.updateFeatures( - new HashSet[FeatureUpdate](Arrays.asList(invalidUpdate, validUpdate)), + Utils.mkMap(Utils.mkEntry("feature_1", invalidUpdate), Utils.mkEntry("feature_2", validUpdate)), new UpdateFeaturesOptions()) - checkException[InvalidRequestException](result, ".*1 broker.*incompatible.*".r) + // 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" -> ".*1 broker.*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, - nodeBefore, - new FeatureMetadata(initialFinalizedFeatures, versionBefore, initialSupportedFeatures)) + FeatureZNode(FeatureZNodeStatus.Enabled, expectedFeatures), + new FeatureMetadata(expectedFeatures, versionBefore + 1, initialSupportedFeatures)) } } From 7620c8803aeaebaeedc6a3ebbb4e94594397cbea Mon Sep 17 00:00:00 2001 From: Kowshik Prakasam Date: Tue, 28 Jul 2020 02:42:47 -0700 Subject: [PATCH 11/41] Minor: Remove newline --- core/src/main/scala/kafka/server/KafkaConfig.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index da0fec82d50e3..0e31d6e55d2a7 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -266,7 +266,6 @@ object Defaults { val PasswordEncoderIterations = 4096 } - object KafkaConfig { private val LogConfigPrefix = "log." From 89372badb529d88219ff5035ebb590a9efcae70a Mon Sep 17 00:00:00 2001 From: Kowshik Prakasam Date: Tue, 28 Jul 2020 02:57:27 -0700 Subject: [PATCH 12/41] Minor: cosmetics --- .../kafka/controller/KafkaController.scala | 67 +++++++++---------- 1 file changed, 31 insertions(+), 36 deletions(-) diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 4a1f76761319f..16a47dacb2d2d 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -1855,7 +1855,8 @@ class KafkaController(val config: KafkaConfig, /** * Returns the new FinalizedVersionRange for the feature, if there are no feature - * incompatibilities with all known brokers. Otherwise returns a suitable error. + * incompatibilities seen with all known brokers for the provided feature update. + * Otherwise returns a suitable error. * * @param update the feature update to be processed (this can not be meant to delete the feature) * @@ -1881,7 +1882,7 @@ class KafkaController(val config: KafkaConfig, } else { Right( new ApiError(Errors.INVALID_REQUEST, - s"Could not apply finalized feature update because $numIncompatibleBrokers" + + s"Could not apply finalized feature update because $numIncompatibleBrokers" + " brokers were found to have incompatible features.")) } } @@ -1889,7 +1890,7 @@ class KafkaController(val config: KafkaConfig, /** * Validate and process a finalized feature update. * - * If processed successfully, then, the return value contains: + * If the processing is successful, 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. * @@ -1918,55 +1919,49 @@ class KafkaController(val config: KafkaConfig, if (UpdateFeaturesRequest.isDeleteRequest(update)) { if (cacheEntry == null) { // Disallow deletion of a non-existing finalized feature. - Right( - new ApiError(Errors.INVALID_REQUEST, - s"Can not delete non-existing finalized feature: '${update.feature}'")) + Right(new ApiError(Errors.INVALID_REQUEST, + s"Can not delete non-existing finalized feature: '${update.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 for feature: '${update.feature}' without setting the" + - s" allowDowngrade flag to true in the request.")) + Right(new ApiError(Errors.INVALID_REQUEST, + s"Can not provide maxVersionLevel: ${update.maxVersionLevel} less" + + s" than 1 for feature: '${update.feature}' without setting the" + + " allowDowngrade flag to true in the request.")) } else { if (cacheEntry == null) { newVersionRangeOrError(update) } else { if (update.maxVersionLevel == cacheEntry.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: '${update.feature}' from existing" + - s" maxVersionLevel:${cacheEntry.max} to the same value.")) + Right(new ApiError(Errors.INVALID_REQUEST, + s"Can not ${if (update.allowDowngrade) "downgrade" else "upgrade"}" + + s" a finalized feature: '${update.feature}' from existing" + + s" maxVersionLevel:${cacheEntry.max} to the same value.")) } else if (update.maxVersionLevel < cacheEntry.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: '${update.feature}' from" + - s" existing maxVersionLevel:${cacheEntry.max} to provided" + - s" maxVersionLevel:${update.maxVersionLevel} without setting the" + - " allowDowngrade flag in the request.")) + Right(new ApiError(Errors.INVALID_REQUEST, + s"Can not downgrade finalized feature: '${update.feature}' from" + + s" existing maxVersionLevel:${cacheEntry.max} to provided" + + s" maxVersionLevel:${update.maxVersionLevel} without setting the" + + " allowDowngrade flag in the request.")) } else if (update.allowDowngrade && update.maxVersionLevel > cacheEntry.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 finalized feature: '${update.feature}' has the allowDowngrade" + - s" flag set in the request, the provided" + - s" maxVersionLevel:${update.maxVersionLevel} can not be greater than" + - s" existing maxVersionLevel:${cacheEntry.max}.")) + Right(new ApiError(Errors.INVALID_REQUEST, + s"When finalized feature: '${update.feature}' has the allowDowngrade" + + " flag set in the request, the provided" + + s" maxVersionLevel:${update.maxVersionLevel} can not be greater than" + + s" existing maxVersionLevel:${cacheEntry.max}.")) } else if (update.maxVersionLevel() < cacheEntry.min()) { // Disallow downgrade of a finalized feature below the existing finalized // minVersionLevel. - Right( - new ApiError(Errors.INVALID_REQUEST, - s"Can not downgrade finalized feature: '${update.feature}' to" + - s" maxVersionLevel:${update.maxVersionLevel} because it's lower than" + - s" the existing minVersionLevel:${cacheEntry.min}.")) + Right(new ApiError(Errors.INVALID_REQUEST, + s"Can not downgrade finalized feature: '${update.feature}' to" + + s" maxVersionLevel:${update.maxVersionLevel} because it's lower than" + + s" the existing minVersionLevel:${cacheEntry.min}.")) } else { newVersionRangeOrError(update) } @@ -1999,9 +1994,9 @@ class KafkaController(val config: KafkaConfig, // Map of feature to error. var errors = scala.collection.mutable.Map[String, ApiError]() - // Process each FeatureUpdate, insert an entry into errors and mutate targetFeatures suitably. - // If a FeatureUpdate is found to be valid, then the corresponding entry in errors contains - // Errors.NONE. Otherwise the entry contains the appropriate error. + // Process each FeatureUpdate. + // When this is done, if a FeatureUpdate is found to be valid, then the corresponding entry in + // errors would contain Errors.NONE. Otherwise the entry would contain the appropriate error. updates.asScala.iterator.foreach { update => processFeatureUpdate(update) match { case Left(newVersionRangeOrNone) => From 78bfc4c1bdcfb25d9682f9afec36232e2c3ad61f Mon Sep 17 00:00:00 2001 From: Kowshik Prakasam Date: Tue, 28 Jul 2020 11:29:45 -0700 Subject: [PATCH 13/41] Minor: Remove unused imports to fix checkstyle issues --- .../org/apache/kafka/clients/admin/DescribeFeaturesOptions.java | 1 - .../apache/kafka/common/requests/UpdateFeaturesResponse.java | 2 -- 2 files changed, 3 deletions(-) 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 index d7043af43c5d8..67ec78b1d1bad 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/DescribeFeaturesOptions.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/DescribeFeaturesOptions.java @@ -16,7 +16,6 @@ */ package org.apache.kafka.clients.admin; -import java.util.Set; import org.apache.kafka.common.annotation.InterfaceStability; /** 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 index ef98ebad5f423..68a8bd1e2b48e 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesResponse.java @@ -17,10 +17,8 @@ package org.apache.kafka.common.requests; import java.nio.ByteBuffer; -import java.util.HashMap; import java.util.Map; import java.util.stream.Collectors; -import org.apache.kafka.common.config.ConfigResource; import org.apache.kafka.common.message.UpdateFeaturesResponseData; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; From c3201a10d10fb0916b4dd2fc176f0eb129d0b75e Mon Sep 17 00:00:00 2001 From: Kowshik Prakasam Date: Tue, 28 Jul 2020 16:09:12 -0700 Subject: [PATCH 14/41] Minor cosmetics to fix checkstyle issue --- .../org/apache/kafka/clients/admin/KafkaAdminClientTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 f7cdb7c0d278b..e3bf01a09785e 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 @@ -496,7 +496,7 @@ private static UpdateFeaturesResponse prepareUpdateFeaturesResponse(Map Date: Tue, 28 Jul 2020 18:43:18 -0700 Subject: [PATCH 15/41] Minor cosmetics --- .../org/apache/kafka/clients/admin/Admin.java | 5 ++-- .../admin/DescribeFeaturesOptions.java | 2 +- .../clients/admin/DescribeFeaturesResult.java | 5 ++++ .../kafka/clients/admin/FeatureMetadata.java | 3 +-- .../kafka/clients/admin/FeatureUpdate.java | 27 ++++++++++++++++++- .../kafka/clients/admin/KafkaAdminClient.java | 10 +++---- .../clients/admin/UpdateFeaturesOptions.java | 6 ++--- .../clients/admin/UpdateFeaturesResult.java | 5 ++++ .../kafka/controller/KafkaController.scala | 11 ++++---- 9 files changed, 55 insertions(+), 19 deletions(-) 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 5a0751d4a292f..4bd048bef858f 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 @@ -1330,11 +1330,12 @@ AlterUserScramCredentialsResult alterUserScramCredentials(List - * The API takes in a map of finalized feature name to {@link FeatureUpdate} that need to be + * The API takes in a map of finalized feature name 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 feature. + * 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 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 index 67ec78b1d1bad..0cedd43fa4fe0 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/DescribeFeaturesOptions.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/DescribeFeaturesOptions.java @@ -21,7 +21,7 @@ /** * Options for {@link AdminClient#describeFeatures(DescribeFeaturesOptions)} * - * The API of this class is evolving. See {@link AdminClient} for details. + * The API of this class is evolving. See {@link Admin} for details. */ @InterfaceStability.Evolving public class DescribeFeaturesOptions extends AbstractOptions { 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 index f16f904ce16e3..9245d9b86b33a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/DescribeFeaturesResult.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/DescribeFeaturesResult.java @@ -18,6 +18,11 @@ 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; 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 index 8cdfcd75323f9..a75a3a0ea1bd3 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/FeatureMetadata.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/FeatureMetadata.java @@ -37,8 +37,7 @@ public class FeatureMetadata { public FeatureMetadata( final Features finalizedFeatures, final int finalizedFeaturesEpoch, - final Features supportedFeatures - ) { + final Features supportedFeatures) { Objects.requireNonNull(finalizedFeatures, "Provided finalizedFeatures can not be null."); Objects.requireNonNull(supportedFeatures, "Provided supportedFeatures can not be null."); this.finalizedFeatures = finalizedFeatures; 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 index 749522b9f9723..d383f622fd69c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/FeatureUpdate.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/FeatureUpdate.java @@ -17,6 +17,7 @@ package org.apache.kafka.clients.admin; import java.util.Map; +import java.util.Objects; /** * Encapsulates details about an update to a finalized feature. This is particularly useful to @@ -38,7 +39,7 @@ public class FeatureUpdate { public FeatureUpdate(final short maxVersionLevel, final boolean allowDowngrade) { if (maxVersionLevel < 1 && !allowDowngrade) { throw new IllegalArgumentException(String.format( - "The allowDowngrade flag is not set when the provided maxVersionLevel:%d is < 1.", + "The allowDowngrade flag should be set when the provided maxVersionLevel:%d is < 1.", maxVersionLevel)); } this.maxVersionLevel = maxVersionLevel; @@ -52,4 +53,28 @@ public short 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); + } } \ No newline at end of file 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 849855a0eb91e..0a31bad156978 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 @@ -4394,8 +4394,8 @@ public UpdateFeaturesResult updateFeatures( final UpdateFeaturesRequestData.FeatureUpdateKeyCollection featureUpdatesRequestData = new UpdateFeaturesRequestData.FeatureUpdateKeyCollection(); for (Map.Entry entry : featureUpdates.entrySet()) { - String feature = entry.getKey(); - FeatureUpdate update = entry.getValue(); + 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."); } @@ -4426,7 +4426,7 @@ void handleResponse(AbstractResponse abstractResponse) { // Check for controller change. for (UpdatableFeatureResult result : response.data().results()) { - Errors error = Errors.forCode(result.errorCode()); + final Errors error = Errors.forCode(result.errorCode()); if (error == Errors.NOT_CONTROLLER) { handleNotControllerError(error); throw error.exception(); @@ -4434,11 +4434,11 @@ void handleResponse(AbstractResponse abstractResponse) { } for (UpdatableFeatureResult result : response.data().results()) { - KafkaFutureImpl future = updateFutures.get(result.feature()); + final KafkaFutureImpl future = updateFutures.get(result.feature()); if (future == null) { log.warn("Server response mentioned unknown feature {}", result.feature()); } else { - Errors error = Errors.forCode(result.errorCode()); + final Errors error = Errors.forCode(result.errorCode()); if (error == Errors.NONE) { future.complete(null); } else { 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 index e936f09ca1955..7a9f2141b2ab1 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/UpdateFeaturesOptions.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/UpdateFeaturesOptions.java @@ -16,13 +16,13 @@ */ package org.apache.kafka.clients.admin; -import java.util.Set; +import java.util.Map; import org.apache.kafka.common.annotation.InterfaceStability; /** - * Options for {@link AdminClient#updateFeatures(Set, UpdateFeaturesOptions)} + * Options for {@link AdminClient#updateFeatures(Map, UpdateFeaturesOptions)}. * - * The API of this class is evolving. See {@link AdminClient} for details. + * 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 index 3c4e52f1ba82b..e4beacad6eb2a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/UpdateFeaturesResult.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/UpdateFeaturesResult.java @@ -19,6 +19,11 @@ 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; diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 16a47dacb2d2d..b020f89888cd1 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -393,7 +393,7 @@ class KafkaController(val config: KafkaConfig, + s" is absent in default finalized $defaultFinalizedFeatures") (featureName, existingVersionRange) } else if (brokerDefaultVersionRange.max() >= existingVersionRange.max() && - existingVersionRange.max() >= brokerDefaultVersionRange.min()) { + brokerDefaultVersionRange.min() <= existingVersionRange.max()) { // Through this change, we deprecate all version levels in the closed range: // [existingVersionRange.min(), brokerDefaultVersionRange.min() - 1] (featureName, new FinalizedVersionRange(brokerDefaultVersionRange.min(), existingVersionRange.max())) @@ -1862,7 +1862,7 @@ class KafkaController(val config: KafkaConfig, * * @return the new FinalizedVersionRange or error, as described above. */ - private def newFinalizedVersionRangeOrError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[FinalizedVersionRange, ApiError] = { + 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") } @@ -1906,7 +1906,8 @@ class KafkaController(val config: KafkaConfig, .getOrElse(Map[String, FinalizedVersionRange]()) def newVersionRangeOrError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[Option[FinalizedVersionRange], ApiError] = { - newFinalizedVersionRangeOrError(update).fold(versionRange => Left(Some(versionRange)), error => Right(error)) + newFinalizedVersionRangeOrIncompatibilityError(update) + .fold(versionRange => Left(Some(versionRange)), error => Right(error)) } if (update.feature.isEmpty) { @@ -1995,8 +1996,8 @@ class KafkaController(val config: KafkaConfig, var errors = scala.collection.mutable.Map[String, ApiError]() // Process each FeatureUpdate. - // When this is done, if a FeatureUpdate is found to be valid, then the corresponding entry in - // errors would contain Errors.NONE. Otherwise the entry would contain the appropriate error. + // If a FeatureUpdate is found to be valid, then the corresponding entry in errors would contain + // Errors.NONE. Otherwise the entry would contain the appropriate error. updates.asScala.iterator.foreach { update => processFeatureUpdate(update) match { case Left(newVersionRangeOrNone) => From 7afd81b242540f8445c0b399c86a1550d2b8e0d4 Mon Sep 17 00:00:00 2001 From: Kowshik Prakasam Date: Wed, 29 Jul 2020 14:47:59 -0700 Subject: [PATCH 16/41] Minor: improve code slightly in KafkaController --- .../kafka/controller/KafkaController.scala | 67 +++++++++---------- 1 file changed, 32 insertions(+), 35 deletions(-) diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index b020f89888cd1..481292e3f2b14 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -1888,7 +1888,8 @@ class KafkaController(val config: KafkaConfig, } /** - * Validate and process a finalized feature update. + * Validate and process a finalized feature update on an existing FinalizedVersionRange for the + * feature. * * If the processing is successful, then, the return value contains: * 1. the new FinalizedVersionRange for the feature, if the feature update was not meant to delete the feature. @@ -1896,15 +1897,14 @@ class KafkaController(val config: KafkaConfig, * * If the processing failed, then returned value contains a suitable ApiError. * - * @param update the feature update to be processed. + * @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 or error, as described above. + * @return the new FinalizedVersionRange or error, as described above. */ - private def processFeatureUpdate(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[Option[FinalizedVersionRange], ApiError] = { - val existingFeatures = featureCache.get - .map(featuresAndEpoch => featuresAndEpoch.features.features().asScala) - .getOrElse(Map[String, FinalizedVersionRange]()) - + private def processFeatureUpdate(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)) @@ -1914,11 +1914,9 @@ class KafkaController(val config: KafkaConfig, // Check that the feature name is not empty. Right(new ApiError(Errors.INVALID_REQUEST, "Feature name can not be empty.")) } else { - val cacheEntry = existingFeatures.get(update.feature).orNull - // We handle deletion requests separately from non-deletion requests. if (UpdateFeaturesRequest.isDeleteRequest(update)) { - if (cacheEntry == null) { + if (existingVersionRange.isEmpty) { // Disallow deletion of a non-existing finalized feature. Right(new ApiError(Errors.INVALID_REQUEST, s"Can not delete non-existing finalized feature: '${update.feature}'")) @@ -1932,41 +1930,39 @@ class KafkaController(val config: KafkaConfig, s" than 1 for feature: '${update.feature}' without setting the" + " allowDowngrade flag to true in the request.")) } else { - if (cacheEntry == null) { - newVersionRangeOrError(update) - } else { - if (update.maxVersionLevel == cacheEntry.max()) { + 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: '${update.feature}' from existing" + - s" maxVersionLevel:${cacheEntry.max} to the same value.")) - } else if (update.maxVersionLevel < cacheEntry.max && !update.allowDowngrade) { + s"Can not ${if (update.allowDowngrade) "downgrade" else "upgrade"}" + + s" a finalized feature: '${update.feature}' from existing" + + s" 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: '${update.feature}' from" + - s" existing maxVersionLevel:${cacheEntry.max} to provided" + - s" maxVersionLevel:${update.maxVersionLevel} without setting the" + - " allowDowngrade flag in the request.")) - } else if (update.allowDowngrade && update.maxVersionLevel > cacheEntry.max) { + s"Can not downgrade finalized feature: '${update.feature}' from" + + s" existing 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 finalized feature: '${update.feature}' has the allowDowngrade" + - " flag set in the request, the provided" + - s" maxVersionLevel:${update.maxVersionLevel} can not be greater than" + - s" existing maxVersionLevel:${cacheEntry.max}.")) - } else if (update.maxVersionLevel() < cacheEntry.min()) { + s"When finalized feature: '${update.feature}' has 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: '${update.feature}' to" + - s" maxVersionLevel:${update.maxVersionLevel} because it's lower than" + - s" the existing minVersionLevel:${cacheEntry.min}.")) + s"Can not downgrade finalized feature: '${update.feature}' to" + + s" maxVersionLevel:${update.maxVersionLevel} because it's lower than" + + s" the existing minVersionLevel:${existing.min}.")) } else { newVersionRangeOrError(update) } - } + ).getOrElse(newVersionRangeOrError(update)) } } } @@ -1999,13 +1995,14 @@ class KafkaController(val config: KafkaConfig, // If a FeatureUpdate is found to be valid, then the corresponding entry in errors would contain // Errors.NONE. Otherwise the entry would contain the appropriate error. updates.asScala.iterator.foreach { update => - processFeatureUpdate(update) match { + processFeatureUpdate(update, existingFeatures.get(update.feature())) match { case Left(newVersionRangeOrNone) => newVersionRangeOrNone .map(newVersionRange => targetFeatures += (update.feature() -> newVersionRange)) .getOrElse(targetFeatures -= update.feature()) errors += (update.feature() -> new ApiError(Errors.NONE)) - case Right(featureUpdateFailureReason) => errors += (update.feature() -> featureUpdateFailureReason) + case Right(featureUpdateFailureReason) => + errors += (update.feature() -> featureUpdateFailureReason) } } From cec250594bcdea3fd0e8d2ee0e4d33f0ae8fca21 Mon Sep 17 00:00:00 2001 From: Kowshik Prakasam Date: Mon, 3 Aug 2020 11:42:45 -0700 Subject: [PATCH 17/41] Address review comments from Boyang --- .../org/apache/kafka/clients/admin/Admin.java | 3 +- .../admin/DescribeFeaturesOptions.java | 11 +- .../kafka/clients/admin/FeatureMetadata.java | 11 +- .../kafka/clients/admin/KafkaAdminClient.java | 20 +-- .../common/requests/ApiVersionsResponse.java | 2 +- .../requests}/FeatureUpdate.java | 8 +- .../requests/UpdateFeaturesRequest.java | 25 ++++ .../requests/UpdateFeaturesResponse.java | 25 ++++ .../common/message/UpdateFeaturesRequest.json | 2 +- .../clients/admin/KafkaAdminClientTest.java | 83 +++++++----- .../kafka/clients/admin/MockAdminClient.java | 1 + .../kafka/controller/KafkaController.scala | 78 ++++++++---- .../scala/kafka/server/BrokerFeatures.scala | 32 +++-- .../kafka/server/FinalizedFeatureCache.scala | 4 +- .../main/scala/kafka/server/KafkaApis.scala | 15 +-- .../kafka/server/BrokerFeaturesTest.scala | 120 +++++++++++++++++- .../FinalizedFeatureChangeListenerTest.scala | 2 +- .../kafka/server/UpdateFeaturesTest.scala | 104 ++++++++------- 18 files changed, 376 insertions(+), 170 deletions(-) rename clients/src/main/java/org/apache/kafka/{clients/admin => common/requests}/FeatureUpdate.java (91%) 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 4bd048bef858f..ebaa09de2b1f7 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 @@ -40,6 +40,7 @@ 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.FeatureUpdate; import org.apache.kafka.common.requests.LeaveGroupResponse; /** @@ -1330,7 +1331,7 @@ AlterUserScramCredentialsResult alterUserScramCredentials(List - * The API takes in a map of finalized feature name to {@link FeatureUpdate} that needs to be + * 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 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 index 0cedd43fa4fe0..696976f1a43b2 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/DescribeFeaturesOptions.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/DescribeFeaturesOptions.java @@ -27,15 +27,16 @@ public class DescribeFeaturesOptions extends AbstractOptions { /** - * - True means the {@link Admin#describeFeatures(DescribeFeaturesOptions)} request can be - * issued only to the controller. - * - False means the {@link Admin#describeFeatures(DescribeFeaturesOptions)} request can be - * issued to any random broker. + * - True means the {@link Admin#describeFeatures(DescribeFeaturesOptions)} request must be + * processed only by the controller. + * - False means the {@link Admin#describeFeatures(DescribeFeaturesOptions)} request could be + * processed by any random broker. */ private boolean sendRequestToController = false; /** - * Sets a flag indicating that the describe features request should be issued to the controller. + * Sets a flag indicating that the describe features request must be processed only + * by the controller. */ public DescribeFeaturesOptions sendRequestToController(boolean sendRequestToController) { this.sendRequestToController = sendRequestToController; 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 index a75a3a0ea1bd3..78b1e366ade99 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/FeatureMetadata.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/FeatureMetadata.java @@ -34,10 +34,9 @@ public class FeatureMetadata { private final Features supportedFeatures; - public FeatureMetadata( - final Features finalizedFeatures, - final int finalizedFeaturesEpoch, - final Features supportedFeatures) { + public FeatureMetadata(final Features finalizedFeatures, + final int finalizedFeaturesEpoch, + final Features supportedFeatures) { Objects.requireNonNull(finalizedFeatures, "Provided finalizedFeatures can not be null."); Objects.requireNonNull(supportedFeatures, "Provided supportedFeatures can not be null."); this.finalizedFeatures = finalizedFeatures; @@ -96,9 +95,9 @@ public int hashCode() { @Override public String toString() { return String.format( - "FeatureMetadata{finalized:%s, finalizedFeaturesEpoch:%d, supported:%s}", + "FeatureMetadata{finalized:%s, finalizedFeaturesEpoch:%s, supported:%s}", finalizedFeatures, - finalizedFeaturesEpoch, + finalizedFeaturesEpoch.map(Object::toString).orElse(""), supportedFeatures); } } 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 0a31bad156978..98940ad5b9803 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 @@ -207,6 +207,7 @@ import org.apache.kafka.common.requests.ElectLeadersResponse; import org.apache.kafka.common.requests.ExpireDelegationTokenRequest; import org.apache.kafka.common.requests.ExpireDelegationTokenResponse; +import org.apache.kafka.common.requests.FeatureUpdate; import org.apache.kafka.common.requests.FindCoordinatorRequest; import org.apache.kafka.common.requests.FindCoordinatorRequest.CoordinatorType; import org.apache.kafka.common.requests.FindCoordinatorResponse; @@ -4390,26 +4391,11 @@ public UpdateFeaturesResult updateFeatures( } Objects.requireNonNull(options, "UpdateFeaturesOptions can not be null"); + final UpdateFeaturesRequestData request = UpdateFeaturesRequest.create(featureUpdates); final Map> updateFutures = new HashMap<>(); - 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."); - } - - updateFutures.put(feature, new KafkaFutureImpl<>()); - final UpdateFeaturesRequestData.FeatureUpdateKey requestItem = - new UpdateFeaturesRequestData.FeatureUpdateKey(); - requestItem.setFeature(feature); - requestItem.setMaxVersionLevel(update.maxVersionLevel()); - requestItem.setAllowDowngrade(update.allowDowngrade()); - featureUpdatesRequestData.add(requestItem); + updateFutures.put(entry.getKey(), new KafkaFutureImpl<>()); } - final UpdateFeaturesRequestData request = new UpdateFeaturesRequestData().setFeatureUpdates(featureUpdatesRequestData); - final long now = time.milliseconds(); final Call call = new Call("updateFeatures", calcDeadlineMs(now, options.timeoutMs()), new ControllerNodeProvider()) { 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 35dc00a9e7764..5d9517635da12 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 @@ -194,7 +194,7 @@ 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, diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/FeatureUpdate.java b/clients/src/main/java/org/apache/kafka/common/requests/FeatureUpdate.java similarity index 91% rename from clients/src/main/java/org/apache/kafka/clients/admin/FeatureUpdate.java rename to clients/src/main/java/org/apache/kafka/common/requests/FeatureUpdate.java index d383f622fd69c..5aea478e087cd 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/FeatureUpdate.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/FeatureUpdate.java @@ -14,14 +14,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.kafka.clients.admin; +package org.apache.kafka.common.requests; -import java.util.Map; import java.util.Objects; /** - * Encapsulates details about an update to a finalized feature. This is particularly useful to - * define each feature update in the {@link Admin#updateFeatures(Map, UpdateFeaturesOptions)} API. + * Encapsulates details about an update to a finalized feature. */ public class FeatureUpdate { private final short maxVersionLevel; @@ -77,4 +75,4 @@ public int hashCode() { public String toString() { return String.format("FeatureUpdate{maxVersionLevel:%d, allowDowngrade:%s}", maxVersionLevel, allowDowngrade); } -} \ No newline at end of file +} 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 index e779c723aa34e..ff425bf444105 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesRequest.java @@ -17,6 +17,9 @@ package org.apache.kafka.common.requests; import java.nio.ByteBuffer; +import java.util.HashMap; +import java.util.Map; +import org.apache.kafka.common.internals.KafkaFutureImpl; import org.apache.kafka.common.message.UpdateFeaturesRequestData.FeatureUpdateKey; import org.apache.kafka.common.message.UpdateFeaturesResponseData; import org.apache.kafka.common.message.UpdateFeaturesRequestData; @@ -90,4 +93,26 @@ public static UpdateFeaturesRequest parse(ByteBuffer buffer, short version) { public static boolean isDeleteRequest(UpdateFeaturesRequestData.FeatureUpdateKey update) { return update.maxVersionLevel() < 1 && update.allowDowngrade(); } + + public static UpdateFeaturesRequestData create(Map featureUpdates) { + final Map> updateFutures = new HashMap<>(); + 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."); + } + + updateFutures.put(feature, new KafkaFutureImpl<>()); + final UpdateFeaturesRequestData.FeatureUpdateKey requestItem = + new UpdateFeaturesRequestData.FeatureUpdateKey(); + requestItem.setFeature(feature); + requestItem.setMaxVersionLevel(update.maxVersionLevel()); + requestItem.setAllowDowngrade(update.allowDowngrade()); + featureUpdatesRequestData.add(requestItem); + } + return new UpdateFeaturesRequestData().setFeatureUpdates(featureUpdatesRequestData); + } } 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 index 68a8bd1e2b48e..c020622384b98 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesResponse.java @@ -17,9 +17,12 @@ package org.apache.kafka.common.requests; import java.nio.ByteBuffer; +import java.util.HashMap; 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; @@ -79,4 +82,26 @@ public UpdateFeaturesResponseData data() { public static UpdateFeaturesResponse parse(ByteBuffer buffer, short version) { return new UpdateFeaturesResponse(ApiKeys.UPDATE_FEATURES.parseResponse(version, buffer), version); } + + public static UpdateFeaturesResponse createWithFeatureUpdateApiErrors(Map updateErrors) { + final UpdatableFeatureResultCollection results = new UpdatableFeatureResultCollection(); + for (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); + } + return new UpdateFeaturesResponse(new UpdateFeaturesResponseData().setResults(results)); + } + + public static UpdateFeaturesResponse createWithFeatureUpdateErrors(Map updateErrors) { + final Map updateApiErrors = new HashMap<>(); + for (Map.Entry entry : updateErrors.entrySet()) { + updateApiErrors.put(entry.getKey(), new ApiError(entry.getValue())); + } + return createWithFeatureUpdateApiErrors(updateApiErrors); + } } diff --git a/clients/src/main/resources/common/message/UpdateFeaturesRequest.json b/clients/src/main/resources/common/message/UpdateFeaturesRequest.json index d2bf14b1e9d53..589903c00c64a 100644 --- a/clients/src/main/resources/common/message/UpdateFeaturesRequest.json +++ b/clients/src/main/resources/common/message/UpdateFeaturesRequest.json @@ -27,7 +27,7 @@ {"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."} + "about": "When set to true, the finalized feature version level is allowed to be downgraded/deleted. The downgraded 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/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java index e3bf01a09785e..f2ffa52413463 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 @@ -115,9 +115,6 @@ import org.apache.kafka.common.message.OffsetDeleteResponseData.OffsetDeleteResponsePartitionCollection; import org.apache.kafka.common.message.OffsetDeleteResponseData.OffsetDeleteResponseTopic; import org.apache.kafka.common.message.OffsetDeleteResponseData.OffsetDeleteResponseTopicCollection; -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.quota.ClientQuotaAlteration; @@ -148,6 +145,7 @@ import org.apache.kafka.common.requests.DescribeLogDirsResponse; import org.apache.kafka.common.requests.DescribeUserScramCredentialsResponse; import org.apache.kafka.common.requests.ElectLeadersResponse; +import org.apache.kafka.common.requests.FeatureUpdate; import org.apache.kafka.common.requests.FindCoordinatorResponse; import org.apache.kafka.common.requests.IncrementalAlterConfigsResponse; import org.apache.kafka.common.requests.JoinGroupRequest; @@ -486,19 +484,6 @@ private static DescribeGroupsResponseData prepareDescribeGroupsResponseData(Stri return data; } - private static UpdateFeaturesResponse prepareUpdateFeaturesResponse(Map featureUpdateErrors) { - final UpdatableFeatureResultCollection results = new UpdatableFeatureResultCollection(); - for (Map.Entry entry : featureUpdateErrors.entrySet()) { - UpdatableFeatureResult result = new UpdatableFeatureResult(); - result.setFeature(entry.getKey()); - Errors error = entry.getValue(); - result.setErrorCode(error.code()); - result.setErrorMessage(error.message()); - results.add(result); - } - return new UpdateFeaturesResponse(new UpdateFeaturesResponseData().setResults(results)); - } - private static FeatureMetadata defaultFeatureMetadata() { return new FeatureMetadata( Features.finalizedFeatures( @@ -3933,28 +3918,25 @@ public void testListOffsetsNonRetriableErrors() throws Exception { @Test public void testUpdateFeaturesDuringSuccess() throws Exception { - testUpdateFeatures( - makeTestFeatureUpdates(), - makeTestFeatureUpdateErrors(Errors.NONE)); + final Map updates = makeTestFeatureUpdates(); + testUpdateFeatures(updates, makeTestFeatureUpdateErrors(updates, Errors.NONE)); } @Test public void testUpdateFeaturesInvalidRequestError() throws Exception { - testUpdateFeatures( - makeTestFeatureUpdates(), - makeTestFeatureUpdateErrors(Errors.INVALID_REQUEST)); + final Map updates = makeTestFeatureUpdates(); + testUpdateFeatures(updates, makeTestFeatureUpdateErrors(updates, Errors.INVALID_REQUEST)); } @Test public void testUpdateFeaturesUpdateFailedError() throws Exception { - testUpdateFeatures( - makeTestFeatureUpdates(), - makeTestFeatureUpdateErrors(Errors.FEATURE_UPDATE_FAILED)); + final Map updates = makeTestFeatureUpdates(); + testUpdateFeatures(updates, makeTestFeatureUpdateErrors(updates, Errors.FEATURE_UPDATE_FAILED)); } @Test public void testUpdateFeaturesPartialSuccess() throws Exception { - final Map errors = makeTestFeatureUpdateErrors(Errors.NONE); + final Map errors = makeTestFeatureUpdateErrors(makeTestFeatureUpdates(), Errors.NONE); errors.put("test_feature_2", Errors.INVALID_REQUEST); testUpdateFeatures(makeTestFeatureUpdates(), errors); } @@ -3965,8 +3947,7 @@ private Map makeTestFeatureUpdates() { Utils.mkEntry("test_feature_2", new FeatureUpdate((short) 3, true))); } - private Map makeTestFeatureUpdateErrors(final Errors error) { - final Map updates = makeTestFeatureUpdates(); + private Map makeTestFeatureUpdateErrors(final Map updates, final Errors error) { final Map errors = new HashMap<>(); for (Map.Entry entry : updates.entrySet()) { errors.put(entry.getKey(), error); @@ -3979,7 +3960,7 @@ private void testUpdateFeatures(Map featureUpdates, try (final AdminClientUnitTestEnv env = mockClientEnv()) { env.kafkaClient().prepareResponse( body -> body instanceof UpdateFeaturesRequest, - prepareUpdateFeaturesResponse(featureUpdateErrors)); + UpdateFeaturesResponse.createWithFeatureUpdateErrors(featureUpdateErrors)); final Map> futures = env.adminClient().updateFeatures( featureUpdates, new UpdateFeaturesOptions().timeoutMs(10000)).values(); @@ -4002,7 +3983,7 @@ public void testUpdateFeaturesHandleNotControllerException() throws Exception { try (final AdminClientUnitTestEnv env = mockClientEnv()) { env.kafkaClient().prepareResponseFrom( request -> request instanceof UpdateFeaturesRequest, - prepareUpdateFeaturesResponse(Utils.mkMap( + UpdateFeaturesResponse.createWithFeatureUpdateErrors(Utils.mkMap( Utils.mkEntry("test_feature_1", Errors.NOT_CONTROLLER), Utils.mkEntry("test_feature_2", Errors.NOT_CONTROLLER))), env.cluster().nodeById(0)); @@ -4013,7 +3994,7 @@ public void testUpdateFeaturesHandleNotControllerException() throws Exception { Collections.emptyList())); env.kafkaClient().prepareResponseFrom( request -> request instanceof UpdateFeaturesRequest, - prepareUpdateFeaturesResponse(Utils.mkMap( + UpdateFeaturesResponse.createWithFeatureUpdateErrors(Utils.mkMap( Utils.mkEntry("test_feature_1", Errors.NONE), Utils.mkEntry("test_feature_2", Errors.NONE))), env.cluster().nodeById(controllerId)); @@ -4027,6 +4008,46 @@ public void testUpdateFeaturesHandleNotControllerException() throws Exception { } } + @Test + public void testUpdateFeaturesShouldFailRequestForEmptyUpdates() { + try (final AdminClientUnitTestEnv env = mockClientEnv()) { + assertThrows( + IllegalArgumentException.class, + () -> env.adminClient().updateFeatures(null, new UpdateFeaturesOptions())); + assertThrows( + IllegalArgumentException.class, + () -> env.adminClient().updateFeatures( + new HashMap<>(), new UpdateFeaturesOptions())); + } + } + + @Test + public void testUpdateFeaturesShouldFailRequestForNullUpdateFeaturesOptions() { + try (final AdminClientUnitTestEnv env = mockClientEnv()) { + assertThrows( + NullPointerException.class, + () -> env.adminClient().updateFeatures(makeTestFeatureUpdates(), null)); + } + } + + @Test + public void testUpdateFeaturesShouldFailRequestForInvalidFeatureName() { + try (final AdminClientUnitTestEnv env = mockClientEnv()) { + assertThrows( + IllegalArgumentException.class, + () -> env.adminClient().updateFeatures( + Utils.mkMap(Utils.mkEntry("", new FeatureUpdate((short) 2, false))), + new UpdateFeaturesOptions())); + } + } + + @Test + public void testUpdateFeaturesShouldFailRequestInClientWhenDowngradeFlagIsNotSetDuringDeletion() { + assertThrows( + IllegalArgumentException.class, + () -> new FeatureUpdate((short) 0, false)); + } + @Test public void testDescribeFeaturesSuccess() throws Exception { try (final AdminClientUnitTestEnv env = mockClientEnv()) { 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 3e9f605923bb7..a5bebd4c6b85e 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 @@ -51,6 +51,7 @@ import java.util.Map; import java.util.Optional; import java.util.Set; +import org.apache.kafka.common.requests.FeatureUpdate; public class MockAdminClient extends AdminClient { public static final String DEFAULT_CLUSTER_ID = "I4ZmrWqfT2e-upky_4fdPA"; diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 481292e3f2b14..52358eb0aa9f2 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -394,18 +394,39 @@ class KafkaController(val config: KafkaConfig, (featureName, existingVersionRange) } else if (brokerDefaultVersionRange.max() >= existingVersionRange.max() && brokerDefaultVersionRange.min() <= existingVersionRange.max()) { - // Through this change, we deprecate all version levels in the closed range: - // [existingVersionRange.min(), brokerDefaultVersionRange.min() - 1] + // Using the change below, we deprecate all version levels in the range: + // [existingVersionRange.min(), brokerDefaultVersionRange.min() - 1]. + // + // NOTE: if existingVersionRange.min() equals brokerDefaultVersionRange.min(), then + // we do not deprecate any version levels with this change. + // + // Examples: + // 1. brokerDefaultVersionRange = [4, 7] and existingVersionRange = [1, 5]. + // In this case, we deprecate all version levels in the range: [1, 3]. + // 2. brokerDefaultVersionRange = [4, 7] and existingVersionRange = [4, 5]. + // In this case, we do not deprecate any version level since + // brokerDefaultVersionRange.min() equals existingVersionRange.min(). (featureName, new FinalizedVersionRange(brokerDefaultVersionRange.min(), existingVersionRange.max())) } else { - // If the existing version levels fall completely outside the - // range of the default finalized version levels (i.e. no intersection), or, if the - // existing version levels are ineligible for a modification since they are - // incompatible with default finalized version levels, then we skip the update. - warn(s"Can not update minimum version level in finalized feature: $featureName," + // This is a serious error. We should never be reaching here, since we already + // verify once during KafkaServer startup that existing finalized feature versions in + // the FeatureZNode contained no incompatibilities. If we are here, it means that one of + // the following is true: + // 1. The existing version levels fall completely outside the range of the default + // finalized version levels (i.e. no intersection), or + // 2. The existing version levels are incompatible with default finalized version + // levels. + // + // Examples of invalid cases that can cause this exception to be triggered: + // 1. No intersection : brokerDefaultVersionRange = [4, 7] and existingVersionRange = [2, 3]. + // 2. No intersection : brokerDefaultVersionRange = [2, 3] and existingVersionRange = [4, 7]. + // 3. Incompatible versions: brokerDefaultVersionRange = [2, 3] and existingVersionRange = [1, 7]. + throw new IllegalStateException( + s"Can not update minimum version level in finalized feature: $featureName," + s" since the existing $existingVersionRange is not eligible for a change" - + s" based on the default $brokerDefaultVersionRange.") - (featureName, existingVersionRange) + + s" based on the default $brokerDefaultVersionRange. This should never happen" + + s" since feature version incompatibilities are already checked during" + + s" Kafka server startup.") } }.asJava) } @@ -1856,7 +1877,7 @@ 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 a suitable error. + * 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) * @@ -1866,24 +1887,29 @@ class KafkaController(val config: KafkaConfig, if (UpdateFeaturesRequest.isDeleteRequest(update)) { throw new IllegalArgumentException(s"Provided feature update can not be meant to delete the feature: $update") } + + val incompatibilityError = "Could not apply finalized feature update because" + + " brokers were found to have incompatible versions for the feature." + // NOTE: Below we set the finalized min version level to be the default minimum version // level. If the finalized feature already exists, then, this can cause deprecation of all // version levels in the closed range: // [existingVersionRange.min(), defaultMinVersionLevel - 1]. - val defaultMinVersionLevel = brokerFeatures.defaultMinVersionLevel(update.feature) - val newVersionRange = new FinalizedVersionRange(defaultMinVersionLevel, update.maxVersionLevel) - val numIncompatibleBrokers = controllerContext.liveOrShuttingDownBrokers.count(broker => { - val singleFinalizedFeature = - Features.finalizedFeatures(Utils.mkMap(Utils.mkEntry(update.feature, newVersionRange))) - BrokerFeatures.hasIncompatibleFeatures(broker.features, singleFinalizedFeature) - }) - if (numIncompatibleBrokers == 0) { - Left(newVersionRange) + if (brokerFeatures.supportedFeatures.get(update.feature()) == null) { + Right(new ApiError(Errors.INVALID_REQUEST, incompatibilityError)) } else { - Right( - new ApiError(Errors.INVALID_REQUEST, - s"Could not apply finalized feature update because $numIncompatibleBrokers" + - " brokers were found to have incompatible features.")) + val defaultMinVersionLevel = brokerFeatures.defaultMinVersionLevel(update.feature) + val newVersionRange = new FinalizedVersionRange(defaultMinVersionLevel, update.maxVersionLevel) + val numIncompatibleBrokers = controllerContext.liveOrShuttingDownBrokers.count(broker => { + val singleFinalizedFeature = + Features.finalizedFeatures(Utils.mkMap(Utils.mkEntry(update.feature, newVersionRange))) + BrokerFeatures.hasIncompatibleFeatures(broker.features, singleFinalizedFeature) + }) + if (numIncompatibleBrokers == 0) { + Left(newVersionRange) + } else { + Right(new ApiError(Errors.INVALID_REQUEST, incompatibilityError)) + } } } @@ -1952,7 +1978,7 @@ class KafkaController(val config: KafkaConfig, " 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) { + } else if (update.maxVersionLevel < existing.min) { // Disallow downgrade of a finalized feature below the existing finalized // minVersionLevel. Right(new ApiError(Errors.INVALID_REQUEST, @@ -2006,6 +2032,10 @@ class KafkaController(val config: KafkaConfig, } } + // 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. if (existingFeatures.equals(targetFeatures)) { callback(errors) } else { diff --git a/core/src/main/scala/kafka/server/BrokerFeatures.scala b/core/src/main/scala/kafka/server/BrokerFeatures.scala index 70861495cd368..677331829b594 100644 --- a/core/src/main/scala/kafka/server/BrokerFeatures.scala +++ b/core/src/main/scala/kafka/server/BrokerFeatures.scala @@ -55,15 +55,15 @@ class BrokerFeatures private (@volatile var supportedFeatures: Features[Supporte } /** - * Returns the default minimum version level for a specific feature. + * Returns the default minimum version level for a specific supported feature. * * @param feature the name of the feature * - * @return the default minimum version level for the feature if its defined. - * otherwise, returns 1. + * @return the default minimum version level for the supported feature if its defined. + * otherwise, returns the minimum version of the supported feature. */ def defaultMinVersionLevel(feature: String): Short = { - defaultFeatureMinVersionLevels.getOrElse(feature, 1) + defaultFeatureMinVersionLevels.getOrElse(feature, supportedFeatures.get(feature).min()) } // For testing only. @@ -92,8 +92,8 @@ class BrokerFeatures private (@volatile var supportedFeatures: Features[Supporte * 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. + * 2) Exists but the FinalizedVersionRange does not match with either the SupportedVersionRange + * of the supported feature, or the default minimum version level of the feature. * * @param finalized The finalized features against which incompatibilities need to be checked for. * @@ -101,7 +101,8 @@ class BrokerFeatures private (@volatile var supportedFeatures: Features[Supporte * is empty, it means there were no feature incompatibilities found. */ def incompatibleFeatures(finalized: Features[FinalizedVersionRange]): Features[FinalizedVersionRange] = { - BrokerFeatures.incompatibleFeatures(supportedFeatures, finalized, logIncompatibilities = true) + BrokerFeatures.incompatibleFeatures( + supportedFeatures, finalized, Some(defaultFeatureMinVersionLevels), logIncompatibilities = true) } } @@ -125,11 +126,20 @@ object BrokerFeatures extends Logging { */ def hasIncompatibleFeatures(supportedFeatures: Features[SupportedVersionRange], finalizedFeatures: Features[FinalizedVersionRange]): Boolean = { - !incompatibleFeatures(supportedFeatures, finalizedFeatures, false).empty + !incompatibleFeatures(supportedFeatures, finalizedFeatures, Option.empty, false).empty + } + + private def isIncompatibleDefaultMinVersionLevel(feature: String, + versionLevels: FinalizedVersionRange, + defaultFeatureMinVersionLevels: Option[Map[String, Short]]): Boolean = { + defaultFeatureMinVersionLevels.exists(defaults => + defaults.get(feature).exists(defaultMinVersionLevel => + defaultMinVersionLevel > versionLevels.max())) } private def incompatibleFeatures(supportedFeatures: Features[SupportedVersionRange], finalizedFeatures: Features[FinalizedVersionRange], + defaultFeatureMinVersionLevels: Option[Map[String, Short]], logIncompatibilities: Boolean): Features[FinalizedVersionRange] = { val incompatibleFeaturesInfo = finalizedFeatures.features.asScala.map { case (feature, versionLevels) => @@ -139,15 +149,19 @@ object BrokerFeatures extends Logging { } else if (versionLevels.isIncompatibleWith(supportedVersions)) { (feature, versionLevels, "{feature=%s, reason='%s is incompatible with %s'}".format( feature, versionLevels, supportedVersions)) + } else if (isIncompatibleDefaultMinVersionLevel(feature, versionLevels, defaultFeatureMinVersionLevels)) { + (feature, versionLevels, "{feature=%s, reason='%s is incompatible with default min_version_level: %d'}".format( + feature, versionLevels, defaultFeatureMinVersionLevels.get(feature))) } else { (feature, versionLevels, null) + } }.filter{ case(_, _, errorReason) => errorReason != null}.toList if (logIncompatibilities && incompatibleFeaturesInfo.nonEmpty) { warn( "Feature incompatibilities seen: " + incompatibleFeaturesInfo.map { - case(_, _, errorReason) => errorReason }) + 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 ddd7f9854a63b..c4e8dca34eba4 100644 --- a/core/src/main/scala/kafka/server/FinalizedFeatureCache.scala +++ b/core/src/main/scala/kafka/server/FinalizedFeatureCache.scala @@ -38,7 +38,9 @@ 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 */ diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index a27c7b603ae40..69fb17642c7c6 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -52,7 +52,7 @@ import org.apache.kafka.common.internals.Topic.{GROUP_METADATA_TOPIC_NAME, TRANS import org.apache.kafka.common.message.AlterConfigsResponseData.AlterConfigsResourceResponse import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic import org.apache.kafka.common.message.CreatePartitionsResponseData.CreatePartitionsTopicResult -import org.apache.kafka.common.message.{AddOffsetsToTxnResponseData, AlterConfigsResponseData, AlterPartitionReassignmentsResponseData, AlterReplicaLogDirsResponseData, CreateAclsResponseData, CreatePartitionsResponseData, CreateTopicsResponseData, DeleteAclsResponseData, DeleteGroupsResponseData, DeleteRecordsResponseData, DeleteTopicsResponseData, DescribeAclsResponseData, DescribeConfigsResponseData, DescribeGroupsResponseData, DescribeLogDirsResponseData, EndTxnResponseData, ExpireDelegationTokenResponseData, FindCoordinatorResponseData, HeartbeatResponseData, InitProducerIdResponseData, JoinGroupResponseData, LeaveGroupResponseData, ListGroupsResponseData, ListPartitionReassignmentsResponseData, MetadataResponseData, OffsetCommitRequestData, OffsetCommitResponseData, OffsetDeleteResponseData, RenewDelegationTokenResponseData, SaslAuthenticateResponseData, SaslHandshakeResponseData, StopReplicaResponseData, SyncGroupResponseData, UpdateFeaturesResponseData, UpdateMetadataResponseData} +import org.apache.kafka.common.message.{AddOffsetsToTxnResponseData, AlterConfigsResponseData, AlterPartitionReassignmentsResponseData, AlterReplicaLogDirsResponseData, CreateAclsResponseData, CreatePartitionsResponseData, CreateTopicsResponseData, DeleteAclsResponseData, DeleteGroupsResponseData, DeleteRecordsResponseData, DeleteTopicsResponseData, DescribeAclsResponseData, DescribeConfigsResponseData, DescribeGroupsResponseData, DescribeLogDirsResponseData, EndTxnResponseData, ExpireDelegationTokenResponseData, FindCoordinatorResponseData, HeartbeatResponseData, InitProducerIdResponseData, JoinGroupResponseData, LeaveGroupResponseData, ListGroupsResponseData, ListPartitionReassignmentsResponseData, MetadataResponseData, OffsetCommitRequestData, OffsetCommitResponseData, OffsetDeleteResponseData, RenewDelegationTokenResponseData, SaslAuthenticateResponseData, SaslHandshakeResponseData, StopReplicaResponseData, SyncGroupResponseData, UpdateMetadataResponseData} import org.apache.kafka.common.message.CreateTopicsResponseData.{CreatableTopicResult, CreatableTopicResultCollection} import org.apache.kafka.common.message.DeleteGroupsResponseData.{DeletableGroupResult, DeletableGroupResultCollection} import org.apache.kafka.common.message.AlterPartitionReassignmentsResponseData.{ReassignablePartitionResponse, ReassignableTopicResponse} @@ -90,7 +90,6 @@ import scala.collection.mutable.ArrayBuffer import scala.collection.{Map, Seq, Set, immutable, mutable} import scala.util.{Failure, Success, Try} import kafka.coordinator.group.GroupOverview -import org.apache.kafka.common.message.UpdateFeaturesResponseData.{UpdatableFeatureResult, UpdatableFeatureResultCollection} /** @@ -3120,17 +3119,7 @@ class KafkaApis(val requestChannel: RequestChannel, } def sendResponseCallback(updateErrors: Map[String, ApiError]): Unit = { - val results = new UpdatableFeatureResultCollection() - updateErrors.foreach { - case (feature, error) => - val result = new UpdatableFeatureResult() - .setFeature(feature) - .setErrorCode(error.error().code()) - .setErrorMessage(error.message()) - results.add(result) - } - val responseData = new UpdateFeaturesResponseData().setResults(results) - sendResponseExemptThrottle(request, new UpdateFeaturesResponse(responseData)) + sendResponseExemptThrottle(request, UpdateFeaturesResponse.createWithFeatureUpdateApiErrors(updateErrors.asJava)) } if (!authorize(request.context, ALTER, CLUSTER, CLUSTER_NAME)) { diff --git a/core/src/test/scala/unit/kafka/server/BrokerFeaturesTest.scala b/core/src/test/scala/unit/kafka/server/BrokerFeaturesTest.scala index 0f937b3b1462a..8deabf85c55a8 100644 --- a/core/src/test/scala/unit/kafka/server/BrokerFeaturesTest.scala +++ b/core/src/test/scala/unit/kafka/server/BrokerFeaturesTest.scala @@ -18,7 +18,7 @@ package kafka.server import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange} -import org.junit.Assert.{assertEquals, assertThrows, assertTrue} +import org.junit.Assert.{assertEquals, assertThrows, assertFalse, assertTrue} import org.junit.Test import scala.jdk.CollectionConverters._ @@ -31,17 +31,16 @@ class BrokerFeaturesTest { } @Test - def testIncompatibleFeatures(): Unit = { + def testIncompatibilitiesDueToAbsentFeature(): Unit = { val brokerFeatures = BrokerFeatures.createDefault() - val supportedFeatures = Map[String, SupportedVersionRange]( + val supportedFeatures = Features.supportedFeatures(Map[String, SupportedVersionRange]( "test_feature_1" -> new SupportedVersionRange(1, 4), - "test_feature_2" -> new SupportedVersionRange(1, 3)) - brokerFeatures.setSupportedFeatures(Features.supportedFeatures(supportedFeatures.asJava)) + "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), "test_feature_3" -> new FinalizedVersionRange(3, 4)) val features = compatibleFeatures++inCompatibleFeatures val finalizedFeatures = Features.finalizedFeatures(features.asJava) @@ -49,8 +48,78 @@ class BrokerFeaturesTest { 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 testIncompatibilitiesWithDefaultMinVersionLevel(): 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 defaultMinVersionLevels = Map[String, Short]( + "test_feature_1" -> 2, + "test_feature_2" -> 2) + brokerFeatures.setDefaultMinVersionLevels(defaultMinVersionLevels) + + val compatibleFeatures = Map[String, FinalizedVersionRange]( + "test_feature_1" -> new FinalizedVersionRange(2, 3)) + val inCompatibleFeatures = Map[String, FinalizedVersionRange]( + "test_feature_2" -> new FinalizedVersionRange(1, 1)) + val features = compatibleFeatures++inCompatibleFeatures + val finalizedFeatures = Features.finalizedFeatures(features.asJava) + + assertEquals( + Features.finalizedFeatures(inCompatibleFeatures.asJava), + brokerFeatures.incompatibleFeatures(finalizedFeatures)) + assertFalse(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 defaultMinVersionLevels = Map[String, Short]( + "test_feature_1" -> 2, + "test_feature_2" -> 2) + brokerFeatures.setDefaultMinVersionLevels(defaultMinVersionLevels) + + 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 testFeatureVersionAssertions(): Unit = { val brokerFeatures = BrokerFeatures.createDefault() @@ -81,4 +150,43 @@ class BrokerFeaturesTest { classOf[IllegalArgumentException], () => brokerFeatures.setDefaultMinVersionLevels(defaultMinVersionLevelsWithInvalidLargeValue)) } + + @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 defaultMinVersionLevels = Map[String, Short]( + "test_feature_1" -> 2, + "test_feature_2" -> 3) + brokerFeatures.setDefaultMinVersionLevels(defaultMinVersionLevels) + + val expectedFeatures = Map[String, FinalizedVersionRange]( + "test_feature_1" -> new FinalizedVersionRange(2, 4), + "test_feature_2" -> new FinalizedVersionRange(3, 3), + "test_feature_3" -> new FinalizedVersionRange(3, 7)) + assertEquals(Features.finalizedFeatures(expectedFeatures.asJava), brokerFeatures.getDefaultFinalizedFeatures) + } + + @Test + def testDefaultMinVersionLevel(): 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 defaultMinVersionLevels = Map[String, Short]("test_feature_1" -> 2) + brokerFeatures.setDefaultMinVersionLevels(defaultMinVersionLevels) + + assertEquals(2, brokerFeatures.defaultMinVersionLevel("test_feature_1")) + assertEquals(1, brokerFeatures.defaultMinVersionLevel("test_feature_2")) + assertThrows( + classOf[NullPointerException], + () => brokerFeatures.defaultMinVersionLevel("test_feature_3")) + } } diff --git a/core/src/test/scala/unit/kafka/server/FinalizedFeatureChangeListenerTest.scala b/core/src/test/scala/unit/kafka/server/FinalizedFeatureChangeListenerTest.scala index 4d853892903e2..324938e26eb14 100644 --- a/core/src/test/scala/unit/kafka/server/FinalizedFeatureChangeListenerTest.scala +++ b/core/src/test/scala/unit/kafka/server/FinalizedFeatureChangeListenerTest.scala @@ -103,7 +103,7 @@ class FinalizedFeatureChangeListenerTest extends ZooKeeperTestHarness { updateAndCheckCache( Features.finalizedFeatures( Map[String, FinalizedVersionRange]( - "feature_1" -> new FinalizedVersionRange(2, 4)).asJava)) + "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. diff --git a/core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala b/core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala index 038eee112f790..152bd0c6f9861 100644 --- a/core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala +++ b/core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala @@ -17,7 +17,6 @@ package kafka.server -import java.util import java.util.Properties import java.util.concurrent.ExecutionException @@ -25,18 +24,18 @@ 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, FeatureMetadata, FeatureUpdate, UpdateFeaturesOptions, UpdateFeaturesResult} +import org.apache.kafka.clients.admin.{Admin, DescribeFeaturesOptions, FeatureMetadata, 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.requests.{FeatureUpdate, 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.{assertThrows, intercept} +import org.scalatest.Assertions.intercept import scala.jdk.CollectionConverters._ import scala.reflect.ClassTag @@ -159,6 +158,9 @@ class UpdateFeaturesTest extends BaseRequestTest { new FeatureMetadata(defaultFinalizedFeatures(), versionBefore, defaultSupportedFeatures())) } + /** + * Tests that an UpdateFeatures request sent to a non-Controller node fails as expected. + */ @Test def testShouldFailRequestIfNotController(): Unit = { TestUtils.waitUntilControllerElected(zkClient) @@ -190,42 +192,10 @@ class UpdateFeaturesTest extends BaseRequestTest { new FeatureMetadata(defaultFinalizedFeatures(), versionBefore, defaultSupportedFeatures())) } - @Test - def testShouldFailRequestForEmptyUpdates(): Unit = { - val nullMap: util.Map[String, FeatureUpdate] = null - val emptyMap: util.Map[String, FeatureUpdate] = Utils.mkMap() - Set(nullMap, emptyMap).foreach { updates => - val client = createAdminClient() - val exception = intercept[IllegalArgumentException] { - client.updateFeatures(updates, new UpdateFeaturesOptions()) - } - assertNotNull(exception) - assertEquals("Feature updates can not be null or empty.", exception.getMessage) - } - } - - @Test - def testShouldFailRequestForNullUpdateFeaturesOptions(): Unit = { - val client = createAdminClient() - val update = new FeatureUpdate(defaultSupportedFeatures().get("feature_1").max(), false) - val exception = intercept[NullPointerException] { - client.updateFeatures(Utils.mkMap(Utils.mkEntry("feature_1", update)), null) - } - assertNotNull(exception) - assertEquals("UpdateFeaturesOptions can not be null", exception.getMessage) - } - - @Test - def testShouldFailRequestForInvalidFeatureName(): Unit = { - val client = createAdminClient() - val update = new FeatureUpdate(defaultSupportedFeatures().get("feature_1").max(), false) - val exception = intercept[IllegalArgumentException] { - client.updateFeatures(Utils.mkMap(Utils.mkEntry("", update)), new UpdateFeaturesOptions()) - } - assertNotNull(exception) - assertTrue((".*Provided feature can not be null or empty.*"r).findFirstIn(exception.getMessage).isDefined) - } - + /** + * 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 = { testWithInvalidFeatureUpdate[InvalidRequestException]( @@ -234,6 +204,10 @@ class UpdateFeaturesTest extends BaseRequestTest { ".*Can not downgrade finalized feature: 'feature_1'.*allowDowngrade.*".r) } + /** + * Tests that an UpdateFeatures request fails in the Controller, when, for a feature the downgrade + * is attempted to a max version level thats higher than the existing max version level. + */ @Test def testShouldFailRequestWhenDowngradeToHigherVersionLevelIsAttempted(): Unit = { testWithInvalidFeatureUpdate[InvalidRequestException]( @@ -242,13 +216,10 @@ class UpdateFeaturesTest extends BaseRequestTest { ".*finalized feature: 'feature_1'.*allowDowngrade.* provided maxVersionLevel:3.*existing maxVersionLevel:2.*".r) } - @Test - def testShouldFailRequestInClientWhenDowngradeFlagIsNotSetDuringDeletion(): Unit = { - assertThrows[IllegalArgumentException] { - new FeatureUpdate(0, false) - } - } - + /** + * 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) @@ -287,6 +258,10 @@ class UpdateFeaturesTest extends BaseRequestTest { new FeatureMetadata(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]( @@ -295,6 +270,10 @@ class UpdateFeaturesTest extends BaseRequestTest { ".*Can not delete non-existing finalized feature: 'feature_non_existing'.*".r) } + /** + * Tests that an UpdateFeatures request fails in the Controller, when, a feature version level + * upgrade is attempted to a version level thats the same as the existing max version level. + */ @Test def testShouldFailRequestWhenUpgradingToSameVersionLevel(): Unit = { testWithInvalidFeatureUpdate[InvalidRequestException]( @@ -303,6 +282,11 @@ class UpdateFeaturesTest extends BaseRequestTest { ".*Can not upgrade a finalized feature: 'feature_1'.*to the same value.*".r) } + /** + * Tests that an UpdateFeatures request fails in the Controller, when, a feature version level + * downgrade is attempted to a version level thats below the default min version level for the + * feature. + */ @Test def testShouldFailRequestWhenDowngradingBelowMinVersionLevel(): Unit = { TestUtils.waitUntilControllerElected(zkClient) @@ -330,6 +314,10 @@ class UpdateFeaturesTest extends BaseRequestTest { new FeatureMetadata(initialFinalizedFeatures, versionBefore, defaultSupportedFeatures())) } + /** + * Tests that an UpdateFeatures request fails in the Controller, when, a feature version level + * upgrade introduces a version incompatibility with existing supported features. + */ @Test def testShouldFailRequestDuringBrokerMaxVersionLevelIncompatibility(): Unit = { TestUtils.waitUntilControllerElected(zkClient) @@ -361,13 +349,17 @@ class UpdateFeaturesTest extends BaseRequestTest { Utils.mkMap(Utils.mkEntry("feature_1", invalidUpdate)), new UpdateFeaturesOptions()) - checkException[InvalidRequestException](result, Map("feature_1" -> ".*1 broker.*incompatible.*".r)) + checkException[InvalidRequestException](result, Map("feature_1" -> ".*brokers.*incompatible.*".r)) checkFeatures( adminClient, nodeBefore, new FeatureMetadata(defaultFinalizedFeatures(), versionBefore, defaultSupportedFeatures())) } + /** + * 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) @@ -407,6 +399,10 @@ class UpdateFeaturesTest extends BaseRequestTest { expected) } + /** + * 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) @@ -450,6 +446,11 @@ class UpdateFeaturesTest extends BaseRequestTest { expected) } + /** + * 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) @@ -493,6 +494,11 @@ class UpdateFeaturesTest extends BaseRequestTest { new FeatureMetadata(expectedFeatures, versionBefore + 1, initialSupportedFeatures)) } + /** + * 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) @@ -537,7 +543,7 @@ class UpdateFeaturesTest extends BaseRequestTest { // 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" -> ".*1 broker.*incompatible.*".r)) + checkException[InvalidRequestException](result, Map("feature_1" -> ".*brokers.*incompatible.*".r)) val expectedFeatures = Features.finalizedFeatures( Utils.mkMap( Utils.mkEntry("feature_1", initialFinalizedFeatures.get("feature_1")), From 3b4b370b6f30d760e9395ea3c3f4e3ad7970e0b3 Mon Sep 17 00:00:00 2001 From: Kowshik Prakasam Date: Wed, 5 Aug 2020 02:51:25 -0700 Subject: [PATCH 18/41] Minor cosmetics --- core/src/main/scala/kafka/controller/KafkaController.scala | 4 ---- 1 file changed, 4 deletions(-) diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 52358eb0aa9f2..e1e9bfa2816ba 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -1891,10 +1891,6 @@ class KafkaController(val config: KafkaConfig, val incompatibilityError = "Could not apply finalized feature update because" + " brokers were found to have incompatible versions for the feature." - // NOTE: Below we set the finalized min version level to be the default minimum version - // level. If the finalized feature already exists, then, this can cause deprecation of all - // version levels in the closed range: - // [existingVersionRange.min(), defaultMinVersionLevel - 1]. if (brokerFeatures.supportedFeatures.get(update.feature()) == null) { Right(new ApiError(Errors.INVALID_REQUEST, incompatibilityError)) } else { From 2a1dee2085a3beb7ec9fe811c88f7934518b51d9 Mon Sep 17 00:00:00 2001 From: Kowshik Prakasam Date: Thu, 6 Aug 2020 00:13:36 -0700 Subject: [PATCH 19/41] Minor cosmetics --- .../requests/UpdateFeaturesRequest.java | 4 +- .../kafka/controller/KafkaController.scala | 68 ++++++++++--------- .../scala/kafka/server/BrokerFeatures.scala | 4 +- .../kafka/server/UpdateFeaturesTest.scala | 16 ++--- 4 files changed, 47 insertions(+), 45 deletions(-) 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 index ff425bf444105..2fcea98eae06c 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesRequest.java @@ -65,9 +65,9 @@ public UpdateFeaturesRequest(Struct struct, short version) { @Override public AbstractResponse getErrorResponse(int throttleTimeMsIgnored, Throwable e) { final ApiError apiError = ApiError.fromThrowable(e); - UpdatableFeatureResultCollection results = new UpdatableFeatureResultCollection(); + final UpdatableFeatureResultCollection results = new UpdatableFeatureResultCollection(); for (FeatureUpdateKey update : this.data.featureUpdates().valuesSet()) { - UpdatableFeatureResult result = new UpdatableFeatureResult() + final UpdatableFeatureResult result = new UpdatableFeatureResult() .setFeature(update.feature()) .setErrorCode(apiError.error().code()) .setErrorMessage(apiError.message()); diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index e1e9bfa2816ba..cb1a9a0d64cff 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -398,20 +398,20 @@ class KafkaController(val config: KafkaConfig, // [existingVersionRange.min(), brokerDefaultVersionRange.min() - 1]. // // NOTE: if existingVersionRange.min() equals brokerDefaultVersionRange.min(), then - // we do not deprecate any version levels with this change. + // we do not deprecate any version levels (since there is none to be deprecated). // // Examples: // 1. brokerDefaultVersionRange = [4, 7] and existingVersionRange = [1, 5]. // In this case, we deprecate all version levels in the range: [1, 3]. // 2. brokerDefaultVersionRange = [4, 7] and existingVersionRange = [4, 5]. - // In this case, we do not deprecate any version level since + // In this case, we do not deprecate any version levels since // brokerDefaultVersionRange.min() equals existingVersionRange.min(). (featureName, new FinalizedVersionRange(brokerDefaultVersionRange.min(), existingVersionRange.max())) } else { // This is a serious error. We should never be reaching here, since we already // verify once during KafkaServer startup that existing finalized feature versions in - // the FeatureZNode contained no incompatibilities. If we are here, it means that one of - // the following is true: + // the FeatureZNode contained no incompatibilities. If we are here, it means that one + // of the following is true: // 1. The existing version levels fall completely outside the range of the default // finalized version levels (i.e. no intersection), or // 2. The existing version levels are incompatible with default finalized version @@ -1913,11 +1913,11 @@ class KafkaController(val config: KafkaConfig, * Validate and process a finalized feature update on an existing FinalizedVersionRange for the * feature. * - * If the processing is successful, then, the return value contains: + * If the processing 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 processing failed, then returned value contains a suitable ApiError. + * If the processing 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 @@ -1941,7 +1941,7 @@ class KafkaController(val config: KafkaConfig, if (existingVersionRange.isEmpty) { // Disallow deletion of a non-existing finalized feature. Right(new ApiError(Errors.INVALID_REQUEST, - s"Can not delete non-existing finalized feature: '${update.feature}'")) + "Can not delete non-existing finalized feature.")) } else { Left(Option.empty) } @@ -1949,38 +1949,35 @@ class KafkaController(val config: KafkaConfig, // 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 for feature: '${update.feature}' without setting the" + - " allowDowngrade flag to true in the request.")) + 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: '${update.feature}' from existing" + - s" maxVersionLevel:${existing.max} to the same value.")) + 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: '${update.feature}' from" + - s" existing maxVersionLevel:${existing.max} to provided" + - s" maxVersionLevel:${update.maxVersionLevel} without setting the" + - " allowDowngrade flag in the 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 finalized feature: '${update.feature}' has the allowDowngrade" + - " flag set in the request, the provided" + - s" maxVersionLevel:${update.maxVersionLevel} can not be greater than" + - s" existing maxVersionLevel:${existing.max}.")) + 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: '${update.feature}' to" + - s" maxVersionLevel:${update.maxVersionLevel} because it's lower than" + - s" the existing minVersionLevel:${existing.min}.")) + s"Can not downgrade finalized feature to maxVersionLevel:${update.maxVersionLevel}" + + s" because it's lower than the existing minVersionLevel:${existing.min}.")) } else { newVersionRangeOrError(update) } @@ -2011,11 +2008,18 @@ class KafkaController(val config: KafkaConfig, // written to FeatureZNode. val targetFeatures = scala.collection.mutable.Map[String, FinalizedVersionRange]() ++ existingFeatures // Map of feature to error. - var errors = scala.collection.mutable.Map[String, ApiError]() - - // Process each FeatureUpdate. - // If a FeatureUpdate is found to be valid, then the corresponding entry in errors would contain - // Errors.NONE. Otherwise the entry would contain the appropriate error. + val errors = scala.collection.mutable.Map[String, ApiError]() + + // Below we process each FeatureUpdate: + // - If a FeatureUpdate is found to be valid, then: + // - The corresponding entry in errors map would be updated to contain ApiError(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 => processFeatureUpdate(update, existingFeatures.get(update.feature())) match { case Left(newVersionRangeOrNone) => @@ -2044,12 +2048,10 @@ class KafkaController(val config: KafkaConfig, // 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 => - errors = errors.map { case (feature, apiError) => + errors.foreach { case (feature, apiError) => if (apiError.error() == Errors.NONE) { - (feature, new ApiError(Errors.FEATURE_UPDATE_FAILED, - Errors.FEATURE_UPDATE_FAILED.message() + " Error: " + e)) - } else { - (feature, apiError) + errors(feature) = new ApiError(Errors.FEATURE_UPDATE_FAILED, + Errors.FEATURE_UPDATE_FAILED.message() + " Error: " + e) } } } finally { diff --git a/core/src/main/scala/kafka/server/BrokerFeatures.scala b/core/src/main/scala/kafka/server/BrokerFeatures.scala index 677331829b594..00027915445d2 100644 --- a/core/src/main/scala/kafka/server/BrokerFeatures.scala +++ b/core/src/main/scala/kafka/server/BrokerFeatures.scala @@ -35,7 +35,7 @@ import scala.jdk.CollectionConverters._ * in the closed range: [1, latest_min_version_level - 1] get deprecated by the controller logic * that applies this map to persistent finalized feature state in ZK (this mutation happens * during controller election and during finalized feature updates via the - * ApiKeys.UPDATE_FINALIZED_FEATURES api). This will automatically mean external clients of Kafka + * ApiKeys.UPDATE_FEATURES api). This will automatically mean external clients of Kafka * would need to stop using the finalized min version levels that have been deprecated. * * This class also provides APIs to check for incompatibilities between the features supported by @@ -88,7 +88,7 @@ class BrokerFeatures private (@volatile var supportedFeatures: Features[Supporte /** * 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 + * 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] diff --git a/core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala b/core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala index 152bd0c6f9861..567a3f360bf32 100644 --- a/core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala +++ b/core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala @@ -201,7 +201,7 @@ class UpdateFeaturesTest extends BaseRequestTest { testWithInvalidFeatureUpdate[InvalidRequestException]( "feature_1", new FeatureUpdate((defaultFinalizedFeatures().get("feature_1").max() - 1).asInstanceOf[Short],false), - ".*Can not downgrade finalized feature: 'feature_1'.*allowDowngrade.*".r) + ".*Can not downgrade finalized feature.*allowDowngrade.*".r) } /** @@ -213,7 +213,7 @@ class UpdateFeaturesTest extends BaseRequestTest { testWithInvalidFeatureUpdate[InvalidRequestException]( "feature_1", new FeatureUpdate(defaultSupportedFeatures().get("feature_1").max(), true), - ".*finalized feature: 'feature_1'.*allowDowngrade.* provided maxVersionLevel:3.*existing maxVersionLevel:2.*".r) + ".*When the allowDowngrade flag set in the request, the provided maxVersionLevel:3.*existing maxVersionLevel:2.*".r) } /** @@ -250,8 +250,8 @@ class UpdateFeaturesTest extends BaseRequestTest { 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 for feature: 'feature_1'.*allowDowngrade.*".r - assertTrue(exceptionMsgPattern.findFirstIn(result.errorMessage).isDefined) + val exceptionMsgPattern = ".*Can not provide maxVersionLevel: 0 less than 1.*allowDowngrade.*".r + assertTrue(result.errorMessage, exceptionMsgPattern.findFirstIn(result.errorMessage).isDefined) checkFeatures( adminClient, nodeBefore, @@ -267,7 +267,7 @@ class UpdateFeaturesTest extends BaseRequestTest { testWithInvalidFeatureUpdate[InvalidRequestException]( "feature_non_existing", new FeatureUpdate(0, true), - ".*Can not delete non-existing finalized feature: 'feature_non_existing'.*".r) + ".*Can not delete non-existing finalized feature.*".r) } /** @@ -279,7 +279,7 @@ class UpdateFeaturesTest extends BaseRequestTest { testWithInvalidFeatureUpdate[InvalidRequestException]( "feature_1", new FeatureUpdate(defaultFinalizedFeatures().get("feature_1").max(), false), - ".*Can not upgrade a finalized feature: 'feature_1'.*to the same value.*".r) + ".*Can not upgrade a finalized feature.*to the same value.*".r) } /** @@ -307,7 +307,7 @@ class UpdateFeaturesTest extends BaseRequestTest { checkException[InvalidRequestException]( result, - Map("feature_1" -> ".*Can not downgrade finalized feature: 'feature_1' to maxVersionLevel:1.*existing minVersionLevel:2.*".r)) + Map("feature_1" -> ".*Can not downgrade finalized feature to maxVersionLevel:1.*existing minVersionLevel:2.*".r)) checkFeatures( adminClient, nodeBefore, @@ -483,7 +483,7 @@ class UpdateFeaturesTest extends BaseRequestTest { 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: 'feature_2'.*allowDowngrade.*".r)) + 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")), From a7c32a0ee6eaf045e01981f5b047e2832a74a685 Mon Sep 17 00:00:00 2001 From: Kowshik Prakasam Date: Tue, 22 Sep 2020 19:17:27 -0700 Subject: [PATCH 20/41] Address latest review comments --- .../kafka/clients/admin/FeatureMetadata.java | 79 ++++++---- .../clients/admin/FinalizedVersionRange.java | 84 +++++++++++ .../kafka/clients/admin/KafkaAdminClient.java | 62 ++++---- .../clients/admin/SupportedVersionRange.java | 80 ++++++++++ .../requests/UpdateFeaturesResponse.java | 24 +-- .../common/message/UpdateFeaturesRequest.json | 2 +- .../message/UpdateFeaturesResponse.json | 8 +- .../clients/admin/KafkaAdminClientTest.java | 141 +++++++++++------- .../kafka/controller/KafkaController.scala | 99 +++++++----- .../scala/kafka/server/BrokerFeatures.scala | 33 +++- .../main/scala/kafka/server/KafkaApis.scala | 28 ++-- .../ControllerIntegrationTest.scala | 2 +- .../unit/kafka/server/RequestQuotaTest.scala | 3 + .../kafka/server/UpdateFeaturesTest.scala | 123 +++++++++++---- 14 files changed, 564 insertions(+), 204 deletions(-) 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 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 index 78b1e366ade99..bed388203aced 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/FeatureMetadata.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/FeatureMetadata.java @@ -16,11 +16,15 @@ */ 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; -import org.apache.kafka.common.feature.Features; -import org.apache.kafka.common.feature.FinalizedVersionRange; -import org.apache.kafka.common.feature.SupportedVersionRange; +import org.apache.kafka.common.message.ApiVersionsResponseData.FinalizedFeatureKey; +import org.apache.kafka.common.message.ApiVersionsResponseData.SupportedFeatureKey; +import org.apache.kafka.common.requests.ApiVersionsResponse; /** * Encapsulates details about finalized as well as supported features. This is particularly useful @@ -28,32 +32,45 @@ */ public class FeatureMetadata { - private final Features finalizedFeatures; + private final Map finalizedFeatures; private final Optional finalizedFeaturesEpoch; - private final Features supportedFeatures; + private final Map supportedFeatures; + + public FeatureMetadata(final Map finalizedFeatures, + final Optional finalizedFeaturesEpoch, + final Map supportedFeatures) { + this.finalizedFeatures = new HashMap<>(finalizedFeatures); + this.finalizedFeaturesEpoch = finalizedFeaturesEpoch; + this.supportedFeatures = new HashMap<>(supportedFeatures); + } + + public FeatureMetadata(ApiVersionsResponse response) { + this.supportedFeatures = new HashMap<>(); + for (SupportedFeatureKey key : response.data().supportedFeatures().valuesSet()) { + supportedFeatures.put(key.name(), new SupportedVersionRange(key.minVersion(), key.maxVersion())); + } + + this.finalizedFeatures = new HashMap<>(); + for (FinalizedFeatureKey key : response.data().finalizedFeatures().valuesSet()) { + finalizedFeatures.put(key.name(), new FinalizedVersionRange(key.minVersionLevel(), key.maxVersionLevel())); + } - public FeatureMetadata(final Features finalizedFeatures, - final int finalizedFeaturesEpoch, - final Features supportedFeatures) { - Objects.requireNonNull(finalizedFeatures, "Provided finalizedFeatures can not be null."); - Objects.requireNonNull(supportedFeatures, "Provided supportedFeatures can not be null."); - this.finalizedFeatures = finalizedFeatures; - if (finalizedFeaturesEpoch >= 0) { - this.finalizedFeaturesEpoch = Optional.of(finalizedFeaturesEpoch); + if (response.data().finalizedFeaturesEpoch() >= 0) { + this.finalizedFeaturesEpoch = Optional.of(response.data().finalizedFeaturesEpoch()); } else { this.finalizedFeaturesEpoch = Optional.empty(); } - this.supportedFeatures = supportedFeatures; } /** - * A map of finalized feature versions, with key being finalized feature name and value - * containing the min/max version levels for the finalized feature. + * 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 Features finalizedFeatures() { - return finalizedFeatures; + public Map finalizedFeatures() { + return new HashMap<>(finalizedFeatures); } /** @@ -65,11 +82,12 @@ public Optional finalizedFeaturesEpoch() { } /** - * A map of supported feature versions, with key being supported feature name and value - * containing the min/max version for the supported feature. + * 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 Features supportedFeatures() { - return supportedFeatures; + public Map supportedFeatures() { + return new HashMap<>(supportedFeatures); } @Override @@ -92,12 +110,23 @@ 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{finalized:%s, finalizedFeaturesEpoch:%s, supported:%s}", - finalizedFeatures, + "FeatureMetadata{finalizedFeatures:%s, finalizedFeaturesEpoch:%s, supportedFeatures:%s}", + mapToString(finalizedFeatures), finalizedFeaturesEpoch.map(Object::toString).orElse(""), - supportedFeatures); + mapToString(supportedFeatures)); } } 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..d4c47fd843a18 --- /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. + */ + public 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{minVersionLevel:%d, maxVersionLevel:%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 98940ad5b9803..46ff921f32013 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 @@ -4360,16 +4360,12 @@ ApiVersionsRequest.Builder createRequest(int timeoutMs) { void handleResponse(AbstractResponse response) { final ApiVersionsResponse apiVersionsResponse = (ApiVersionsResponse) response; if (apiVersionsResponse.data.errorCode() == Errors.NONE.code()) { - future.complete( - new FeatureMetadata( - apiVersionsResponse.finalizedFeatures(), - apiVersionsResponse.finalizedFeaturesEpoch(), - apiVersionsResponse.supportedFeatures())); - } else if (options.sendRequestToController() && apiVersionsResponse.data.errorCode() == Errors.NOT_CONTROLLER.code()) { + future.complete(new FeatureMetadata(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()); + future.completeExceptionally(Errors.forCode(apiVersionsResponse.data.errorCode()).exception()); } } @@ -4410,31 +4406,35 @@ void handleResponse(AbstractResponse abstractResponse) { final UpdateFeaturesResponse response = (UpdateFeaturesResponse) abstractResponse; - // Check for controller change. - for (UpdatableFeatureResult result : response.data().results()) { - final Errors error = Errors.forCode(result.errorCode()); - if (error == Errors.NOT_CONTROLLER) { - handleNotControllerError(error); - throw error.exception(); - } - } - - for (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())); + Errors topLevelError = Errors.forCode(response.data().errorCode()); + switch (topLevelError) { + case NONE: + for (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(Errors.forCode(topLevelError.code())); + break; + default: + for (Map.Entry> entry : updateFutures.entrySet()) { + entry.getValue().completeExceptionally(topLevelError.exception()); + } + break; } - // 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); } @Override 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..586c37ed82302 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/admin/SupportedVersionRange.java @@ -0,0 +1,80 @@ +/* + * 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 condition is met: + * minVersion >= 1 and maxVersion >= 1 and maxVersion >= minVersion. + * + * @param minVersion The minimum version value. + * @param maxVersion The maximum version value. + * + * @throws IllegalArgumentException Raised when the condition described above is not met. + */ + public SupportedVersionRange(final short minVersion, final short maxVersion) { + if (minVersion < 1 || maxVersion < 1 || maxVersion < minVersion) { + throw new IllegalArgumentException( + String.format( + "Expected minVersion >= 1, maxVersion >= 1 and maxVersion >= minVersion, 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 instanceof SupportedVersionRange)) { + 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("FinalizedVersionRange{minVersion:%d, maxVersion:%d}", minVersion, maxVersion); + } +} + 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 index c020622384b98..93d448e602247 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesResponse.java @@ -17,7 +17,6 @@ package org.apache.kafka.common.requests; import java.nio.ByteBuffer; -import java.util.HashMap; import java.util.Map; import java.util.stream.Collectors; import org.apache.kafka.common.message.UpdateFeaturesResponseData; @@ -65,6 +64,11 @@ public Map errorCounts() { return apiErrorCounts(errors()); } + @Override + public int throttleTimeMs() { + return data.throttleTimeMs(); + } + @Override protected Struct toStruct(short version) { return data.toStruct(version); @@ -83,7 +87,7 @@ public static UpdateFeaturesResponse parse(ByteBuffer buffer, short version) { return new UpdateFeaturesResponse(ApiKeys.UPDATE_FEATURES.parseResponse(version, buffer), version); } - public static UpdateFeaturesResponse createWithFeatureUpdateApiErrors(Map updateErrors) { + public static UpdateFeaturesResponse createWithErrors(ApiError topLevelError, Map updateErrors, int throttleTimeMs) { final UpdatableFeatureResultCollection results = new UpdatableFeatureResultCollection(); for (Map.Entry updateError : updateErrors.entrySet()) { final String feature = updateError.getKey(); @@ -94,14 +98,12 @@ public static UpdateFeaturesResponse createWithFeatureUpdateApiErrors(Map updateErrors) { - final Map updateApiErrors = new HashMap<>(); - for (Map.Entry entry : updateErrors.entrySet()) { - updateApiErrors.put(entry.getKey(), new ApiError(entry.getValue())); - } - return createWithFeatureUpdateApiErrors(updateApiErrors); + 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/UpdateFeaturesRequest.json b/clients/src/main/resources/common/message/UpdateFeaturesRequest.json index 589903c00c64a..82ebe0e9f45e5 100644 --- a/clients/src/main/resources/common/message/UpdateFeaturesRequest.json +++ b/clients/src/main/resources/common/message/UpdateFeaturesRequest.json @@ -14,7 +14,7 @@ // limitations under the License. { - "apiKey": 50, + "apiKey": 56, "type": "request", "name": "UpdateFeaturesRequest", "validVersions": "0", diff --git a/clients/src/main/resources/common/message/UpdateFeaturesResponse.json b/clients/src/main/resources/common/message/UpdateFeaturesResponse.json index 1aaf4d8c8b914..142c36dee5c09 100644 --- a/clients/src/main/resources/common/message/UpdateFeaturesResponse.json +++ b/clients/src/main/resources/common/message/UpdateFeaturesResponse.json @@ -14,12 +14,18 @@ // limitations under the License. { - "apiKey": 50, + "apiKey": 56, "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, 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 f2ffa52413463..d9993d9a2b669 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 @@ -67,8 +67,6 @@ 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.feature.FinalizedVersionRange; -import org.apache.kafka.common.feature.SupportedVersionRange; import org.apache.kafka.common.message.AlterPartitionReassignmentsResponseData; import org.apache.kafka.common.message.AlterReplicaLogDirsResponseData; import org.apache.kafka.common.message.AlterReplicaLogDirsResponseData.AlterReplicaLogDirPartitionResult; @@ -486,11 +484,37 @@ private static DescribeGroupsResponseData prepareDescribeGroupsResponseData(Stri private static FeatureMetadata defaultFeatureMetadata() { return new FeatureMetadata( - Features.finalizedFeatures( - Utils.mkMap(Utils.mkEntry("test_feature_1", new FinalizedVersionRange((short) 2, (short) 3)))), - 1, - Features.supportedFeatures( - Utils.mkMap(Utils.mkEntry("test_feature_1", new SupportedVersionRange((short) 1, (short) 5))))); + Utils.mkMap(Utils.mkEntry("test_feature_1", new FinalizedVersionRange((short) 2, (short) 3))), + Optional.of(1), + Utils.mkMap(Utils.mkEntry("test_feature_1", new SupportedVersionRange((short) 1, (short) 5)))); + } + + private static Features + convertSupportedFeaturesMap(Map features) { + Map featuresMap = new HashMap<>(); + for (Map.Entry entry : features.entrySet()) { + 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) { + Map featuresMap = new HashMap<>(); + for (Map.Entry entry : features.entrySet()) { + 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) { @@ -499,9 +523,9 @@ private static ApiVersionsResponse prepareApiVersionsResponseForDescribeFeatures ApiVersionsResponse.DEFAULT_API_VERSIONS_RESPONSE.throttleTimeMs(), error, ApiVersionsResponse.DEFAULT_API_VERSIONS_RESPONSE.data().apiKeys(), - defaultFeatureMetadata().supportedFeatures(), - defaultFeatureMetadata().finalizedFeatures(), - defaultFeatureMetadata().finalizedFeaturesEpoch().orElse(-1))); + convertSupportedFeaturesMap(defaultFeatureMetadata().supportedFeatures()), + convertFinalizedFeaturesMap(defaultFeatureMetadata().finalizedFeatures()), + defaultFeatureMetadata().finalizedFeaturesEpoch().get())); } return new ApiVersionsResponse(new ApiVersionsResponseData().setErrorCode(error.code())); } @@ -3916,76 +3940,91 @@ public void testListOffsetsNonRetriableErrors() throws Exception { } } - @Test - public void testUpdateFeaturesDuringSuccess() throws Exception { - final Map updates = makeTestFeatureUpdates(); - testUpdateFeatures(updates, makeTestFeatureUpdateErrors(updates, Errors.NONE)); - } - - @Test - public void testUpdateFeaturesInvalidRequestError() throws Exception { - final Map updates = makeTestFeatureUpdates(); - testUpdateFeatures(updates, makeTestFeatureUpdateErrors(updates, Errors.INVALID_REQUEST)); - } - - @Test - public void testUpdateFeaturesUpdateFailedError() throws Exception { - final Map updates = makeTestFeatureUpdates(); - testUpdateFeatures(updates, makeTestFeatureUpdateErrors(updates, Errors.FEATURE_UPDATE_FAILED)); - } - - @Test - public void testUpdateFeaturesPartialSuccess() throws Exception { - final Map errors = makeTestFeatureUpdateErrors(makeTestFeatureUpdates(), Errors.NONE); - errors.put("test_feature_2", Errors.INVALID_REQUEST); - testUpdateFeatures(makeTestFeatureUpdates(), errors); - } - 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<>(); + private Map makeTestFeatureUpdateErrors(final Map updates, final Errors error) { + final Map errors = new HashMap<>(); for (Map.Entry entry : updates.entrySet()) { - errors.put(entry.getKey(), error); + errors.put(entry.getKey(), new ApiError(error)); } return errors; } private void testUpdateFeatures(Map featureUpdates, - Map featureUpdateErrors) throws Exception { + ApiError topLevelError, + Map featureUpdateErrors) throws Exception { try (final AdminClientUnitTestEnv env = mockClientEnv()) { env.kafkaClient().prepareResponse( body -> body instanceof UpdateFeaturesRequest, - UpdateFeaturesResponse.createWithFeatureUpdateErrors(featureUpdateErrors)); + UpdateFeaturesResponse.createWithErrors(topLevelError, featureUpdateErrors, 0)); final Map> futures = env.adminClient().updateFeatures( featureUpdates, new UpdateFeaturesOptions().timeoutMs(10000)).values(); for (Map.Entry> entry : futures.entrySet()) { final KafkaFuture future = entry.getValue(); - final Errors error = featureUpdateErrors.get(entry.getKey()); - if (error == Errors.NONE) { - future.get(); + 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(), error.exception().getClass()); + 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.createWithFeatureUpdateErrors(Utils.mkMap( - Utils.mkEntry("test_feature_1", Errors.NOT_CONTROLLER), - Utils.mkEntry("test_feature_2", Errors.NOT_CONTROLLER))), + 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(), @@ -3994,9 +4033,11 @@ public void testUpdateFeaturesHandleNotControllerException() throws Exception { Collections.emptyList())); env.kafkaClient().prepareResponseFrom( request -> request instanceof UpdateFeaturesRequest, - UpdateFeaturesResponse.createWithFeatureUpdateErrors(Utils.mkMap( - Utils.mkEntry("test_feature_1", Errors.NONE), - Utils.mkEntry("test_feature_2", Errors.NONE))), + 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( diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index cb1a9a0d64cff..c86b453cd3e31 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -64,7 +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 = (Map[String, ApiError]) => Unit + type UpdateFeaturesCallback = Either[ApiError, Map[String, ApiError]] => Unit } class KafkaController(val config: KafkaConfig, @@ -1192,30 +1192,14 @@ class KafkaController(val config: KafkaConfig, /** * Send the leader information for selected partitions to selected brokers so that they can correctly respond to - * metadata requests. Particularly, when feature versioning is enabled, we filter out brokers with incompatible - * features from receiving the metadata requests. This is because we do not want to activate incompatible brokers, - * as these may have harmful consequences to the cluster. + * metadata requests * * @param brokers The brokers that the update metadata request should be sent to */ private[controller] def sendUpdateMetadataRequest(brokers: Seq[Int], partitions: Set[TopicPartition]): Unit = { try { - val filteredBrokers = scala.collection.mutable.Set[Int]() ++ brokers - if (config.isFeatureVersioningEnabled) { - def hasIncompatibleFeatures(broker: Broker): Boolean = { - featureCache.get.exists( - latestFinalizedFeatures => - BrokerFeatures.hasIncompatibleFeatures(broker.features, latestFinalizedFeatures.features)) - } - controllerContext.liveOrShuttingDownBrokers.foreach(broker => { - if (filteredBrokers.contains(broker.id) && hasIncompatibleFeatures(broker)) { - warn(s"No UpdateMetadataRequest will be sent to broker: ${broker.id} due to incompatible features") - filteredBrokers -= broker.id - } - }) - } brokerRequestBatch.newBatch() - brokerRequestBatch.addUpdateMetadataRequestForBrokers(filteredBrokers.toSeq, partitions) + brokerRequestBatch.addUpdateMetadataRequestForBrokers(brokers, partitions) brokerRequestBatch.sendRequestsToBrokers(epoch) } catch { case e: IllegalStateException => @@ -1570,6 +1554,27 @@ class KafkaController(val config: KafkaConfig, } } + /** + * Partitions the provided map of brokers and epochs into 2 new maps: + * - The first map contains brokers whose features were found to be compatible with the existing + * finalized features. + * - The second map contains 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]) = { + brokersAndEpochs.partition { + case (broker, _) => + !config.isFeatureVersioningEnabled || + !featureCache.get.exists( + latestFinalizedFeatures => + BrokerFeatures.hasIncompatibleFeatures(broker.features, latestFinalizedFeatures.features)) + } + } + private def processBrokerChange(): Unit = { if (!isActive) return val curBrokerAndEpochs = zkClient.getAllBrokerAndEpochsInCluster @@ -1595,14 +1600,25 @@ 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) { @@ -1895,16 +1911,29 @@ class KafkaController(val config: KafkaConfig, Right(new ApiError(Errors.INVALID_REQUEST, incompatibilityError)) } else { val defaultMinVersionLevel = brokerFeatures.defaultMinVersionLevel(update.feature) - val newVersionRange = new FinalizedVersionRange(defaultMinVersionLevel, update.maxVersionLevel) - val numIncompatibleBrokers = controllerContext.liveOrShuttingDownBrokers.count(broker => { - val singleFinalizedFeature = - Features.finalizedFeatures(Utils.mkMap(Utils.mkEntry(update.feature, newVersionRange))) - BrokerFeatures.hasIncompatibleFeatures(broker.features, singleFinalizedFeature) - }) - if (numIncompatibleBrokers == 0) { - Left(newVersionRange) + var newVersionRange: FinalizedVersionRange = null + try { + newVersionRange = new FinalizedVersionRange(defaultMinVersionLevel, update.maxVersionLevel) + } catch { + // Ignoring because it means the provided maxVersionLevel is invalid. + case _: IllegalArgumentException => {} + } + 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" default minVersionLevel:$defaultMinVersionLevel.")) } else { - Right(new ApiError(Errors.INVALID_REQUEST, incompatibilityError)) + val numIncompatibleBrokers = controllerContext.liveOrShuttingDownBrokers.count(broker => { + val singleFinalizedFeature = + Features.finalizedFeatures(Utils.mkMap(Utils.mkEntry(update.feature, newVersionRange))) + BrokerFeatures.hasIncompatibleFeatures(broker.features, singleFinalizedFeature) + }) + if (numIncompatibleBrokers == 0) { + Left(newVersionRange) + } else { + Right(new ApiError(Errors.INVALID_REQUEST, incompatibilityError)) + } } } } @@ -1991,10 +2020,7 @@ class KafkaController(val config: KafkaConfig, if (isActive) { processFeatureUpdatesWithActiveController(request, callback) } else { - val results = request.data().featureUpdates().asScala.map { - update => update.feature() -> new ApiError(Errors.NOT_CONTROLLER) - }.toMap - callback(results) + callback(Left(new ApiError(Errors.NOT_CONTROLLER))) } } @@ -2037,7 +2063,7 @@ class KafkaController(val config: KafkaConfig, // of the FeatureZNode with the new features. This may result in partial or full modification // of the existing finalized features. if (existingFeatures.equals(targetFeatures)) { - callback(errors) + callback(Right(errors)) } else { try { val newNode = new FeatureZNode(FeatureZNodeStatus.Enabled, Features.finalizedFeatures(targetFeatures.asJava)) @@ -2055,7 +2081,7 @@ class KafkaController(val config: KafkaConfig, } } } finally { - callback(errors) + callback(Right(errors)) } } } @@ -2686,6 +2712,7 @@ case class ListPartitionReassignments(partitionsOpt: Option[Set[TopicPartition]] case class UpdateFeatures(request: UpdateFeaturesRequest, callback: UpdateFeaturesCallback) extends ControllerEvent { override def state: ControllerState = ControllerState.UpdateFeatures + override def preempt(): Unit = {} } diff --git a/core/src/main/scala/kafka/server/BrokerFeatures.scala b/core/src/main/scala/kafka/server/BrokerFeatures.scala index 00027915445d2..e000e92cb43ee 100644 --- a/core/src/main/scala/kafka/server/BrokerFeatures.scala +++ b/core/src/main/scala/kafka/server/BrokerFeatures.scala @@ -24,23 +24,40 @@ import org.apache.kafka.common.feature.Features._ import scala.jdk.CollectionConverters._ /** - * A class that encapsulates the following: + * A class that encapsulates the attributes explained below 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. + * + * Attributes: * * 1. The latest features supported by the Broker. * - * 2. The default minimum version levels for specific features. This map enables feature - * version level deprecation. This is how it works: in order to deprecate feature version levels, - * in this map the default minimum version level of a feature can be set to a new value that's - * higher than 1 (let's call this latest_min_version_level). In doing so, the feature version levels + * 2. The optional default minimum version levels for specific finalized features. + * - If you would want to deprecate a version level for some feature, then in this map you + * need to supply the starting version value (greater than 1) that's just 1 beyond the highest + * deprecated version. Ex: if this map contains {"feature1" -> 5}, then it indicates that feature + * version levels: [1, 4] need to be deprecated. The value '5' is the default minimum version level. + * - If you do not want to deprecate a version level for a feature, you do not have to supply + * values in this map. The default minimum version level for absent features in this map + * is assumed to be 1. + * + * The primary use case to provide this map is feature version level deprecation. + * When features are finalized via the ApiKeys.UPDATE_FEATURES api, the controller takes the + * value provided in this map (if present) as the default minimum version level for the feature. + * This is how it works: in order to deprecate feature version levels, in this map the default + * minimum version level of a feature can be set to a new value that's higher than 1 + * (let's call this latest_min_version_level). In doing so, the feature version levels * in the closed range: [1, latest_min_version_level - 1] get deprecated by the controller logic * that applies this map to persistent finalized feature state in ZK (this mutation happens * during controller election and during finalized feature updates via the * ApiKeys.UPDATE_FEATURES api). This will automatically mean external clients of Kafka * would need to stop using the finalized min version levels that have been deprecated. * - * This class 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. + * NOTE: The difference between the values in this map and the minimum version value for a + * broker's supported feature is the following: Version levels below the values specified in this + * map are considered deprecated by the controller, whereas version levels below the minimum + * version value for a supported feature are considered unknown/unsupported. */ class BrokerFeatures private (@volatile var supportedFeatures: Features[SupportedVersionRange], @volatile var defaultFeatureMinVersionLevels: Map[String, Short]) { diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 69fb17642c7c6..66d5dc0d7a8af 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -3112,20 +3112,30 @@ class KafkaApis(val requestChannel: RequestChannel, def handleUpdateFeatures(request: RequestChannel.Request): Unit = { val updateFeaturesRequest = request.body[UpdateFeaturesRequest] - def featureUpdateErrors(error: Errors, msgOverride: Option[String]): Map[String, ApiError] = { - updateFeaturesRequest.data().featureUpdates().asScala.map( - update => update.feature() -> new ApiError(error, msgOverride.getOrElse(error.message())) - ).toMap - } - def sendResponseCallback(updateErrors: Map[String, ApiError]): Unit = { - sendResponseExemptThrottle(request, UpdateFeaturesResponse.createWithFeatureUpdateApiErrors(updateErrors.asJava)) + def sendResponseCallback(errors: Either[ApiError, Map[String, ApiError]]): Unit = { + def createResponse(throttleTimeMs: Int): UpdateFeaturesResponse = { + errors match { + case Left(topLevelError) => { + val featureUpdateNoErrors = updateFeaturesRequest + .data().featureUpdates().asScala + .map(update => update.feature() -> ApiError.NONE) + .toMap.asJava + UpdateFeaturesResponse.createWithErrors(topLevelError, featureUpdateNoErrors, 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(featureUpdateErrors(Errors.CLUSTER_AUTHORIZATION_FAILED, Option.empty)) + sendResponseCallback(Left(new ApiError(Errors.CLUSTER_AUTHORIZATION_FAILED))) } else if (!config.isFeatureVersioningEnabled) { - sendResponseCallback(featureUpdateErrors(Errors.INVALID_REQUEST, Some("Feature versioning system is disabled."))) + sendResponseCallback(Left(new ApiError(Errors.INVALID_REQUEST, "Feature versioning system is disabled."))) } else { controller.updateFeatures(updateFeaturesRequest, sendResponseCallback) } diff --git a/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala b/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala index c5e1e9a36801a..e2c3a00700e41 100644 --- a/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala @@ -27,7 +27,7 @@ import kafka.server.{KafkaConfig, KafkaServer} import kafka.utils.{LogCaptureAppender, TestUtils} import kafka.zk._ import org.junit.{After, Before, Test} -import org.junit.Assert.{assertEquals, assertNotEquals, assertTrue} +import org.junit.Assert.{assertEquals, assertTrue} import org.apache.kafka.common.{ElectionType, TopicPartition} import org.apache.kafka.common.errors.{ControllerMovedException, StaleBrokerEpochException} import org.apache.kafka.common.metrics.KafkaMetric 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/UpdateFeaturesTest.scala b/core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala index 567a3f360bf32..c29fa5df44ec4 100644 --- a/core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala +++ b/core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala @@ -17,7 +17,7 @@ package kafka.server -import java.util.Properties +import java.util.{Optional, Properties} import java.util.concurrent.ExecutionException import kafka.api.KAFKA_2_7_IV0 @@ -34,7 +34,7 @@ import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{FeatureUpdate, UpdateFeaturesRequest, UpdateFeaturesResponse} import org.apache.kafka.common.utils.Utils import org.junit.Test -import org.junit.Assert.{assertEquals, assertFalse, assertNotEquals, assertNotNull, assertTrue} +import org.junit.Assert.{assertEquals, assertFalse, assertNotEquals, assertNotNull, assertNull, assertTrue} import org.scalatest.Assertions.intercept import scala.jdk.CollectionConverters._ @@ -107,6 +107,19 @@ class UpdateFeaturesTest extends BaseRequestTest { FeatureZNode.decode(mayBeFeatureZNodeBytes.get) } + private def makeFeatureMetadata(finalized: Features[FinalizedVersionRange], + epoch: Integer, + supported: Features[SupportedVersionRange]): FeatureMetadata = { + new FeatureMetadata( + finalized.features().asScala.map { + case(name, versionRange) => (name, new org.apache.kafka.clients.admin.FinalizedVersionRange(versionRange.min(), versionRange.max())) + }.asJava, + Optional.of(epoch), + supported.features().asScala.map { + case(name, versionRange) => (name, new org.apache.kafka.clients.admin.SupportedVersionRange(versionRange.min(), versionRange.max())) + }.asJava) + } + private def checkFeatures(client: Admin, expectedNode: FeatureZNode, expectedMetadata: FeatureMetadata): Unit = { assertEquals(expectedNode, getFeatureZNode()) val featureMetadata = client.describeFeatures( @@ -125,9 +138,9 @@ class UpdateFeaturesTest extends BaseRequestTest { val cause = exception.getCause assertNotNull(cause) assertEquals(cause.getClass, tag.runtimeClass) - assertTrue(cause.getMessage, exceptionMsgPattern.findFirstIn(cause.getMessage).isDefined) + assertTrue(s"Received unexpected error message: ${cause.getMessage}", + exceptionMsgPattern.findFirstIn(cause.getMessage).isDefined) } - } /** @@ -155,7 +168,7 @@ class UpdateFeaturesTest extends BaseRequestTest { checkFeatures( adminClient, nodeBefore, - new FeatureMetadata(defaultFinalizedFeatures(), versionBefore, defaultSupportedFeatures())) + makeFeatureMetadata(defaultFinalizedFeatures(), versionBefore, defaultSupportedFeatures())) } /** @@ -180,16 +193,18 @@ class UpdateFeaturesTest extends BaseRequestTest { new UpdateFeaturesRequest.Builder(new UpdateFeaturesRequestData().setFeatureUpdates(updates)).build(), notControllerSocketServer) + assertEquals(Errors.NOT_CONTROLLER, Errors.forCode(response.data.errorCode())) + assertNotNull(response.data.errorMessage()) + assertEquals(1, response.data.results.size) val result = response.data.results.asScala.head assertEquals("feature_1", result.feature) - assertEquals(Errors.NOT_CONTROLLER, Errors.forCode(result.errorCode)) - assertNotNull(result.errorMessage) - assertFalse(result.errorMessage.isEmpty) + assertEquals(Errors.NONE, Errors.forCode(result.errorCode)) + assertNull(result.errorMessage) checkFeatures( createAdminClient(), nodeBefore, - new FeatureMetadata(defaultFinalizedFeatures(), versionBefore, defaultSupportedFeatures())) + makeFeatureMetadata(defaultFinalizedFeatures(), versionBefore, defaultSupportedFeatures())) } /** @@ -255,7 +270,7 @@ class UpdateFeaturesTest extends BaseRequestTest { checkFeatures( adminClient, nodeBefore, - new FeatureMetadata(defaultFinalizedFeatures(), versionBefore, defaultSupportedFeatures())) + makeFeatureMetadata(defaultFinalizedFeatures(), versionBefore, defaultSupportedFeatures())) } /** @@ -282,23 +297,23 @@ class UpdateFeaturesTest extends BaseRequestTest { ".*Can not upgrade a finalized feature.*to the same value.*".r) } - /** - * Tests that an UpdateFeatures request fails in the Controller, when, a feature version level - * downgrade is attempted to a version level thats below the default min version level for the - * feature. - */ - @Test - def testShouldFailRequestWhenDowngradingBelowMinVersionLevel(): Unit = { + private def testShouldFailRequestWhenNewMaxVersionLevelIsBelowDefaultMinVersionLevel( + minVersionLevel: Short, + initialMaxVersionLevel: Option[Short], + allowDowngrade: Boolean + ): Unit = { TestUtils.waitUntilControllerElected(zkClient) updateSupportedFeaturesInAllBrokers(defaultSupportedFeatures()) - val minVersionLevel = 2.asInstanceOf[Short] + val initialFinalizedFeatures = initialMaxVersionLevel.map( + maxVersionLevel => Features.finalizedFeatures( + Utils.mkMap(Utils.mkEntry("feature_1", new FinalizedVersionRange(minVersionLevel, maxVersionLevel)))) + ).getOrElse(Features.emptyFinalizedFeatures()) updateDefaultMinVersionLevelsInAllBrokers(Map[String, Short]("feature_1" -> minVersionLevel)) - val initialFinalizedFeatures = Features.finalizedFeatures( - Utils.mkMap(Utils.mkEntry("feature_1", new FinalizedVersionRange(minVersionLevel, 2)))) val versionBefore = updateFeatureZNode(initialFinalizedFeatures) - val update = new FeatureUpdate((minVersionLevel - 1).asInstanceOf[Short], true) + val newMaxVersionLevel = (minVersionLevel - 1).asInstanceOf[Short] + val update = new FeatureUpdate(newMaxVersionLevel, allowDowngrade) val adminClient = createAdminClient() val nodeBefore = getFeatureZNode() @@ -307,19 +322,45 @@ class UpdateFeaturesTest extends BaseRequestTest { checkException[InvalidRequestException]( result, - Map("feature_1" -> ".*Can not downgrade finalized feature to maxVersionLevel:1.*existing minVersionLevel:2.*".r)) + Map("feature_1" -> s".*maxVersionLevel:$newMaxVersionLevel.*minVersionLevel:$minVersionLevel.*".r)) checkFeatures( adminClient, nodeBefore, - new FeatureMetadata(initialFinalizedFeatures, versionBefore, defaultSupportedFeatures())) + makeFeatureMetadata(initialFinalizedFeatures, versionBefore, defaultSupportedFeatures())) } /** * Tests that an UpdateFeatures request fails in the Controller, when, a feature version level - * upgrade introduces a version incompatibility with existing supported features. + * downgrade is attempted to a version level thats below the default min version level for the + * feature. + */ + @Test + def testShouldFailRequestWhenDowngradingBelowMinVersionLevel(): Unit = { + testShouldFailRequestWhenNewMaxVersionLevelIsBelowDefaultMinVersionLevel(2, Some(2.asInstanceOf[Short]), true) + } + + /** + * Tests that an UpdateFeatures request fails in the Controller, when for a non-existing feature, + * a version level upgrade is attempted to a value thats below the default min version + * level for the feature. + */ + @Test + def testShouldFailRequestWhenUpgradingBelowDefaultMinVersionLevelOfANonExistingFinalizedFeature(): Unit = { + testShouldFailRequestWhenNewMaxVersionLevelIsBelowDefaultMinVersionLevel(2, Option.empty, false) + } + + /** + * Tests that an UpdateFeatures request fails in the Controller, when for a non-existing feature, + * a version level downgrade is attempted to a value thats below the default min version + * level for the feature. */ @Test - def testShouldFailRequestDuringBrokerMaxVersionLevelIncompatibility(): Unit = { + def testShouldFailRequestWhenDowngradingBelowDefaultMinVersionLevelOfANonExistingFinalizedFeature(): Unit = { + testShouldFailRequestWhenNewMaxVersionLevelIsBelowDefaultMinVersionLevel(2, Option.empty, true) + } + + + private def testShouldFailRequestDuringBrokerMaxVersionLevelIncompatibility(initialFinalizedFeatures: Features[FinalizedVersionRange]): Unit = { TestUtils.waitUntilControllerElected(zkClient) val controller = servers.filter { server => server.kafkaController.isActive}.head @@ -340,7 +381,7 @@ class UpdateFeaturesTest extends BaseRequestTest { unsupportedMaxVersion)))) updateSupportedFeatures(badSupportedFeatures, unsupportedBrokers) - val versionBefore = updateFeatureZNode(defaultFinalizedFeatures()) + val versionBefore = updateFeatureZNode(initialFinalizedFeatures) val invalidUpdate = new FeatureUpdate(defaultSupportedFeatures().get("feature_1").max(), false) val nodeBefore = getFeatureZNode() @@ -353,7 +394,27 @@ class UpdateFeaturesTest extends BaseRequestTest { checkFeatures( adminClient, nodeBefore, - new FeatureMetadata(defaultFinalizedFeatures(), versionBefore, defaultSupportedFeatures())) + makeFeatureMetadata(initialFinalizedFeatures, versionBefore, defaultSupportedFeatures())) + } + + /** + * 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 = { + testShouldFailRequestDuringBrokerMaxVersionLevelIncompatibility(defaultFinalizedFeatures()) + } + + /** + * 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 = { + testShouldFailRequestDuringBrokerMaxVersionLevelIncompatibility(Features.emptyFinalizedFeatures()) } /** @@ -379,7 +440,7 @@ class UpdateFeaturesTest extends BaseRequestTest { val update1 = new FeatureUpdate(targetFinalizedFeatures.get("feature_1").max(), false) val update2 = new FeatureUpdate(targetFinalizedFeatures.get("feature_2").max(), false) - val expected = new FeatureMetadata( + val expected = makeFeatureMetadata( targetFinalizedFeatures, versionBefore + 1, Features.supportedFeatures( @@ -426,7 +487,7 @@ class UpdateFeaturesTest extends BaseRequestTest { val update1 = new FeatureUpdate(targetFinalizedFeatures.get("feature_1").max(), false) val update2 = new FeatureUpdate(targetFinalizedFeatures.get("feature_2").max(), true) - val expected = new FeatureMetadata( + val expected = makeFeatureMetadata( targetFinalizedFeatures, versionBefore + 1, Features.supportedFeatures( @@ -491,7 +552,7 @@ class UpdateFeaturesTest extends BaseRequestTest { checkFeatures( adminClient, FeatureZNode(FeatureZNodeStatus.Enabled, expectedFeatures), - new FeatureMetadata(expectedFeatures, versionBefore + 1, initialSupportedFeatures)) + makeFeatureMetadata(expectedFeatures, versionBefore + 1, initialSupportedFeatures)) } /** @@ -551,6 +612,6 @@ class UpdateFeaturesTest extends BaseRequestTest { checkFeatures( adminClient, FeatureZNode(FeatureZNodeStatus.Enabled, expectedFeatures), - new FeatureMetadata(expectedFeatures, versionBefore + 1, initialSupportedFeatures)) + makeFeatureMetadata(expectedFeatures, versionBefore + 1, initialSupportedFeatures)) } } From 47297375a0df522a6c70923e2776ddc3dc7b81fd Mon Sep 17 00:00:00 2001 From: Kowshik Prakasam Date: Thu, 24 Sep 2020 14:49:27 -0700 Subject: [PATCH 21/41] Fix checkstyle issues for CI --- .../apache/kafka/common/requests/UpdateFeaturesRequest.java | 2 -- .../apache/kafka/clients/admin/KafkaAdminClientTest.java | 6 ++---- 2 files changed, 2 insertions(+), 6 deletions(-) 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 index 2fcea98eae06c..661b0d9f376ea 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesRequest.java @@ -95,7 +95,6 @@ public static boolean isDeleteRequest(UpdateFeaturesRequestData.FeatureUpdateKey } public static UpdateFeaturesRequestData create(Map featureUpdates) { - final Map> updateFutures = new HashMap<>(); final UpdateFeaturesRequestData.FeatureUpdateKeyCollection featureUpdatesRequestData = new UpdateFeaturesRequestData.FeatureUpdateKeyCollection(); for (Map.Entry entry : featureUpdates.entrySet()) { @@ -105,7 +104,6 @@ public static UpdateFeaturesRequestData create(Map featur throw new IllegalArgumentException("Provided feature can not be null or empty."); } - updateFutures.put(feature, new KafkaFutureImpl<>()); final UpdateFeaturesRequestData.FeatureUpdateKey requestItem = new UpdateFeaturesRequestData.FeatureUpdateKey(); requestItem.setFeature(feature); 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 d9993d9a2b669..3f68651447711 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 @@ -489,8 +489,7 @@ private static FeatureMetadata defaultFeatureMetadata() { Utils.mkMap(Utils.mkEntry("test_feature_1", new SupportedVersionRange((short) 1, (short) 5)))); } - private static Features - convertSupportedFeaturesMap(Map features) { + private static Features convertSupportedFeaturesMap(Map features) { Map featuresMap = new HashMap<>(); for (Map.Entry entry : features.entrySet()) { SupportedVersionRange versionRange = entry.getValue(); @@ -503,8 +502,7 @@ private static FeatureMetadata defaultFeatureMetadata() { return Features.supportedFeatures(featuresMap); } - private static Features - convertFinalizedFeaturesMap(Map features) { + private static Features convertFinalizedFeaturesMap(Map features) { Map featuresMap = new HashMap<>(); for (Map.Entry entry : features.entrySet()) { FinalizedVersionRange versionRange = entry.getValue(); From deaad42a8640d2322d9cc20ba0bd82d43ddfbc64 Mon Sep 17 00:00:00 2001 From: Kowshik Prakasam Date: Thu, 24 Sep 2020 15:09:45 -0700 Subject: [PATCH 22/41] Small improvements --- .../org/apache/kafka/clients/admin/Admin.java | 8 +- .../admin/DescribeFeaturesOptions.java | 2 +- .../kafka/clients/admin/FeatureMetadata.java | 21 ---- .../kafka/clients/admin/KafkaAdminClient.java | 27 ++++- .../clients/admin/SupportedVersionRange.java | 6 +- .../clients/admin/UpdateFeaturesResult.java | 2 +- .../requests/UpdateFeaturesResponse.java | 2 +- .../kafka/controller/KafkaController.scala | 98 ++++++++++--------- .../scala/kafka/server/BrokerFeatures.scala | 16 +-- .../kafka/server/BrokerFeaturesTest.scala | 12 +-- 10 files changed, 101 insertions(+), 93 deletions(-) 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 ebaa09de2b1f7..3ce8e22d12741 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 @@ -1328,8 +1328,8 @@ AlterUserScramCredentialsResult alterUserScramCredentials(List * 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 @@ -1339,13 +1339,13 @@ AlterUserScramCredentialsResult alterUserScramCredentials(List *
  • 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 + * 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 be less than 1.
  • + * the max version level to a value less than 1. *
*

* The following exceptions can be anticipated when calling {@code get()} on the futures 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 index 696976f1a43b2..cd8fc11ac0237 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/DescribeFeaturesOptions.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/DescribeFeaturesOptions.java @@ -19,7 +19,7 @@ import org.apache.kafka.common.annotation.InterfaceStability; /** - * Options for {@link AdminClient#describeFeatures(DescribeFeaturesOptions)} + * Options for {@link AdminClient#describeFeatures(DescribeFeaturesOptions)}. * * The API of this class is evolving. See {@link Admin} for details. */ 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 index bed388203aced..7cb9220ec6006 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/FeatureMetadata.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/FeatureMetadata.java @@ -22,9 +22,6 @@ import java.util.Map; import java.util.Objects; import java.util.Optional; -import org.apache.kafka.common.message.ApiVersionsResponseData.FinalizedFeatureKey; -import org.apache.kafka.common.message.ApiVersionsResponseData.SupportedFeatureKey; -import org.apache.kafka.common.requests.ApiVersionsResponse; /** * Encapsulates details about finalized as well as supported features. This is particularly useful @@ -46,24 +43,6 @@ public FeatureMetadata(final Map finalizedFeature this.supportedFeatures = new HashMap<>(supportedFeatures); } - public FeatureMetadata(ApiVersionsResponse response) { - this.supportedFeatures = new HashMap<>(); - for (SupportedFeatureKey key : response.data().supportedFeatures().valuesSet()) { - supportedFeatures.put(key.name(), new SupportedVersionRange(key.minVersion(), key.maxVersion())); - } - - this.finalizedFeatures = new HashMap<>(); - for (FinalizedFeatureKey key : response.data().finalizedFeatures().valuesSet()) { - finalizedFeatures.put(key.name(), new FinalizedVersionRange(key.minVersionLevel(), key.maxVersionLevel())); - } - - if (response.data().finalizedFeaturesEpoch() >= 0) { - this.finalizedFeaturesEpoch = Optional.of(response.data().finalizedFeaturesEpoch()); - } else { - this.finalizedFeaturesEpoch = Optional.empty(); - } - } - /** * 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 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 46ff921f32013..77e7d480a8c48 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; @@ -4351,6 +4353,27 @@ public DescribeFeaturesResult describeFeatures(final DescribeFeaturesOptions opt Call call = new Call( "describeFeatures", calcDeadlineMs(now, options.timeoutMs()), provider) { + private FeatureMetadata createFeatureMetadata(final ApiVersionsResponse response) { + final Map finalizedFeatures = new HashMap<>(); + for (FinalizedFeatureKey key : response.data().finalizedFeatures().valuesSet()) { + finalizedFeatures.put(key.name(), new FinalizedVersionRange(key.minVersionLevel(), key.maxVersionLevel())); + } + + Optional finalizedFeaturesEpoch; + if (response.data().finalizedFeaturesEpoch() >= 0) { + finalizedFeaturesEpoch = Optional.of(response.data().finalizedFeaturesEpoch()); + } else { + finalizedFeaturesEpoch = Optional.empty(); + } + + final Map supportedFeatures = new HashMap<>(); + for (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(); @@ -4360,7 +4383,7 @@ ApiVersionsRequest.Builder createRequest(int timeoutMs) { void handleResponse(AbstractResponse response) { final ApiVersionsResponse apiVersionsResponse = (ApiVersionsResponse) response; if (apiVersionsResponse.data.errorCode() == Errors.NONE.code()) { - future.complete(new FeatureMetadata(apiVersionsResponse)); + future.complete(createFeatureMetadata(apiVersionsResponse)); } else if (options.sendRequestToController() && apiVersionsResponse.data.errorCode() == Errors.NOT_CONTROLLER.code()) { handleNotControllerError(Errors.NOT_CONTROLLER); @@ -4427,7 +4450,7 @@ void handleResponse(AbstractResponse abstractResponse) { feature -> "The controller response did not contain a result for feature " + feature); break; case NOT_CONTROLLER: - handleNotControllerError(Errors.forCode(topLevelError.code())); + handleNotControllerError(topLevelError); break; default: for (Map.Entry> entry : updateFutures.entrySet()) { 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 index 586c37ed82302..8f8eeb35224e2 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/SupportedVersionRange.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/SupportedVersionRange.java @@ -39,8 +39,8 @@ public SupportedVersionRange(final short minVersion, final short maxVersion) { if (minVersion < 1 || maxVersion < 1 || maxVersion < minVersion) { throw new IllegalArgumentException( String.format( - "Expected minVersion >= 1, maxVersion >= 1 and maxVersion >= minVersion, but received" + - " minVersion: %d, maxVersion: %d", minVersion, maxVersion)); + "Expected minVersion >= 1, maxVersion >= 1 and maxVersion >= minVersion," + + " but received minVersion: %d, maxVersion: %d", minVersion, maxVersion)); } this.minVersion = minVersion; this.maxVersion = maxVersion; @@ -74,7 +74,7 @@ public int hashCode() { @Override public String toString() { - return String.format("FinalizedVersionRange{minVersion:%d, maxVersion:%d}", minVersion, maxVersion); + return String.format("SupportedVersionRange{minVersion:%d, maxVersion:%d}", minVersion, maxVersion); } } 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 index e4beacad6eb2a..d0d27aedb6298 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/UpdateFeaturesResult.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/UpdateFeaturesResult.java @@ -28,7 +28,7 @@ public class UpdateFeaturesResult { private final Map> futures; /** - * @param futures a map from feature names to future, which can be used to check the status of + * @param futures a map from feature name to future, which can be used to check the status of * individual feature updates. */ public UpdateFeaturesResult(final Map> futures) { 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 index 93d448e602247..669d393912f93 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesResponse.java @@ -98,7 +98,7 @@ public static UpdateFeaturesResponse createWithErrors(ApiError topLevelError, Ma .setErrorMessage(error.message()); results.add(result); } - UpdateFeaturesResponseData responseData = new UpdateFeaturesResponseData() + final UpdateFeaturesResponseData responseData = new UpdateFeaturesResponseData() .setThrottleTimeMs(throttleTimeMs) .setErrorCode(topLevelError.error().code()) .setErrorMessage(topLevelError.message()) diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index c86b453cd3e31..15772a969bd63 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -300,16 +300,16 @@ class KafkaController(val config: KafkaConfig, * 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 - * {@link SupportedVersionRange}). It refers to a feature that a particular broker advertises + * 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 {@link FinalizedVersionRange}). Whenever the feature versioning system (KIP-584) is + * 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 @@ -376,7 +376,7 @@ class KafkaController(val config: KafkaConfig, * will switch the FeatureZNode status to disabled with empty features. */ private def enableFeatureVersioning(): Unit = { - val defaultFinalizedFeatures = brokerFeatures.getDefaultFinalizedFeatures + val defaultFinalizedFeatures = brokerFeatures.defaultFinalizedFeatures val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path) if (version == ZkVersion.UnknownVersion) { val newVersion = createFeatureZNode(new FeatureZNode(FeatureZNodeStatus.Enabled, defaultFinalizedFeatures)) @@ -1556,22 +1556,25 @@ class KafkaController(val config: KafkaConfig, /** * Partitions the provided map of brokers and epochs into 2 new maps: - * - The first map contains brokers whose features were found to be compatible with the existing - * finalized features. - * - The second map contains brokers whose features were found to be incompatible with the existing - * finalized features. + * - 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 + * @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.isFeatureVersioningEnabled || - !featureCache.get.exists( - latestFinalizedFeatures => - BrokerFeatures.hasIncompatibleFeatures(broker.features, latestFinalizedFeatures.features)) + !featureCache.get.exists( + latestFinalizedFeatures => + BrokerFeatures.hasIncompatibleFeatures(broker.features, latestFinalizedFeatures.features)) } } @@ -1893,7 +1896,7 @@ 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. + * 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) * @@ -1904,35 +1907,36 @@ class KafkaController(val config: KafkaConfig, throw new IllegalArgumentException(s"Provided feature update can not be meant to delete the feature: $update") } - val incompatibilityError = "Could not apply finalized feature update because" + - " brokers were found to have incompatible versions for the feature." - - if (brokerFeatures.supportedFeatures.get(update.feature()) == null) { - Right(new ApiError(Errors.INVALID_REQUEST, incompatibilityError)) + val defaultMinVersionLevelOpt = brokerFeatures.defaultMinVersionLevel(update.feature) + if (defaultMinVersionLevelOpt.isEmpty) { + Right(new ApiError(Errors.INVALID_REQUEST, + "Could not apply finalized feature update because the provided feature" + + " is not supported.")) } else { - val defaultMinVersionLevel = brokerFeatures.defaultMinVersionLevel(update.feature) var newVersionRange: FinalizedVersionRange = null try { - newVersionRange = new FinalizedVersionRange(defaultMinVersionLevel, update.maxVersionLevel) + newVersionRange = new FinalizedVersionRange(defaultMinVersionLevelOpt.get, update.maxVersionLevel) } catch { // Ignoring because it means the provided maxVersionLevel is invalid. case _: IllegalArgumentException => {} } 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" default minVersionLevel:$defaultMinVersionLevel.")) + Right(new ApiError(Errors.INVALID_REQUEST, + "Could not apply finalized feature update because the provided" + + s" maxVersionLevel:${update.maxVersionLevel} is lower than the" + + s" default minVersionLevel:${defaultMinVersionLevelOpt.get}.")) } else { + val newFinalizedFeature = + Features.finalizedFeatures(Utils.mkMap(Utils.mkEntry(update.feature, newVersionRange))) val numIncompatibleBrokers = controllerContext.liveOrShuttingDownBrokers.count(broker => { - val singleFinalizedFeature = - Features.finalizedFeatures(Utils.mkMap(Utils.mkEntry(update.feature, newVersionRange))) - BrokerFeatures.hasIncompatibleFeatures(broker.features, singleFinalizedFeature) + BrokerFeatures.hasIncompatibleFeatures(broker.features, newFinalizedFeature) }) if (numIncompatibleBrokers == 0) { Left(newVersionRange) } else { - Right(new ApiError(Errors.INVALID_REQUEST, incompatibilityError)) + Right(new ApiError(Errors.INVALID_REQUEST, + "Could not apply finalized feature update because" + + " brokers were found to have incompatible versions for the feature.")) } } } @@ -2061,28 +2065,26 @@ class KafkaController(val config: KafkaConfig, // 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. - if (existingFeatures.equals(targetFeatures)) { - callback(Right(errors)) - } else { - try { + // of the existing finalized features in ZK. + try { + if (!existingFeatures.equals(targetFeatures)) { val newNode = new FeatureZNode(FeatureZNodeStatus.Enabled, Features.finalizedFeatures(targetFeatures.asJava)) val newVersion = zkClient.updateFeatureZNode(newNode) featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs) - } 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 => - errors.foreach { case (feature, apiError) => - if (apiError.error() == Errors.NONE) { - errors(feature) = new ApiError(Errors.FEATURE_UPDATE_FAILED, - Errors.FEATURE_UPDATE_FAILED.message() + " Error: " + e) - } - } - } finally { - callback(Right(errors)) } + } 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 => + errors.foreach { case (feature, apiError) => + if (apiError.error() == Errors.NONE) { + errors(feature) = new ApiError(Errors.FEATURE_UPDATE_FAILED, + Errors.FEATURE_UPDATE_FAILED.message() + " Error: " + e) + } + } + } finally { + callback(Right(errors)) } } diff --git a/core/src/main/scala/kafka/server/BrokerFeatures.scala b/core/src/main/scala/kafka/server/BrokerFeatures.scala index e000e92cb43ee..652026e096c32 100644 --- a/core/src/main/scala/kafka/server/BrokerFeatures.scala +++ b/core/src/main/scala/kafka/server/BrokerFeatures.scala @@ -57,7 +57,7 @@ import scala.jdk.CollectionConverters._ * NOTE: The difference between the values in this map and the minimum version value for a * broker's supported feature is the following: Version levels below the values specified in this * map are considered deprecated by the controller, whereas version levels below the minimum - * version value for a supported feature are considered unknown/unsupported. + * version value for a supported feature are considered unknown/unsupported. */ class BrokerFeatures private (@volatile var supportedFeatures: Features[SupportedVersionRange], @volatile var defaultFeatureMinVersionLevels: Map[String, Short]) { @@ -77,10 +77,14 @@ class BrokerFeatures private (@volatile var supportedFeatures: Features[Supporte * @param feature the name of the feature * * @return the default minimum version level for the supported feature if its defined. - * otherwise, returns the minimum version of the supported feature. + * otherwise, returns the minimum version of the supported feature (if the feature + * exists) or none. */ - def defaultMinVersionLevel(feature: String): Short = { - defaultFeatureMinVersionLevels.getOrElse(feature, supportedFeatures.get(feature).min()) + def defaultMinVersionLevel(feature: String): Option[Short] = { + defaultFeatureMinVersionLevels.get(feature).map(Some(_)).getOrElse { + val versionRange = supportedFeatures.get(feature) + if (versionRange == null) Option.empty else Some(versionRange.min()) + } } // For testing only. @@ -94,11 +98,11 @@ class BrokerFeatures private (@volatile var supportedFeatures: Features[Supporte * Returns the default finalized features that a new Kafka cluster with IBP config >= KAFKA_2_7_IV0 * needs to be bootstrapped with. */ - def getDefaultFinalizedFeatures: Features[FinalizedVersionRange] = { + def defaultFinalizedFeatures: Features[FinalizedVersionRange] = { Features.finalizedFeatures( supportedFeatures.features.asScala.map { case(name, versionRange) => ( - name, new FinalizedVersionRange(defaultMinVersionLevel(name), versionRange.max)) + name, new FinalizedVersionRange(defaultMinVersionLevel(name).get, versionRange.max)) }.asJava) } diff --git a/core/src/test/scala/unit/kafka/server/BrokerFeaturesTest.scala b/core/src/test/scala/unit/kafka/server/BrokerFeaturesTest.scala index 8deabf85c55a8..5f564af5c55fe 100644 --- a/core/src/test/scala/unit/kafka/server/BrokerFeaturesTest.scala +++ b/core/src/test/scala/unit/kafka/server/BrokerFeaturesTest.scala @@ -169,7 +169,7 @@ class BrokerFeaturesTest { "test_feature_1" -> new FinalizedVersionRange(2, 4), "test_feature_2" -> new FinalizedVersionRange(3, 3), "test_feature_3" -> new FinalizedVersionRange(3, 7)) - assertEquals(Features.finalizedFeatures(expectedFeatures.asJava), brokerFeatures.getDefaultFinalizedFeatures) + assertEquals(Features.finalizedFeatures(expectedFeatures.asJava), brokerFeatures.defaultFinalizedFeatures) } @Test @@ -183,10 +183,10 @@ class BrokerFeaturesTest { val defaultMinVersionLevels = Map[String, Short]("test_feature_1" -> 2) brokerFeatures.setDefaultMinVersionLevels(defaultMinVersionLevels) - assertEquals(2, brokerFeatures.defaultMinVersionLevel("test_feature_1")) - assertEquals(1, brokerFeatures.defaultMinVersionLevel("test_feature_2")) - assertThrows( - classOf[NullPointerException], - () => brokerFeatures.defaultMinVersionLevel("test_feature_3")) + assertTrue(brokerFeatures.defaultMinVersionLevel("test_feature_1").isDefined) + assertEquals(2, brokerFeatures.defaultMinVersionLevel("test_feature_1").get) + assertTrue(brokerFeatures.defaultMinVersionLevel("test_feature_2").isDefined) + assertEquals(1, brokerFeatures.defaultMinVersionLevel("test_feature_2").get) + assertTrue(brokerFeatures.defaultMinVersionLevel("test_nonexistent_feature").isEmpty) } } From 21491b233eef172ef29642233ddc15848d32d3a1 Mon Sep 17 00:00:00 2001 From: Kowshik Prakasam Date: Thu, 24 Sep 2020 17:31:44 -0700 Subject: [PATCH 23/41] Rebase on latest AK trunk --- .../main/java/org/apache/kafka/common/protocol/ApiKeys.java | 2 +- .../main/resources/common/message/UpdateFeaturesRequest.json | 2 +- .../main/resources/common/message/UpdateFeaturesResponse.json | 2 +- core/src/main/scala/kafka/controller/KafkaController.scala | 2 +- .../unit/kafka/controller/ControllerIntegrationTest.scala | 3 --- 5 files changed, 4 insertions(+), 7 deletions(-) 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 183ea59da4d47..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 @@ -251,7 +251,7 @@ public Struct parseResponse(short version, ByteBuffer buffer) { DescribeQuorumRequestData.SCHEMAS, DescribeQuorumResponseData.SCHEMAS), ALTER_ISR(56, "AlterIsr", AlterIsrRequestData.SCHEMAS, AlterIsrResponseData.SCHEMAS), UPDATE_FEATURES(57, "UpdateFeatures", - UpdateFeaturesRequestData.SCHEMAS, UpdateFeaturesResponseData.SCHEMAS); + 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/resources/common/message/UpdateFeaturesRequest.json b/clients/src/main/resources/common/message/UpdateFeaturesRequest.json index 82ebe0e9f45e5..3ce8ee8c26602 100644 --- a/clients/src/main/resources/common/message/UpdateFeaturesRequest.json +++ b/clients/src/main/resources/common/message/UpdateFeaturesRequest.json @@ -14,7 +14,7 @@ // limitations under the License. { - "apiKey": 56, + "apiKey": 57, "type": "request", "name": "UpdateFeaturesRequest", "validVersions": "0", diff --git a/clients/src/main/resources/common/message/UpdateFeaturesResponse.json b/clients/src/main/resources/common/message/UpdateFeaturesResponse.json index 142c36dee5c09..615f6177cfbee 100644 --- a/clients/src/main/resources/common/message/UpdateFeaturesResponse.json +++ b/clients/src/main/resources/common/message/UpdateFeaturesResponse.json @@ -14,7 +14,7 @@ // limitations under the License. { - "apiKey": 56, + "apiKey": 57, "type": "response", "name": "UpdateFeaturesResponse", "validVersions": "0", diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 15772a969bd63..639eedcf23ac9 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -22,7 +22,7 @@ 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} diff --git a/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala b/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala index e2c3a00700e41..a59a20c1a89d4 100644 --- a/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala @@ -26,9 +26,6 @@ import kafka.metrics.KafkaYammerMetrics import kafka.server.{KafkaConfig, KafkaServer} import kafka.utils.{LogCaptureAppender, TestUtils} import kafka.zk._ -import org.junit.{After, Before, Test} -import org.junit.Assert.{assertEquals, assertTrue} -import org.apache.kafka.common.{ElectionType, TopicPartition} import org.apache.kafka.common.errors.{ControllerMovedException, StaleBrokerEpochException} import org.apache.kafka.common.metrics.KafkaMetric import org.apache.kafka.common.protocol.Errors From 8ec01e7d089412a190d95ab329256e466c41376e Mon Sep 17 00:00:00 2001 From: Kowshik Prakasam Date: Thu, 24 Sep 2020 23:56:29 -0700 Subject: [PATCH 24/41] Fix checkstyle issue --- .../org/apache/kafka/common/requests/UpdateFeaturesRequest.java | 2 -- 1 file changed, 2 deletions(-) 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 index 661b0d9f376ea..cfc5ee2c098f6 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesRequest.java @@ -17,9 +17,7 @@ package org.apache.kafka.common.requests; import java.nio.ByteBuffer; -import java.util.HashMap; import java.util.Map; -import org.apache.kafka.common.internals.KafkaFutureImpl; import org.apache.kafka.common.message.UpdateFeaturesRequestData.FeatureUpdateKey; import org.apache.kafka.common.message.UpdateFeaturesResponseData; import org.apache.kafka.common.message.UpdateFeaturesRequestData; From 436d816faff1bdabd7dd4cbde26e0f0303f7f484 Mon Sep 17 00:00:00 2001 From: Kowshik Prakasam Date: Fri, 25 Sep 2020 01:55:21 -0700 Subject: [PATCH 25/41] Remove unused code --- .../common/requests/ApiVersionsResponse.java | 25 ------------------- 1 file changed, 25 deletions(-) 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 5d9517635da12..605cbd8e5ac6f 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 @@ -16,7 +16,6 @@ */ package org.apache.kafka.common.requests; -import java.util.HashMap; import org.apache.kafka.common.feature.Features; import org.apache.kafka.common.feature.FinalizedVersionRange; import org.apache.kafka.common.feature.SupportedVersionRange; @@ -72,30 +71,6 @@ public ApiVersionsResponseData data() { return data; } - public Features supportedFeatures() { - final Map features = new HashMap<>(); - - for (SupportedFeatureKey key : data.supportedFeatures().valuesSet()) { - features.put(key.name(), new SupportedVersionRange(key.minVersion(), key.maxVersion())); - } - - return Features.supportedFeatures(features); - } - - public Features finalizedFeatures() { - final Map features = new HashMap<>(); - - for (FinalizedFeatureKey key : data.finalizedFeatures().valuesSet()) { - features.put(key.name(), new FinalizedVersionRange(key.minVersionLevel(), key.maxVersionLevel())); - } - - return Features.finalizedFeatures(features); - } - - public int finalizedFeaturesEpoch() { - return data.finalizedFeaturesEpoch(); - } - @Override protected Struct toStruct(short version) { return this.data.toStruct(version); From 1150a1fd9302e2e311ad3118bafab125a88da2d1 Mon Sep 17 00:00:00 2001 From: Kowshik Prakasam Date: Fri, 25 Sep 2020 17:50:38 -0700 Subject: [PATCH 26/41] Reinstante timeoutMs & change FinalizedFeaturesEpoch to long data type --- .../kafka/clients/admin/FeatureMetadata.java | 6 +++--- .../kafka/clients/admin/KafkaAdminClient.java | 8 +++----- .../common/requests/ApiVersionsResponse.java | 8 ++++---- .../requests/UpdateFeaturesRequest.java | 6 ++++-- .../common/message/ApiVersionsResponse.json | 2 +- .../common/message/UpdateFeaturesRequest.json | 2 ++ .../clients/admin/KafkaAdminClientTest.java | 20 +++++++++++++------ .../kafka/server/FinalizedFeatureCache.scala | 10 +++++----- .../kafka/server/UpdateFeaturesTest.scala | 2 +- 9 files changed, 37 insertions(+), 27 deletions(-) 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 index 7cb9220ec6006..c7be80f1a1c37 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/FeatureMetadata.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/FeatureMetadata.java @@ -31,12 +31,12 @@ public class FeatureMetadata { private final Map finalizedFeatures; - private final Optional finalizedFeaturesEpoch; + private final Optional finalizedFeaturesEpoch; private final Map supportedFeatures; public FeatureMetadata(final Map finalizedFeatures, - final Optional finalizedFeaturesEpoch, + final Optional finalizedFeaturesEpoch, final Map supportedFeatures) { this.finalizedFeatures = new HashMap<>(finalizedFeatures); this.finalizedFeaturesEpoch = finalizedFeaturesEpoch; @@ -56,7 +56,7 @@ public Map finalizedFeatures() { * The epoch for the finalized features. * If the returned value is empty, it means the finalized features are absent/unavailable. */ - public Optional finalizedFeaturesEpoch() { + public Optional finalizedFeaturesEpoch() { return finalizedFeaturesEpoch; } 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 77e7d480a8c48..04583135c332a 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 @@ -145,7 +145,6 @@ 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; @@ -4359,8 +4358,8 @@ private FeatureMetadata createFeatureMetadata(final ApiVersionsResponse response finalizedFeatures.put(key.name(), new FinalizedVersionRange(key.minVersionLevel(), key.maxVersionLevel())); } - Optional finalizedFeaturesEpoch; - if (response.data().finalizedFeaturesEpoch() >= 0) { + Optional finalizedFeaturesEpoch; + if (response.data().finalizedFeaturesEpoch() >= 0L) { finalizedFeaturesEpoch = Optional.of(response.data().finalizedFeaturesEpoch()); } else { finalizedFeaturesEpoch = Optional.empty(); @@ -4410,7 +4409,6 @@ public UpdateFeaturesResult updateFeatures( } Objects.requireNonNull(options, "UpdateFeaturesOptions can not be null"); - final UpdateFeaturesRequestData request = UpdateFeaturesRequest.create(featureUpdates); final Map> updateFutures = new HashMap<>(); for (Map.Entry entry : featureUpdates.entrySet()) { updateFutures.put(entry.getKey(), new KafkaFutureImpl<>()); @@ -4421,7 +4419,7 @@ public UpdateFeaturesResult updateFeatures( @Override UpdateFeaturesRequest.Builder createRequest(int timeoutMs) { - return new UpdateFeaturesRequest.Builder(request); + return new UpdateFeaturesRequest.Builder(UpdateFeaturesRequest.create(featureUpdates, timeoutMs)); } @Override 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 605cbd8e5ac6f..61be347f1144e 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 = -1; public static final ApiVersionsResponse DEFAULT_API_VERSIONS_RESPONSE = createApiVersionsResponse( @@ -144,7 +144,7 @@ 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 new ApiVersionsResponse(createApiVersionsResponseData( DEFAULT_API_VERSIONS_RESPONSE.throttleTimeMs(), @@ -174,7 +174,7 @@ private static ApiVersionsResponse createApiVersionsResponse( final byte minMagic, final Features latestSupportedFeatures, final Features finalizedFeatures, - final int finalizedFeaturesEpoch + final long finalizedFeaturesEpoch ) { ApiVersionsResponseKeyCollection apiKeys = new ApiVersionsResponseKeyCollection(); for (ApiKeys apiKey : ApiKeys.enabledApis()) { @@ -202,7 +202,7 @@ public static ApiVersionsResponseData createApiVersionsResponseData( final ApiVersionsResponseKeyCollection apiKeys, final Features latestSupportedFeatures, final Features finalizedFeatures, - final int finalizedFeaturesEpoch + final long finalizedFeaturesEpoch ) { ApiVersionsResponseData data = new ApiVersionsResponseData(); data.setThrottleTimeMs(throttleTimeMs); 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 index cfc5ee2c098f6..ace2a5d471b43 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesRequest.java @@ -92,7 +92,7 @@ public static boolean isDeleteRequest(UpdateFeaturesRequestData.FeatureUpdateKey return update.maxVersionLevel() < 1 && update.allowDowngrade(); } - public static UpdateFeaturesRequestData create(Map featureUpdates) { + public static UpdateFeaturesRequestData create(Map featureUpdates, int timeoutMs) { final UpdateFeaturesRequestData.FeatureUpdateKeyCollection featureUpdatesRequestData = new UpdateFeaturesRequestData.FeatureUpdateKeyCollection(); for (Map.Entry entry : featureUpdates.entrySet()) { @@ -109,6 +109,8 @@ public static UpdateFeaturesRequestData create(Map featur requestItem.setAllowDowngrade(update.allowDowngrade()); featureUpdatesRequestData.add(requestItem); } - return new UpdateFeaturesRequestData().setFeatureUpdates(featureUpdatesRequestData); + return new UpdateFeaturesRequestData() + .setTimeoutMs(timeoutMs) + .setFeatureUpdates(featureUpdatesRequestData); } } diff --git a/clients/src/main/resources/common/message/ApiVersionsResponse.json b/clients/src/main/resources/common/message/ApiVersionsResponse.json index 834564c146e26..104a0388b3d5d 100644 --- a/clients/src/main/resources/common/message/ApiVersionsResponse.json +++ b/clients/src/main/resources/common/message/ApiVersionsResponse.json @@ -55,7 +55,7 @@ "about": "The maximum supported version for the feature." } ] }, - {"name": "FinalizedFeaturesEpoch", "type": "int32", "versions": "3+", + {"name": "FinalizedFeaturesEpoch", "type": "int64", "versions": "3+", "tag": 1, "taggedVersions": "3+", "default": "-1", "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", diff --git a/clients/src/main/resources/common/message/UpdateFeaturesRequest.json b/clients/src/main/resources/common/message/UpdateFeaturesRequest.json index 3ce8ee8c26602..3c90d52a2b704 100644 --- a/clients/src/main/resources/common/message/UpdateFeaturesRequest.json +++ b/clients/src/main/resources/common/message/UpdateFeaturesRequest.json @@ -20,6 +20,8 @@ "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, 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 3f68651447711..268994f1624f6 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 @@ -485,7 +485,7 @@ private static DescribeGroupsResponseData prepareDescribeGroupsResponseData(Stri private static FeatureMetadata defaultFeatureMetadata() { return new FeatureMetadata( Utils.mkMap(Utils.mkEntry("test_feature_1", new FinalizedVersionRange((short) 2, (short) 3))), - Optional.of(1), + Optional.of(1L), Utils.mkMap(Utils.mkEntry("test_feature_1", new SupportedVersionRange((short) 1, (short) 5)))); } @@ -4072,11 +4072,19 @@ public void testUpdateFeaturesShouldFailRequestForNullUpdateFeaturesOptions() { @Test public void testUpdateFeaturesShouldFailRequestForInvalidFeatureName() { try (final AdminClientUnitTestEnv env = mockClientEnv()) { - assertThrows( - IllegalArgumentException.class, - () -> env.adminClient().updateFeatures( - Utils.mkMap(Utils.mkEntry("", new FeatureUpdate((short) 2, false))), - new UpdateFeaturesOptions())); + 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()); } } diff --git a/core/src/main/scala/kafka/server/FinalizedFeatureCache.scala b/core/src/main/scala/kafka/server/FinalizedFeatureCache.scala index c4e8dca34eba4..b47cda3617a80 100644 --- a/core/src/main/scala/kafka/server/FinalizedFeatureCache.scala +++ b/core/src/main/scala/kafka/server/FinalizedFeatureCache.scala @@ -28,9 +28,9 @@ class FeatureCacheUpdateException(message: String) extends RuntimeException(mess } // 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) } } @@ -68,8 +68,8 @@ class FinalizedFeatureCache(private val brokerFeatures: BrokerFeatures) extends * @throws TimeoutException if the cache's epoch has not reached at least * minExpectedEpoch within timeoutMs. */ - def waitUntilEpochOrThrow(minExpectedEpoch: Int, timeoutMs: Long): Unit = { - if(minExpectedEpoch < 0) { + def waitUntilEpochOrThrow(minExpectedEpoch: Long, timeoutMs: Long): Unit = { + if(minExpectedEpoch < 0L) { throw new IllegalArgumentException( s"Expected minExpectedEpoch >= 0, but $minExpectedEpoch was provided.") } @@ -102,7 +102,7 @@ class FinalizedFeatureCache(private val brokerFeatures: BrokerFeatures) extends * 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("") if (featuresAndEpoch.isDefined && featuresAndEpoch.get.epoch > latest.epoch) { diff --git a/core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala b/core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala index c29fa5df44ec4..3929dbdae69a3 100644 --- a/core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala +++ b/core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala @@ -108,7 +108,7 @@ class UpdateFeaturesTest extends BaseRequestTest { } private def makeFeatureMetadata(finalized: Features[FinalizedVersionRange], - epoch: Integer, + epoch: Long, supported: Features[SupportedVersionRange]): FeatureMetadata = { new FeatureMetadata( finalized.features().asScala.map { From 45372f3006d668835d96d5e0fb41a04225bc3b4a Mon Sep 17 00:00:00 2001 From: Kowshik Prakasam Date: Sat, 26 Sep 2020 18:10:44 -0700 Subject: [PATCH 27/41] Minor improvements --- .../admin/DescribeFeaturesOptions.java | 9 +++---- .../kafka/clients/admin/KafkaAdminClient.java | 13 +++++----- .../requests/UpdateFeaturesRequest.java | 8 +++--- .../requests/UpdateFeaturesResponse.java | 2 +- .../common/message/UpdateFeaturesRequest.json | 2 +- .../clients/admin/KafkaAdminClientTest.java | 26 +++++-------------- 6 files changed, 24 insertions(+), 36 deletions(-) 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 index cd8fc11ac0237..4a37956aab98c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/DescribeFeaturesOptions.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/DescribeFeaturesOptions.java @@ -28,15 +28,14 @@ public class DescribeFeaturesOptions extends AbstractOptions finalizedFeatures = new HashMap<>(); - for (FinalizedFeatureKey key : response.data().finalizedFeatures().valuesSet()) { + for (final FinalizedFeatureKey key : response.data().finalizedFeatures().valuesSet()) { finalizedFeatures.put(key.name(), new FinalizedVersionRange(key.minVersionLevel(), key.maxVersionLevel())); } @@ -4366,7 +4366,7 @@ private FeatureMetadata createFeatureMetadata(final ApiVersionsResponse response } final Map supportedFeatures = new HashMap<>(); - for (SupportedFeatureKey key : response.data().supportedFeatures().valuesSet()) { + for (final SupportedFeatureKey key : response.data().supportedFeatures().valuesSet()) { supportedFeatures.put(key.name(), new SupportedVersionRange(key.minVersion(), key.maxVersion())); } @@ -4404,13 +4404,12 @@ void handleFailure(Throwable throwable) { @Override public UpdateFeaturesResult updateFeatures( final Map featureUpdates, final UpdateFeaturesOptions options) { - if (featureUpdates == null || featureUpdates.isEmpty()) { + if (featureUpdates.isEmpty()) { throw new IllegalArgumentException("Feature updates can not be null or empty."); } - Objects.requireNonNull(options, "UpdateFeaturesOptions can not be null"); final Map> updateFutures = new HashMap<>(); - for (Map.Entry entry : featureUpdates.entrySet()) { + for (final Map.Entry entry : featureUpdates.entrySet()) { updateFutures.put(entry.getKey(), new KafkaFutureImpl<>()); } final long now = time.milliseconds(); @@ -4430,7 +4429,7 @@ void handleResponse(AbstractResponse abstractResponse) { Errors topLevelError = Errors.forCode(response.data().errorCode()); switch (topLevelError) { case NONE: - for (UpdatableFeatureResult result : response.data().results()) { + 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()); @@ -4451,7 +4450,7 @@ void handleResponse(AbstractResponse abstractResponse) { handleNotControllerError(topLevelError); break; default: - for (Map.Entry> entry : updateFutures.entrySet()) { + for (final Map.Entry> entry : updateFutures.entrySet()) { entry.getValue().completeExceptionally(topLevelError.exception()); } break; 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 index ace2a5d471b43..008a9f37ffa09 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesRequest.java @@ -61,7 +61,7 @@ public UpdateFeaturesRequest(Struct struct, short version) { } @Override - public AbstractResponse getErrorResponse(int throttleTimeMsIgnored, Throwable e) { + 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()) { @@ -71,8 +71,10 @@ public AbstractResponse getErrorResponse(int throttleTimeMsIgnored, Throwable e) .setErrorMessage(apiError.message()); results.add(result); } - return new UpdateFeaturesResponse(new UpdateFeaturesResponseData().setResults(results)); - } + final UpdateFeaturesResponseData responseData = new UpdateFeaturesResponseData() + .setThrottleTimeMs(throttleTimeMs) + .setResults(results); + return new UpdateFeaturesResponse(responseData); } @Override protected Struct toStruct() { 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 index 669d393912f93..5754f13595c8b 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesResponse.java @@ -89,7 +89,7 @@ public static UpdateFeaturesResponse parse(ByteBuffer buffer, short version) { public static UpdateFeaturesResponse createWithErrors(ApiError topLevelError, Map updateErrors, int throttleTimeMs) { final UpdatableFeatureResultCollection results = new UpdatableFeatureResultCollection(); - for (Map.Entry updateError : updateErrors.entrySet()) { + for (final Map.Entry updateError : updateErrors.entrySet()) { final String feature = updateError.getKey(); final ApiError error = updateError.getValue(); final UpdatableFeatureResult result = new UpdatableFeatureResult(); diff --git a/clients/src/main/resources/common/message/UpdateFeaturesRequest.json b/clients/src/main/resources/common/message/UpdateFeaturesRequest.json index 3c90d52a2b704..ab882dff1c754 100644 --- a/clients/src/main/resources/common/message/UpdateFeaturesRequest.json +++ b/clients/src/main/resources/common/message/UpdateFeaturesRequest.json @@ -29,7 +29,7 @@ {"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 downgraded request will fail if the new maximum version level is a value that's not lower than the existing maximum finalized version level."} + "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/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java index 268994f1624f6..f79fed431da0b 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 @@ -490,9 +490,9 @@ private static FeatureMetadata defaultFeatureMetadata() { } private static Features convertSupportedFeaturesMap(Map features) { - Map featuresMap = new HashMap<>(); - for (Map.Entry entry : features.entrySet()) { - SupportedVersionRange versionRange = entry.getValue(); + 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( @@ -503,9 +503,9 @@ private static Features c } private static Features convertFinalizedFeaturesMap(Map features) { - Map featuresMap = new HashMap<>(); - for (Map.Entry entry : features.entrySet()) { - FinalizedVersionRange versionRange = entry.getValue(); + 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( @@ -3962,7 +3962,7 @@ private void testUpdateFeatures(Map featureUpdates, final Map> futures = env.adminClient().updateFeatures( featureUpdates, new UpdateFeaturesOptions().timeoutMs(10000)).values(); - for (Map.Entry> entry : futures.entrySet()) { + for (final Map.Entry> entry : futures.entrySet()) { final KafkaFuture future = entry.getValue(); final ApiError error = featureUpdateErrors.get(entry.getKey()); if (topLevelError.error() == Errors.NONE) { @@ -4050,9 +4050,6 @@ public void testUpdateFeaturesHandleNotControllerException() throws Exception { @Test public void testUpdateFeaturesShouldFailRequestForEmptyUpdates() { try (final AdminClientUnitTestEnv env = mockClientEnv()) { - assertThrows( - IllegalArgumentException.class, - () -> env.adminClient().updateFeatures(null, new UpdateFeaturesOptions())); assertThrows( IllegalArgumentException.class, () -> env.adminClient().updateFeatures( @@ -4060,15 +4057,6 @@ public void testUpdateFeaturesShouldFailRequestForEmptyUpdates() { } } - @Test - public void testUpdateFeaturesShouldFailRequestForNullUpdateFeaturesOptions() { - try (final AdminClientUnitTestEnv env = mockClientEnv()) { - assertThrows( - NullPointerException.class, - () -> env.adminClient().updateFeatures(makeTestFeatureUpdates(), null)); - } - } - @Test public void testUpdateFeaturesShouldFailRequestForInvalidFeatureName() { try (final AdminClientUnitTestEnv env = mockClientEnv()) { From 389b7aaaa60c53dd2e079fc60418187f22524e8a Mon Sep 17 00:00:00 2001 From: Kowshik Prakasam Date: Sat, 26 Sep 2020 21:03:28 -0700 Subject: [PATCH 28/41] Cosmetics --- .../org/apache/kafka/clients/admin/Admin.java | 6 ++-- .../clients/admin/KafkaAdminClientTest.java | 5 ++- .../kafka/controller/KafkaController.scala | 17 +++++----- .../scala/kafka/server/BrokerFeatures.scala | 31 +++++++++---------- 4 files changed, 32 insertions(+), 27 deletions(-) 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 3ce8e22d12741..a61c6ef1bdef5 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 @@ -1360,8 +1360,10 @@ AlterUserScramCredentialsResult alterUserScramCredentials(List *

  • {@link FeatureUpdateFailedException} - * If the updates could not be applied on the controller, despite the request being valid. - * This may be a temporary problem.
  • + * 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. 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 f79fed431da0b..386ab57367589 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 @@ -525,7 +525,10 @@ private static ApiVersionsResponse prepareApiVersionsResponseForDescribeFeatures convertFinalizedFeaturesMap(defaultFeatureMetadata().finalizedFeatures()), defaultFeatureMetadata().finalizedFeaturesEpoch().get())); } - return new ApiVersionsResponse(new ApiVersionsResponseData().setErrorCode(error.code())); + return new ApiVersionsResponse( + new ApiVersionsResponseData() + .setThrottleTimeMs(ApiVersionsResponse.DEFAULT_API_VERSIONS_RESPONSE.throttleTimeMs()) + .setErrorCode(error.code())); } /** diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 639eedcf23ac9..742a4f9be9dc4 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -1917,8 +1917,10 @@ class KafkaController(val config: KafkaConfig, try { newVersionRange = new FinalizedVersionRange(defaultMinVersionLevelOpt.get, update.maxVersionLevel) } catch { - // Ignoring because it means the provided maxVersionLevel is invalid. - case _: IllegalArgumentException => {} + 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, @@ -2034,13 +2036,14 @@ class KafkaController(val config: KafkaConfig, val existingFeatures = featureCache.get .map(featuresAndEpoch => featuresAndEpoch.features.features().asScala) .getOrElse(Map[String, FinalizedVersionRange]()) - // Map of feature to FinalizedVersionRange. This contains the target features to be eventually - // written to FeatureZNode. + // 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 - // Map of feature to error. + // 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: + // 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 ApiError(Errors.NONE). // - If the FeatureUpdate is an add or update request, then the targetFeatures map is updated @@ -2080,7 +2083,7 @@ class KafkaController(val config: KafkaConfig, errors.foreach { case (feature, apiError) => if (apiError.error() == Errors.NONE) { errors(feature) = new ApiError(Errors.FEATURE_UPDATE_FAILED, - Errors.FEATURE_UPDATE_FAILED.message() + " Error: " + e) + Errors.FEATURE_UPDATE_FAILED.message() + " Error: " + e) } } } finally { diff --git a/core/src/main/scala/kafka/server/BrokerFeatures.scala b/core/src/main/scala/kafka/server/BrokerFeatures.scala index 652026e096c32..72924bc805afa 100644 --- a/core/src/main/scala/kafka/server/BrokerFeatures.scala +++ b/core/src/main/scala/kafka/server/BrokerFeatures.scala @@ -152,10 +152,10 @@ object BrokerFeatures extends Logging { private def isIncompatibleDefaultMinVersionLevel(feature: String, versionLevels: FinalizedVersionRange, - defaultFeatureMinVersionLevels: Option[Map[String, Short]]): Boolean = { - defaultFeatureMinVersionLevels.exists(defaults => - defaults.get(feature).exists(defaultMinVersionLevel => - defaultMinVersionLevel > versionLevels.max())) + defaultFeatureMinVersionLevels: Map[String, Short]): Boolean = { + defaultFeatureMinVersionLevels + .get(feature) + .exists(defaultMinVersionLevel => defaultMinVersionLevel > versionLevels.max()) } private def incompatibleFeatures(supportedFeatures: Features[SupportedVersionRange], @@ -170,22 +170,21 @@ object BrokerFeatures extends Logging { } else if (versionLevels.isIncompatibleWith(supportedVersions)) { (feature, versionLevels, "{feature=%s, reason='%s is incompatible with %s'}".format( feature, versionLevels, supportedVersions)) - } else if (isIncompatibleDefaultMinVersionLevel(feature, versionLevels, defaultFeatureMinVersionLevels)) { + } else if (defaultFeatureMinVersionLevels.isDefined && + isIncompatibleDefaultMinVersionLevel(feature, versionLevels, defaultFeatureMinVersionLevels.get)) { (feature, versionLevels, "{feature=%s, reason='%s is incompatible with default min_version_level: %d'}".format( feature, versionLevels, defaultFeatureMinVersionLevels.get(feature))) } else { (feature, versionLevels, null) - } }.filter{ case(_, _, errorReason) => errorReason != null}.toList if (logIncompatibilities && incompatibleFeaturesInfo.nonEmpty) { - warn( - "Feature incompatibilities seen: " + incompatibleFeaturesInfo.map { - case(_, _, errorReason) => errorReason }.mkString(", ")) + warn("Feature incompatibilities seen: " + + incompatibleFeaturesInfo.map { case(_, _, errorReason) => errorReason }.mkString(", ")) } - Features.finalizedFeatures(incompatibleFeaturesInfo.map { - case(feature, versionLevels, _) => (feature, versionLevels) }.toMap.asJava) + Features.finalizedFeatures( + incompatibleFeaturesInfo.map { case(feature, versionLevels, _) => (feature, versionLevels) }.toMap.asJava) } /** @@ -198,16 +197,14 @@ object BrokerFeatures extends Logging { * @return - true, if the above described check passes. * - false, otherwise. */ - private def areFeatureMinVersionLevelsCompatible( - supportedFeatures: Features[SupportedVersionRange], - featureMinVersionLevels: Map[String, Short] - ): Boolean = { + private def areFeatureMinVersionLevelsCompatible(supportedFeatures: Features[SupportedVersionRange], + featureMinVersionLevels: Map[String, Short]): Boolean = { featureMinVersionLevels.forall { case(featureName, minVersionLevel) => val supportedFeature = supportedFeatures.get(featureName) (supportedFeature != null) && - !new FinalizedVersionRange(minVersionLevel, supportedFeature.max()) - .isIncompatibleWith(supportedFeature) + !new FinalizedVersionRange(minVersionLevel, supportedFeature.max()) + .isIncompatibleWith(supportedFeature) } } } From 5f3af1873803372dd17ac828a19d7450cffbe0f0 Mon Sep 17 00:00:00 2001 From: Kowshik Prakasam Date: Sun, 27 Sep 2020 00:19:21 -0700 Subject: [PATCH 29/41] Small improvement --- .../main/scala/kafka/server/FinalizedFeatureCache.scala | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/kafka/server/FinalizedFeatureCache.scala b/core/src/main/scala/kafka/server/FinalizedFeatureCache.scala index b47cda3617a80..32fb5cdbe580c 100644 --- a/core/src/main/scala/kafka/server/FinalizedFeatureCache.scala +++ b/core/src/main/scala/kafka/server/FinalizedFeatureCache.scala @@ -104,10 +104,10 @@ class FinalizedFeatureCache(private val brokerFeatures: BrokerFeatures) extends */ 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 = brokerFeatures.incompatibleFeatures(latest.features) @@ -117,8 +117,7 @@ class FinalizedFeatureCache(private val brokerFeatures: BrokerFeatures) extends ).format(brokerFeatures.supportedFeatures, latest) throw new FeatureCacheUpdateException(errorMsg) } else { - val logMsg = "Updated cache from existing %s to latest %s".format( - oldFeatureAndEpoch, latest) + val logMsg = "Updated cache from existing %s to latest %s.".format(existing, latest) synchronized { featuresAndEpoch = Some(latest) notifyAll() From 4d067f9afa67cc4a3c7f592c31f9137c9836d609 Mon Sep 17 00:00:00 2001 From: Kowshik Prakasam Date: Sun, 27 Sep 2020 02:27:03 -0700 Subject: [PATCH 30/41] Implement firstActiveVersion --- .../common/feature/BaseVersionRange.java | 33 ++- .../common/feature/FinalizedVersionRange.java | 6 +- .../common/feature/SupportedVersionRange.java | 63 ++++- .../common/message/ApiVersionsResponse.json | 2 +- .../clients/admin/KafkaAdminClientTest.java | 5 +- .../kafka/common/feature/FeaturesTest.java | 36 +-- .../feature/FinalizedVersionRangeTest.java | 21 +- .../feature/SupportedVersionRangeTest.java | 101 +++++-- .../requests/ApiVersionsResponseTest.java | 2 +- .../kafka/controller/KafkaController.scala | 23 +- .../scala/kafka/server/BrokerFeatures.scala | 169 +++++------- core/src/main/scala/kafka/zk/ZkData.scala | 2 +- .../kafka/cluster/BrokerEndPointTest.scala | 6 +- .../kafka/server/BrokerFeaturesTest.scala | 99 ++----- .../server/FinalizedFeatureCacheTest.scala | 8 +- .../FinalizedFeatureChangeListenerTest.scala | 8 +- .../kafka/server/UpdateFeaturesTest.scala | 254 ++++++++++-------- .../unit/kafka/zk/KafkaZkClientTest.scala | 8 +- 18 files changed, 460 insertions(+), 386 deletions(-) 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..05244636265b0 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,35 @@ 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) { + return false; + } + + if (getClass() != other.getClass()) { return false; } @@ -117,8 +134,16 @@ 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; } + + protected String minKeyLabel() { + return minKeyLabel; + } + + protected String maxKeyLabel() { + return maxKeyLabel; + } } 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..545e42169ddcd 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 @@ -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, first_active_version, max] range of the provided SupportedVersionRange parameter. * * @param supportedVersionRange the SupportedVersionRange to be checked * @@ -48,6 +48,8 @@ public static FinalizedVersionRange fromMap(Map versionRangeMap) * - false otherwise */ public boolean isIncompatibleWith(SupportedVersionRange supportedVersionRange) { - return min() < supportedVersionRange.min() || max() > supportedVersionRange.max(); + return min() < supportedVersionRange.min() || + max() > supportedVersionRange.max() || + max() < supportedVersionRange.firstActiveVersion(); } } 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..3c6a070d8c1d8 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 @@ -17,9 +17,16 @@ package org.apache.kafka.common.feature; import java.util.Map; +import java.util.Objects; +import org.apache.kafka.common.utils.Utils; /** - * An extended {@link BaseVersionRange} representing the min/max versions for supported features. + * An extended {@link BaseVersionRange} representing the min, max and first active versions for a + * supported feature: + * - minVersion: This is the minimum supported version for the feature. + * - maxVersion: This the maximum supported version for the feature. + * - firstActiveVersion: This is the first active version for the feature. Versions in the range + * [minVersion, firstActiveVersion - 1] are considered to be deprecated. */ public class SupportedVersionRange extends BaseVersionRange { // Label for the min version key, that's used only to convert to/from a map. @@ -28,13 +35,63 @@ 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); + // Label for the first active version key, that's used only to convert to/from a map. + private static final String FIRST_ACTIVE_VERSION_KEY_LABEL = "first_active_version"; + + private final short firstActiveVersionValue; + + public SupportedVersionRange(short minVersion, short firstActiveVersion, short maxVersion) { + super(MIN_VERSION_KEY_LABEL, minVersion, MAX_VERSION_KEY_LABEL, maxVersion); + if (firstActiveVersion < minVersion || firstActiveVersion > maxVersion) { + throw new IllegalArgumentException( + String.format( + "Expected firstActiveVersion >= minVersion and" + + " firstActiveVersion <= maxVersion, but received" + + " minVersion:%d, firstActiveVersion:%d, maxVersion:%d", + minVersion, + firstActiveVersion, + maxVersion)); + } + this.firstActiveVersionValue = firstActiveVersion; + } + + public short firstActiveVersion() { + return firstActiveVersionValue; + } + + public Map toMap() { + return Utils.mkMap(Utils.mkEntry(minKeyLabel(), min()), + Utils.mkEntry(FIRST_ACTIVE_VERSION_KEY_LABEL, firstActiveVersionValue), + Utils.mkEntry(maxKeyLabel(), max())); + } + + @Override + public boolean equals(Object other) { + if (this == other) { + return true; + } + + if (other == null) { + return false; + } + + if (getClass() != other.getClass()) { + return false; + } + + final SupportedVersionRange that = (SupportedVersionRange) other; + return super.equals(other) && this.firstActiveVersionValue == that.firstActiveVersionValue; + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), firstActiveVersionValue); } public static SupportedVersionRange fromMap(Map versionRangeMap) { return new SupportedVersionRange( BaseVersionRange.valueOrThrow(MIN_VERSION_KEY_LABEL, versionRangeMap), + BaseVersionRange.valueOrThrow(FIRST_ACTIVE_VERSION_KEY_LABEL, versionRangeMap), BaseVersionRange.valueOrThrow(MAX_VERSION_KEY_LABEL, versionRangeMap)); } } diff --git a/clients/src/main/resources/common/message/ApiVersionsResponse.json b/clients/src/main/resources/common/message/ApiVersionsResponse.json index 104a0388b3d5d..e56ba7a0c4c36 100644 --- a/clients/src/main/resources/common/message/ApiVersionsResponse.json +++ b/clients/src/main/resources/common/message/ApiVersionsResponse.json @@ -56,7 +56,7 @@ ] }, {"name": "FinalizedFeaturesEpoch", "type": "int64", "versions": "3+", - "tag": 1, "taggedVersions": "3+", "default": "-1", + "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/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java index 386ab57367589..0ca0dc6fa680a 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 @@ -495,8 +495,9 @@ private static Features c final SupportedVersionRange versionRange = entry.getValue(); featuresMap.put( entry.getKey(), - new org.apache.kafka.common.feature.SupportedVersionRange( - versionRange.minVersion(), versionRange.maxVersion())); + new org.apache.kafka.common.feature.SupportedVersionRange(versionRange.minVersion(), + versionRange.minVersion(), + versionRange.maxVersion())); } return Features.supportedFeatures(featuresMap); diff --git a/clients/src/test/java/org/apache/kafka/common/feature/FeaturesTest.java b/clients/src/test/java/org/apache/kafka/common/feature/FeaturesTest.java index 896196dc550a4..81db1ef096ec6 100644 --- a/clients/src/test/java/org/apache/kafka/common/feature/FeaturesTest.java +++ b/clients/src/test/java/org/apache/kafka/common/feature/FeaturesTest.java @@ -59,8 +59,8 @@ public void testNullFeatures() { @Test public void testGetAllFeaturesAPI() { - SupportedVersionRange v1 = new SupportedVersionRange((short) 1, (short) 2); - SupportedVersionRange v2 = new SupportedVersionRange((short) 3, (short) 4); + SupportedVersionRange v1 = new SupportedVersionRange((short) 1, (short) 2, (short) 3); + SupportedVersionRange v2 = new SupportedVersionRange((short) 3, (short) 4, (short) 5); Map allFeatures = mkMap(mkEntry("feature_1", v1), mkEntry("feature_2", v2)); Features features = Features.supportedFeatures(allFeatures); @@ -69,8 +69,8 @@ public void testGetAllFeaturesAPI() { @Test public void testGetAPI() { - SupportedVersionRange v1 = new SupportedVersionRange((short) 1, (short) 2); - SupportedVersionRange v2 = new SupportedVersionRange((short) 3, (short) 4); + SupportedVersionRange v1 = new SupportedVersionRange((short) 1, (short) 2, (short) 3); + SupportedVersionRange v2 = new SupportedVersionRange((short) 3, (short) 4, (short) 5); Map allFeatures = mkMap(mkEntry("feature_1", v1), mkEntry("feature_2", v2)); Features features = Features.supportedFeatures(allFeatures); assertEquals(v1, features.get("feature_1")); @@ -80,15 +80,19 @@ public void testGetAPI() { @Test public void testFromFeaturesMapToFeaturesMap() { - SupportedVersionRange v1 = new SupportedVersionRange((short) 1, (short) 2); - SupportedVersionRange v2 = new SupportedVersionRange((short) 3, (short) 4); + SupportedVersionRange v1 = new SupportedVersionRange((short) 1, (short) 2, (short) 3); + SupportedVersionRange v2 = new SupportedVersionRange((short) 3, (short) 4, (short) 5); Map allFeatures = mkMap(mkEntry("feature_1", v1), mkEntry("feature_2", v2)); Features features = Features.supportedFeatures(allFeatures); Map> expected = mkMap( - mkEntry("feature_1", mkMap(mkEntry("min_version", (short) 1), mkEntry("max_version", (short) 2))), - mkEntry("feature_2", mkMap(mkEntry("min_version", (short) 3), mkEntry("max_version", (short) 4)))); + mkEntry("feature_1", mkMap(mkEntry("min_version", (short) 1), + mkEntry("first_active_version", (short) 2), + mkEntry("max_version", (short) 3))), + mkEntry("feature_2", mkMap(mkEntry("min_version", (short) 3), + mkEntry("first_active_version", (short) 4), + mkEntry("max_version", (short) 5)))); assertEquals(expected, features.toMap()); assertEquals(features, Features.fromSupportedFeaturesMap(expected)); } @@ -117,21 +121,23 @@ public void testToStringFinalizedFeatures() { Features features = Features.finalizedFeatures(allFeatures); assertEquals( - "Features{(feature_1 -> FinalizedVersionRange[min_version_level:1, max_version_level:2]), (feature_2 -> FinalizedVersionRange[min_version_level:3, max_version_level:4])}", + "Features{(feature_1 -> FinalizedVersionRange[min_version_level:1, max_version_level:2])," + + " (feature_2 -> FinalizedVersionRange[min_version_level:3, max_version_level:4])}", features.toString()); } @Test public void testToStringSupportedFeatures() { - SupportedVersionRange v1 = new SupportedVersionRange((short) 1, (short) 2); - SupportedVersionRange v2 = new SupportedVersionRange((short) 3, (short) 4); + SupportedVersionRange v1 = new SupportedVersionRange((short) 1, (short) 2, (short) 3); + SupportedVersionRange v2 = new SupportedVersionRange((short) 3, (short) 4, (short) 5); Map allFeatures = mkMap(mkEntry("feature_1", v1), mkEntry("feature_2", v2)); Features features = Features.supportedFeatures(allFeatures); assertEquals( - "Features{(feature_1 -> SupportedVersionRange[min_version:1, max_version:2]), (feature_2 -> SupportedVersionRange[min_version:3, max_version:4])}", + "Features{(feature_1 -> SupportedVersionRange[min_version:1, first_active_version:2, max_version:3])," + + " (feature_2 -> SupportedVersionRange[min_version:3, first_active_version:4, max_version:5])}", features.toString()); } @@ -139,7 +145,7 @@ public void testToStringSupportedFeatures() { public void testSuppportedFeaturesFromMapFailureWithInvalidMissingMaxVersion() { // This is invalid because 'max_version' key is missing. Map> invalidFeatures = mkMap( - mkEntry("feature_1", mkMap(mkEntry("min_version", (short) 1)))); + mkEntry("feature_1", mkMap(mkEntry("min_version", (short) 1), mkEntry("first_active_version", (short) 2)))); assertThrows( IllegalArgumentException.class, () -> Features.fromSupportedFeaturesMap(invalidFeatures)); @@ -157,13 +163,13 @@ public void testFinalizedFeaturesFromMapFailureWithInvalidMissingMaxVersionLevel @Test public void testEquals() { - SupportedVersionRange v1 = new SupportedVersionRange((short) 1, (short) 2); + SupportedVersionRange v1 = new SupportedVersionRange((short) 1, (short) 2, (short) 3); Map allFeatures = mkMap(mkEntry("feature_1", v1)); Features features = Features.supportedFeatures(allFeatures); Features featuresClone = Features.supportedFeatures(allFeatures); assertTrue(features.equals(featuresClone)); - SupportedVersionRange v2 = new SupportedVersionRange((short) 1, (short) 3); + SupportedVersionRange v2 = new SupportedVersionRange((short) 1, (short) 3, (short) 4); Map allFeaturesDifferent = mkMap(mkEntry("feature_1", v2)); Features featuresDifferent = Features.supportedFeatures(allFeaturesDifferent); assertFalse(features.equals(featuresDifferent)); diff --git a/clients/src/test/java/org/apache/kafka/common/feature/FinalizedVersionRangeTest.java b/clients/src/test/java/org/apache/kafka/common/feature/FinalizedVersionRangeTest.java index 3d62a8fdbad8a..32aeaec23b46e 100644 --- a/clients/src/test/java/org/apache/kafka/common/feature/FinalizedVersionRangeTest.java +++ b/clients/src/test/java/org/apache/kafka/common/feature/FinalizedVersionRangeTest.java @@ -61,14 +61,21 @@ public void testToString() { } @Test - public void testIsCompatibleWith() { - assertFalse(new FinalizedVersionRange((short) 1, (short) 1).isIncompatibleWith(new SupportedVersionRange((short) 1, (short) 1))); - assertFalse(new FinalizedVersionRange((short) 2, (short) 3).isIncompatibleWith(new SupportedVersionRange((short) 1, (short) 4))); - assertFalse(new FinalizedVersionRange((short) 1, (short) 4).isIncompatibleWith(new SupportedVersionRange((short) 1, (short) 4))); + public void testIsIncompatibleWith() { + assertFalse(new FinalizedVersionRange((short) 1, (short) 1).isIncompatibleWith(new SupportedVersionRange((short) 1, (short) 1, (short) 1))); + assertFalse(new FinalizedVersionRange((short) 2, (short) 3).isIncompatibleWith(new SupportedVersionRange((short) 1, (short) 1, (short) 4))); + assertFalse(new FinalizedVersionRange((short) 2, (short) 3).isIncompatibleWith(new SupportedVersionRange((short) 1, (short) 2, (short) 4))); + assertFalse(new FinalizedVersionRange((short) 2, (short) 3).isIncompatibleWith(new SupportedVersionRange((short) 1, (short) 3, (short) 4))); + assertFalse(new FinalizedVersionRange((short) 1, (short) 4).isIncompatibleWith(new SupportedVersionRange((short) 1, (short) 1, (short) 4))); - assertTrue(new FinalizedVersionRange((short) 1, (short) 4).isIncompatibleWith(new SupportedVersionRange((short) 2, (short) 3))); - assertTrue(new FinalizedVersionRange((short) 1, (short) 4).isIncompatibleWith(new SupportedVersionRange((short) 2, (short) 4))); - assertTrue(new FinalizedVersionRange((short) 2, (short) 4).isIncompatibleWith(new SupportedVersionRange((short) 2, (short) 3))); + // minVersion and maxVersion are incompatible. + assertTrue(new FinalizedVersionRange((short) 1, (short) 4).isIncompatibleWith(new SupportedVersionRange((short) 2, (short) 2, (short) 3))); + // Only minVersion is incompatible. + assertTrue(new FinalizedVersionRange((short) 1, (short) 4).isIncompatibleWith(new SupportedVersionRange((short) 2, (short) 2, (short) 4))); + // Only maxVersion is incompatible. + assertTrue(new FinalizedVersionRange((short) 2, (short) 4).isIncompatibleWith(new SupportedVersionRange((short) 2, (short) 2, (short) 3))); + // Only firstActiveVersion is incompatible. + assertTrue(new FinalizedVersionRange((short) 1, (short) 4).isIncompatibleWith(new SupportedVersionRange((short) 1, (short) 5, (short) 5))); } @Test diff --git a/clients/src/test/java/org/apache/kafka/common/feature/SupportedVersionRangeTest.java b/clients/src/test/java/org/apache/kafka/common/feature/SupportedVersionRangeTest.java index 4c7b5591dad0f..8658afa1e02ef 100644 --- a/clients/src/test/java/org/apache/kafka/common/feature/SupportedVersionRangeTest.java +++ b/clients/src/test/java/org/apache/kafka/common/feature/SupportedVersionRangeTest.java @@ -38,35 +38,52 @@ public void testFailDueToInvalidParams() { // min and max can't be < 1. assertThrows( IllegalArgumentException.class, - () -> new SupportedVersionRange((short) 0, (short) 0)); + () -> new SupportedVersionRange((short) 0, (short) 1, (short) 0)); // min can't be < 1. assertThrows( IllegalArgumentException.class, - () -> new SupportedVersionRange((short) 0, (short) 1)); + () -> new SupportedVersionRange((short) 0, (short) 1, (short) 1)); // max can't be < 1. assertThrows( IllegalArgumentException.class, - () -> new SupportedVersionRange((short) 1, (short) 0)); + () -> new SupportedVersionRange((short) 1, (short) 1, (short) 0)); + // firstActiveVersion can't be < 1. + assertThrows( + IllegalArgumentException.class, + () -> new SupportedVersionRange((short) 1, (short) 0, (short) 1)); // min can't be > max. assertThrows( IllegalArgumentException.class, - () -> new SupportedVersionRange((short) 2, (short) 1)); + () -> new SupportedVersionRange((short) 2, (short) 1, (short) 1)); + // firstActiveVersion can't be < min. + assertThrows( + IllegalArgumentException.class, + () -> new SupportedVersionRange((short) 2, (short) 1, (short) 2)); + // firstActiveVersion can't be > max. + assertThrows( + IllegalArgumentException.class, + () -> new SupportedVersionRange((short) 1, (short) 3, (short) 2)); } @Test public void testFromToMap() { - SupportedVersionRange versionRange = new SupportedVersionRange((short) 1, (short) 2); + SupportedVersionRange versionRange = new SupportedVersionRange((short) 1, (short) 2, (short) 3); assertEquals(1, versionRange.min()); - assertEquals(2, versionRange.max()); + assertEquals(2, versionRange.firstActiveVersion()); + assertEquals(3, versionRange.max()); Map versionRangeMap = versionRange.toMap(); assertEquals( - mkMap(mkEntry("min_version", versionRange.min()), mkEntry("max_version", versionRange.max())), + mkMap(mkEntry("min_version", versionRange.min()), + mkEntry("first_active_version", versionRange.firstActiveVersion()), + mkEntry("max_version", versionRange.max())), versionRangeMap); SupportedVersionRange newVersionRange = SupportedVersionRange.fromMap(versionRangeMap); assertEquals(1, newVersionRange.min()); - assertEquals(2, newVersionRange.max()); + assertEquals(2, newVersionRange.firstActiveVersion()); + assertEquals(3, newVersionRange.max()); + assertEquals(versionRange, newVersionRange); } @@ -74,69 +91,105 @@ public void testFromToMap() { public void testFromMapFailure() { // min_version can't be < 1. Map invalidWithBadMinVersion = - mkMap(mkEntry("min_version", (short) 0), mkEntry("max_version", (short) 1)); + mkMap(mkEntry("min_version", (short) 0), + mkEntry("first_active_version", (short) 1), + mkEntry("max_version", (short) 1)); assertThrows( IllegalArgumentException.class, () -> SupportedVersionRange.fromMap(invalidWithBadMinVersion)); // max_version can't be < 1. Map invalidWithBadMaxVersion = - mkMap(mkEntry("min_version", (short) 1), mkEntry("max_version", (short) 0)); + mkMap(mkEntry("min_version", (short) 1), + mkEntry("first_active_version", (short) 1), + mkEntry("max_version", (short) 0)); assertThrows( IllegalArgumentException.class, () -> SupportedVersionRange.fromMap(invalidWithBadMaxVersion)); // min_version and max_version can't be < 1. Map invalidWithBadMinMaxVersion = - mkMap(mkEntry("min_version", (short) 0), mkEntry("max_version", (short) 0)); + mkMap(mkEntry("min_version", (short) 0), + mkEntry("first_active_version", (short) 1), + mkEntry("max_version", (short) 0)); assertThrows( IllegalArgumentException.class, () -> SupportedVersionRange.fromMap(invalidWithBadMinMaxVersion)); // min_version can't be > max_version. Map invalidWithLowerMaxVersion = - mkMap(mkEntry("min_version", (short) 2), mkEntry("max_version", (short) 1)); + mkMap(mkEntry("min_version", (short) 2), + mkEntry("first_active_version", (short) 1), + mkEntry("max_version", (short) 1)); assertThrows( IllegalArgumentException.class, () -> SupportedVersionRange.fromMap(invalidWithLowerMaxVersion)); + // first_active_version can't be < min_version + Map invalidWithLowerFirstActiveVersion = + mkMap(mkEntry("min_version", (short) 2), + mkEntry("first_active_version", (short) 1), + mkEntry("max_version", (short) 3)); + assertThrows( + IllegalArgumentException.class, + () -> SupportedVersionRange.fromMap(invalidWithLowerFirstActiveVersion)); + + // first_active_version can't be > max_version + Map invalidWithHigherFirstActiveVersion = + mkMap(mkEntry("min_version", (short) 1), + mkEntry("first_active_version", (short) 3), + mkEntry("max_version", (short) 2)); + assertThrows( + IllegalArgumentException.class, + () -> SupportedVersionRange.fromMap(invalidWithHigherFirstActiveVersion)); + // min_version key missing. Map invalidWithMinKeyMissing = - mkMap(mkEntry("max_version", (short) 1)); + mkMap(mkEntry("first_active_version", (short) 1), + mkEntry("max_version", (short) 1)); assertThrows( IllegalArgumentException.class, () -> SupportedVersionRange.fromMap(invalidWithMinKeyMissing)); // max_version key missing. Map invalidWithMaxKeyMissing = - mkMap(mkEntry("min_version", (short) 1)); + mkMap(mkEntry("min_version", (short) 1), + mkEntry("first_active_version", (short) 1)); assertThrows( IllegalArgumentException.class, () -> SupportedVersionRange.fromMap(invalidWithMaxKeyMissing)); + + // first_active_version key missing. + Map invalidWithFirstActiveVersionKeyMissing = + mkMap(mkEntry("min_version", (short) 1), + mkEntry("max_version", (short) 1)); + assertThrows( + IllegalArgumentException.class, + () -> SupportedVersionRange.fromMap(invalidWithFirstActiveVersionKeyMissing)); } @Test public void testToString() { assertEquals( - "SupportedVersionRange[min_version:1, max_version:1]", - new SupportedVersionRange((short) 1, (short) 1).toString()); - assertEquals( - "SupportedVersionRange[min_version:1, max_version:2]", - new SupportedVersionRange((short) 1, (short) 2).toString()); + "SupportedVersionRange[min_version:1, first_active_version:2, max_version:3]", + new SupportedVersionRange((short) 1, (short) 2, (short) 3).toString()); } @Test public void testEquals() { - SupportedVersionRange tested = new SupportedVersionRange((short) 1, (short) 1); + SupportedVersionRange tested = new SupportedVersionRange((short) 1, (short) 2, (short) 4); assertTrue(tested.equals(tested)); - assertFalse(tested.equals(new SupportedVersionRange((short) 1, (short) 2))); + assertFalse(tested.equals(new SupportedVersionRange(tested.firstActiveVersion(), tested.firstActiveVersion(), tested.max()))); + assertFalse(tested.equals(new SupportedVersionRange(tested.min(), (short) (tested.firstActiveVersion() + 1), tested.max()))); + assertFalse(tested.equals(new SupportedVersionRange(tested.min(), tested.firstActiveVersion(), (short) (tested.max() + 1)))); assertFalse(tested.equals(null)); } @Test - public void testMinMax() { - SupportedVersionRange versionRange = new SupportedVersionRange((short) 1, (short) 2); + public void testAttributes() { + SupportedVersionRange versionRange = new SupportedVersionRange((short) 1, (short) 2, (short) 3); assertEquals(1, versionRange.min()); - assertEquals(2, versionRange.max()); + assertEquals(2, versionRange.firstActiveVersion()); + assertEquals(3, versionRange.max()); } } diff --git a/clients/src/test/java/org/apache/kafka/common/requests/ApiVersionsResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/ApiVersionsResponseTest.java index fbb2d0c5ced51..0debaa154102b 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/ApiVersionsResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/ApiVersionsResponseTest.java @@ -108,7 +108,7 @@ public void shouldReturnFeatureKeysWhenMagicIsCurrentValueAndThrottleMsIsDefault ApiVersionsResponse response = ApiVersionsResponse.apiVersionsResponse( 10, RecordBatch.MAGIC_VALUE_V1, - Features.supportedFeatures(Utils.mkMap(Utils.mkEntry("feature", new SupportedVersionRange((short) 1, (short) 4)))), + Features.supportedFeatures(Utils.mkMap(Utils.mkEntry("feature", new SupportedVersionRange((short) 1, (short) 2, (short) 4)))), Features.finalizedFeatures(Utils.mkMap(Utils.mkEntry("feature", new FinalizedVersionRange((short) 2, (short) 3)))), 10); verifyApiKeysForMagic(response, RecordBatch.MAGIC_VALUE_V1); diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 742a4f9be9dc4..d9607cd97a437 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -1907,15 +1907,15 @@ class KafkaController(val config: KafkaConfig, throw new IllegalArgumentException(s"Provided feature update can not be meant to delete the feature: $update") } - val defaultMinVersionLevelOpt = brokerFeatures.defaultMinVersionLevel(update.feature) - if (defaultMinVersionLevelOpt.isEmpty) { + 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(defaultMinVersionLevelOpt.get, update.maxVersionLevel) + newVersionRange = new FinalizedVersionRange(supportedVersionRange.firstActiveVersion, update.maxVersionLevel) } catch { case _: IllegalArgumentException => { // This exception means the provided maxVersionLevel is invalid. It is handled below @@ -1926,7 +1926,7 @@ class KafkaController(val config: KafkaConfig, Right(new ApiError(Errors.INVALID_REQUEST, "Could not apply finalized feature update because the provided" + s" maxVersionLevel:${update.maxVersionLevel} is lower than the" + - s" default minVersionLevel:${defaultMinVersionLevelOpt.get}.")) + s" first active version:${supportedVersionRange.firstActiveVersion}.")) } else { val newFinalizedFeature = Features.finalizedFeatures(Utils.mkMap(Utils.mkEntry(update.feature, newVersionRange))) @@ -1945,22 +1945,21 @@ class KafkaController(val config: KafkaConfig, } /** - * Validate and process a finalized feature update on an existing FinalizedVersionRange for the - * feature. - * - * If the processing succeeds, then, the return value contains: + * 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 processing fails, then returned value contains a suitable ApiError. + * 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 or error, as described above. + * @return the new FinalizedVersionRange to be updated into ZK or error + * as described above. */ - private def processFeatureUpdate(update: UpdateFeaturesRequestData.FeatureUpdateKey, + private def validateFeatureUpdate(update: UpdateFeaturesRequestData.FeatureUpdateKey, existingVersionRange: Option[FinalizedVersionRange]): Either[Option[FinalizedVersionRange], ApiError] = { def newVersionRangeOrError(update: UpdateFeaturesRequestData.FeatureUpdateKey): Either[Option[FinalizedVersionRange], ApiError] = { newFinalizedVersionRangeOrIncompatibilityError(update) @@ -2054,7 +2053,7 @@ class KafkaController(val config: KafkaConfig, // - 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 => - processFeatureUpdate(update, existingFeatures.get(update.feature())) match { + validateFeatureUpdate(update, existingFeatures.get(update.feature())) match { case Left(newVersionRangeOrNone) => newVersionRangeOrNone .map(newVersionRange => targetFeatures += (update.feature() -> newVersionRange)) diff --git a/core/src/main/scala/kafka/server/BrokerFeatures.scala b/core/src/main/scala/kafka/server/BrokerFeatures.scala index 72924bc805afa..4b07bf43fb97b 100644 --- a/core/src/main/scala/kafka/server/BrokerFeatures.scala +++ b/core/src/main/scala/kafka/server/BrokerFeatures.scala @@ -24,76 +24,80 @@ import org.apache.kafka.common.feature.Features._ import scala.jdk.CollectionConverters._ /** - * A class that encapsulates the attributes explained below 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. + * 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. + * The class also enables feature version level deprecation, as explained below. This class is + * immutable in production. It provides few APIs to mutate state only for the purpose of testing. * - * Attributes: + * Feature version level deprecation: + * ================================== * - * 1. The latest features supported by the Broker. + * Deprecation of certain version levels of a feature is a process to stop supporting the + * functionality offered by the feature at a those version levels, across the entire Kafka cluster. + * Feature version deprecation is a simple 2-step process explained below. In each step below, an + * example is provided to help understand the process better: * - * 2. The optional default minimum version levels for specific finalized features. - * - If you would want to deprecate a version level for some feature, then in this map you - * need to supply the starting version value (greater than 1) that's just 1 beyond the highest - * deprecated version. Ex: if this map contains {"feature1" -> 5}, then it indicates that feature - * version levels: [1, 4] need to be deprecated. The value '5' is the default minimum version level. - * - If you do not want to deprecate a version level for a feature, you do not have to supply - * values in this map. The default minimum version level for absent features in this map - * is assumed to be 1. + * STEP 1: + * In the first step, a major Kafka release is made with a Broker code change (explained later + * below) that establishes the intent to deprecate certain versions of one or more features + * cluster-wide. When this new Kafka release is deployed to the cluster, the feature versioning + * system (via the controller) will automatically persist the new minVersionLevel for the feature in + * Zk to propagate the deprecation of certain versions. After this happens, any external client that + * queries the Broker to learn the feature versions will at some point start to see the new value + * for the finalized minVersionLevel for the feature. This makes the version deprecation permanent. * - * The primary use case to provide this map is feature version level deprecation. - * When features are finalized via the ApiKeys.UPDATE_FEATURES api, the controller takes the - * value provided in this map (if present) as the default minimum version level for the feature. - * This is how it works: in order to deprecate feature version levels, in this map the default - * minimum version level of a feature can be set to a new value that's higher than 1 - * (let's call this latest_min_version_level). In doing so, the feature version levels - * in the closed range: [1, latest_min_version_level - 1] get deprecated by the controller logic - * that applies this map to persistent finalized feature state in ZK (this mutation happens - * during controller election and during finalized feature updates via the - * ApiKeys.UPDATE_FEATURES api). This will automatically mean external clients of Kafka - * would need to stop using the finalized min version levels that have been deprecated. + * Here is how the above code change needs to be done: + * In order to deprecate feature version levels, in the supportedFeatures map you need to supply a + * specific firstActiveVersion value that's higher than the minVersion for the feature. The + * value for firstActiveVersion should be 1 beyond the highest version that you intend to deprecate + * for that feature. When features are finalized via the ApiKeys.UPDATE_FEATURES api, the feature + * version levels in the closed range: [minVersion, firstActiveVersion - 1] are automatically + * deprecated in ZK by the controller logic. + * Example: + * - Let us assume the existing finalized feature in ZK: + * { + * "feature_1" -> FinalizedVersionRange(minVersionLevel=1, maxVersionLevel=5) + * } + * Now, supposing you would like to deprecate feature version levels: [1, 2]. + * Then, in the supportedFeatures map you should supply the following: + * supportedFeatures = { + * "feature1" -> SupportedVersionRange(minVersion=1, firstActiveVersion=3, maxVersion=5) + * } + * - If you do NOT want to deprecate a version level for a feature, then in the supportedFeatures + * map you should supply the firstActiveVersion to be the same as the minVersion supplied for that + * feature. + * Example: + * supportedFeatures = { + * "feature1" -> SupportedVersionRange(minVersion=1, firstActiveVersion=1, maxVersion=5) + * } + * This indicates no intent to deprecate any version levels for the feature. * - * NOTE: The difference between the values in this map and the minimum version value for a - * broker's supported feature is the following: Version levels below the values specified in this - * map are considered deprecated by the controller, whereas version levels below the minimum - * version value for a supported feature are considered unknown/unsupported. + * STEP 2: + * After the first step is over, you may (at some point) want to permanently remove the code/logic + * for the functionality offered by the deprecated feature. This is the second step. Here a + * subsequent major Kafka release is made with another Broker code change that removes the code for + * the functionality offered by the deprecated feature versions. This would completely drop support + * for the deprecated versions. Such a code change needs to be supplemented by supplying a + * suitable higher minVersion value for the feature in the supportedFeatures map. + * Example: + * - In the example above in step 1, we showed how to deprecate version levels [1, 2] for + * "feature_1". Now let us assume the following finalized feature in ZK (after the deprecation + * has been carried out): + * { + * "feature_1" -> FinalizedVersionRange(minVersionLevel=3, maxVersionLevel=5) + * } + * Now, supposing you would like to permanently remove support for feature versions: [1, 2]. + * Then, in the supportedFeatures map you should now supply the following: + * supportedFeatures = { + * "feature1" -> SupportedVersionRange(minVersion=3, firstActiveVersion=3, maxVersion=5) + * } */ -class BrokerFeatures private (@volatile var supportedFeatures: Features[SupportedVersionRange], - @volatile var defaultFeatureMinVersionLevels: Map[String, Short]) { - require(BrokerFeatures.areFeatureMinVersionLevelsCompatible( - supportedFeatures, defaultFeatureMinVersionLevels)) - +class BrokerFeatures private (@volatile var supportedFeatures: Features[SupportedVersionRange]) { // For testing only. def setSupportedFeatures(newFeatures: Features[SupportedVersionRange]): Unit = { - require( - BrokerFeatures.areFeatureMinVersionLevelsCompatible(newFeatures, defaultFeatureMinVersionLevels)) supportedFeatures = newFeatures } - /** - * Returns the default minimum version level for a specific supported feature. - * - * @param feature the name of the feature - * - * @return the default minimum version level for the supported feature if its defined. - * otherwise, returns the minimum version of the supported feature (if the feature - * exists) or none. - */ - def defaultMinVersionLevel(feature: String): Option[Short] = { - defaultFeatureMinVersionLevels.get(feature).map(Some(_)).getOrElse { - val versionRange = supportedFeatures.get(feature) - if (versionRange == null) Option.empty else Some(versionRange.min()) - } - } - - // For testing only. - def setDefaultMinVersionLevels(newMinVersionLevels: Map[String, Short]): Unit = { - require( - BrokerFeatures.areFeatureMinVersionLevelsCompatible(supportedFeatures, newMinVersionLevels)) - defaultFeatureMinVersionLevels = newMinVersionLevels - } - /** * Returns the default finalized features that a new Kafka cluster with IBP config >= KAFKA_2_7_IV0 * needs to be bootstrapped with. @@ -102,7 +106,7 @@ class BrokerFeatures private (@volatile var supportedFeatures: Features[Supporte Features.finalizedFeatures( supportedFeatures.features.asScala.map { case(name, versionRange) => ( - name, new FinalizedVersionRange(defaultMinVersionLevel(name).get, versionRange.max)) + name, new FinalizedVersionRange(versionRange.firstActiveVersion, versionRange.max)) }.asJava) } @@ -113,8 +117,8 @@ class BrokerFeatures private (@volatile var supportedFeatures: Features[Supporte * 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 either the SupportedVersionRange - * of the supported feature, or the default minimum version level of the feature. + * 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. * @@ -122,8 +126,7 @@ class BrokerFeatures private (@volatile var supportedFeatures: Features[Supporte * is empty, it means there were no feature incompatibilities found. */ def incompatibleFeatures(finalized: Features[FinalizedVersionRange]): Features[FinalizedVersionRange] = { - BrokerFeatures.incompatibleFeatures( - supportedFeatures, finalized, Some(defaultFeatureMinVersionLevels), logIncompatibilities = true) + BrokerFeatures.incompatibleFeatures(supportedFeatures, finalized, logIncompatibilities = true) } } @@ -132,7 +135,7 @@ 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, Map[String, Short]()) + new BrokerFeatures(emptySupportedFeatures) } /** @@ -147,20 +150,11 @@ object BrokerFeatures extends Logging { */ def hasIncompatibleFeatures(supportedFeatures: Features[SupportedVersionRange], finalizedFeatures: Features[FinalizedVersionRange]): Boolean = { - !incompatibleFeatures(supportedFeatures, finalizedFeatures, Option.empty, false).empty - } - - private def isIncompatibleDefaultMinVersionLevel(feature: String, - versionLevels: FinalizedVersionRange, - defaultFeatureMinVersionLevels: Map[String, Short]): Boolean = { - defaultFeatureMinVersionLevels - .get(feature) - .exists(defaultMinVersionLevel => defaultMinVersionLevel > versionLevels.max()) + !incompatibleFeatures(supportedFeatures, finalizedFeatures, logIncompatibilities = false).empty } private def incompatibleFeatures(supportedFeatures: Features[SupportedVersionRange], finalizedFeatures: Features[FinalizedVersionRange], - defaultFeatureMinVersionLevels: Option[Map[String, Short]], logIncompatibilities: Boolean): Features[FinalizedVersionRange] = { val incompatibleFeaturesInfo = finalizedFeatures.features.asScala.map { case (feature, versionLevels) => @@ -170,10 +164,6 @@ object BrokerFeatures extends Logging { } else if (versionLevels.isIncompatibleWith(supportedVersions)) { (feature, versionLevels, "{feature=%s, reason='%s is incompatible with %s'}".format( feature, versionLevels, supportedVersions)) - } else if (defaultFeatureMinVersionLevels.isDefined && - isIncompatibleDefaultMinVersionLevel(feature, versionLevels, defaultFeatureMinVersionLevels.get)) { - (feature, versionLevels, "{feature=%s, reason='%s is incompatible with default min_version_level: %d'}".format( - feature, versionLevels, defaultFeatureMinVersionLevels.get(feature))) } else { (feature, versionLevels, null) } @@ -186,25 +176,4 @@ object BrokerFeatures extends Logging { Features.finalizedFeatures( incompatibleFeaturesInfo.map { case(feature, versionLevels, _) => (feature, versionLevels) }.toMap.asJava) } - - /** - * A check that ensures each feature defined with min version level is a supported feature, and - * the min version level value is valid (i.e. it is compatible with the supported version range). - * - * @param supportedFeatures the supported features - * @param featureMinVersionLevels the feature minimum version levels - * - * @return - true, if the above described check passes. - * - false, otherwise. - */ - private def areFeatureMinVersionLevelsCompatible(supportedFeatures: Features[SupportedVersionRange], - featureMinVersionLevels: Map[String, Short]): Boolean = { - featureMinVersionLevels.forall { - case(featureName, minVersionLevel) => - val supportedFeature = supportedFeatures.get(featureName) - (supportedFeature != null) && - !new FinalizedVersionRange(minVersionLevel, supportedFeature.max()) - .isIncompatibleWith(supportedFeature) - } - } } diff --git a/core/src/main/scala/kafka/zk/ZkData.scala b/core/src/main/scala/kafka/zk/ZkData.scala index 87c2335c538a6..18f364a8bcaac 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 = { diff --git a/core/src/test/scala/unit/kafka/cluster/BrokerEndPointTest.scala b/core/src/test/scala/unit/kafka/cluster/BrokerEndPointTest.scala index 3708f73aedb64..2f3850b802bdd 100644 --- a/core/src/test/scala/unit/kafka/cluster/BrokerEndPointTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/BrokerEndPointTest.scala @@ -185,7 +185,7 @@ class BrokerEndPointTest { "endpoints":["CLIENT://host1:9092", "REPLICATION://host1:9093"], "listener_security_protocol_map":{"CLIENT":"SSL", "REPLICATION":"PLAINTEXT"}, "rack":"dc1", - "features": {"feature1": {"min_version": 1, "max_version": 2}, "feature2": {"min_version": 2, "max_version": 4}} + "features": {"feature1": {"min_version": 1, "first_active_version": 1, "max_version": 2}, "feature2": {"min_version": 2, "first_active_version": 2, "max_version": 4}} }""" val broker = parseBrokerJson(1, json) assertEquals(1, broker.id) @@ -195,8 +195,8 @@ class BrokerEndPointTest { assertEquals(Some("dc1"), broker.rack) assertEquals(Features.supportedFeatures( Map[String, SupportedVersionRange]( - "feature1" -> new SupportedVersionRange(1, 2), - "feature2" -> new SupportedVersionRange(2, 4)).asJava), + "feature1" -> new SupportedVersionRange(1, 1, 2), + "feature2" -> new SupportedVersionRange(2, 2, 4)).asJava), broker.features) } diff --git a/core/src/test/scala/unit/kafka/server/BrokerFeaturesTest.scala b/core/src/test/scala/unit/kafka/server/BrokerFeaturesTest.scala index 5f564af5c55fe..bc04cd9bfb266 100644 --- a/core/src/test/scala/unit/kafka/server/BrokerFeaturesTest.scala +++ b/core/src/test/scala/unit/kafka/server/BrokerFeaturesTest.scala @@ -18,7 +18,7 @@ package kafka.server import org.apache.kafka.common.feature.{Features, FinalizedVersionRange, SupportedVersionRange} -import org.junit.Assert.{assertEquals, assertThrows, assertFalse, assertTrue} +import org.junit.Assert.{assertEquals, assertFalse, assertTrue} import org.junit.Test import scala.jdk.CollectionConverters._ @@ -34,8 +34,8 @@ class BrokerFeaturesTest { 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) + "test_feature_1" -> new SupportedVersionRange(1, 1, 4), + "test_feature_2" -> new SupportedVersionRange(1, 1, 3)).asJava) brokerFeatures.setSupportedFeatures(supportedFeatures) val compatibleFeatures = Map[String, FinalizedVersionRange]( @@ -55,8 +55,8 @@ class BrokerFeaturesTest { 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) + "test_feature_1" -> new SupportedVersionRange(1, 1, 4), + "test_feature_2" -> new SupportedVersionRange(1, 1, 3)).asJava) brokerFeatures.setSupportedFeatures(supportedFeatures) val compatibleFeatures = Map[String, FinalizedVersionRange]( @@ -73,44 +73,34 @@ class BrokerFeaturesTest { } @Test - def testIncompatibilitiesWithDefaultMinVersionLevel(): Unit = { + def testIncompatibilitiesWithFirstActiveVersion(): 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) + "test_feature_1" -> new SupportedVersionRange(1, 2, 4), + "test_feature_2" -> new SupportedVersionRange(1, 3, 4)).asJava) brokerFeatures.setSupportedFeatures(supportedFeatures) - val defaultMinVersionLevels = Map[String, Short]( - "test_feature_1" -> 2, - "test_feature_2" -> 2) - brokerFeatures.setDefaultMinVersionLevels(defaultMinVersionLevels) - val compatibleFeatures = Map[String, FinalizedVersionRange]( - "test_feature_1" -> new FinalizedVersionRange(2, 3)) + "test_feature_1" -> new FinalizedVersionRange(1, 2)) val inCompatibleFeatures = Map[String, FinalizedVersionRange]( - "test_feature_2" -> new FinalizedVersionRange(1, 1)) + "test_feature_2" -> new FinalizedVersionRange(1, 2)) val features = compatibleFeatures++inCompatibleFeatures val finalizedFeatures = Features.finalizedFeatures(features.asJava) assertEquals( Features.finalizedFeatures(inCompatibleFeatures.asJava), brokerFeatures.incompatibleFeatures(finalizedFeatures)) - assertFalse(BrokerFeatures.hasIncompatibleFeatures(supportedFeatures, 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) + "test_feature_1" -> new SupportedVersionRange(1, 2, 4), + "test_feature_2" -> new SupportedVersionRange(1, 3, 3)).asJava) brokerFeatures.setSupportedFeatures(supportedFeatures) - val defaultMinVersionLevels = Map[String, Short]( - "test_feature_1" -> 2, - "test_feature_2" -> 2) - brokerFeatures.setDefaultMinVersionLevels(defaultMinVersionLevels) - val compatibleFeatures = Map[String, FinalizedVersionRange]( "test_feature_1" -> new FinalizedVersionRange(2, 3), "test_feature_2" -> new FinalizedVersionRange(1, 3)) @@ -119,74 +109,19 @@ class BrokerFeaturesTest { assertFalse(BrokerFeatures.hasIncompatibleFeatures(supportedFeatures, finalizedFeatures)) } - - @Test - def testFeatureVersionAssertions(): 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 defaultMinVersionLevelsWithNonExistingFeature = Map[String, Short]( - "test_feature_1" -> 2, - "test_feature_2" -> 2, - "test_feature_non_existing" -> 5) - assertThrows( - classOf[IllegalArgumentException], - () => brokerFeatures.setDefaultMinVersionLevels(defaultMinVersionLevelsWithNonExistingFeature)) - - val defaultMinVersionLevelsWithInvalidSmallValue = Map[String, Short]( - "test_feature_1" -> 2, - "test_feature_2" -> (supportedFeatures.get("test_feature_2").min() - 1).asInstanceOf[Short]) - assertThrows( - classOf[IllegalArgumentException], - () => brokerFeatures.setDefaultMinVersionLevels(defaultMinVersionLevelsWithInvalidSmallValue)) - - val defaultMinVersionLevelsWithInvalidLargeValue = Map[String, Short]( - "test_feature_1" -> 2, - "test_feature_2" -> (supportedFeatures.get("test_feature_2").max() + 1).asInstanceOf[Short]) - assertThrows( - classOf[IllegalArgumentException], - () => brokerFeatures.setDefaultMinVersionLevels(defaultMinVersionLevelsWithInvalidLargeValue)) - } - @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) + "test_feature_1" -> new SupportedVersionRange(1, 2, 4), + "test_feature_2" -> new SupportedVersionRange(1, 3, 3), + "test_feature_3" -> new SupportedVersionRange(3, 5, 7)).asJava) brokerFeatures.setSupportedFeatures(supportedFeatures) - val defaultMinVersionLevels = Map[String, Short]( - "test_feature_1" -> 2, - "test_feature_2" -> 3) - brokerFeatures.setDefaultMinVersionLevels(defaultMinVersionLevels) - val expectedFeatures = Map[String, FinalizedVersionRange]( "test_feature_1" -> new FinalizedVersionRange(2, 4), "test_feature_2" -> new FinalizedVersionRange(3, 3), - "test_feature_3" -> new FinalizedVersionRange(3, 7)) + "test_feature_3" -> new FinalizedVersionRange(5, 7)) assertEquals(Features.finalizedFeatures(expectedFeatures.asJava), brokerFeatures.defaultFinalizedFeatures) } - - @Test - def testDefaultMinVersionLevel(): 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 defaultMinVersionLevels = Map[String, Short]("test_feature_1" -> 2) - brokerFeatures.setDefaultMinVersionLevels(defaultMinVersionLevels) - - assertTrue(brokerFeatures.defaultMinVersionLevel("test_feature_1").isDefined) - assertEquals(2, brokerFeatures.defaultMinVersionLevel("test_feature_1").get) - assertTrue(brokerFeatures.defaultMinVersionLevel("test_feature_2").isDefined) - assertEquals(1, brokerFeatures.defaultMinVersionLevel("test_feature_2").get) - assertTrue(brokerFeatures.defaultMinVersionLevel("test_nonexistent_feature").isEmpty) - } } diff --git a/core/src/test/scala/unit/kafka/server/FinalizedFeatureCacheTest.scala b/core/src/test/scala/unit/kafka/server/FinalizedFeatureCacheTest.scala index bdaaeca4f6080..eb888f9487a47 100644 --- a/core/src/test/scala/unit/kafka/server/FinalizedFeatureCacheTest.scala +++ b/core/src/test/scala/unit/kafka/server/FinalizedFeatureCacheTest.scala @@ -33,7 +33,7 @@ class FinalizedFeatureCacheTest { @Test def testUpdateOrThrowFailedDueToInvalidEpoch(): Unit = { val supportedFeatures = Map[String, SupportedVersionRange]( - "feature_1" -> new SupportedVersionRange(1, 4)) + "feature_1" -> new SupportedVersionRange(1, 2, 4)) val brokerFeatures = BrokerFeatures.createDefault() brokerFeatures.setSupportedFeatures(Features.supportedFeatures(supportedFeatures.asJava)) @@ -60,7 +60,7 @@ class FinalizedFeatureCacheTest { @Test def testUpdateOrThrowFailedDueToInvalidFeatures(): Unit = { val supportedFeatures = Map[String, SupportedVersionRange]( - "feature_1" -> new SupportedVersionRange(1, 1)) + "feature_1" -> new SupportedVersionRange(1, 1, 1)) val brokerFeatures = BrokerFeatures.createDefault() brokerFeatures.setSupportedFeatures(Features.supportedFeatures(supportedFeatures.asJava)) @@ -80,7 +80,7 @@ class FinalizedFeatureCacheTest { @Test def testUpdateOrThrowSuccess(): Unit = { val supportedFeatures = Map[String, SupportedVersionRange]( - "feature_1" -> new SupportedVersionRange(1, 4)) + "feature_1" -> new SupportedVersionRange(1, 2, 4)) val brokerFeatures = BrokerFeatures.createDefault() brokerFeatures.setSupportedFeatures(Features.supportedFeatures(supportedFeatures.asJava)) @@ -98,7 +98,7 @@ class FinalizedFeatureCacheTest { @Test def testClear(): Unit = { val supportedFeatures = Map[String, SupportedVersionRange]( - "feature_1" -> new SupportedVersionRange(1, 4)) + "feature_1" -> new SupportedVersionRange(1, 2, 4)) val brokerFeatures = BrokerFeatures.createDefault() brokerFeatures.setSupportedFeatures(Features.supportedFeatures(supportedFeatures.asJava)) diff --git a/core/src/test/scala/unit/kafka/server/FinalizedFeatureChangeListenerTest.scala b/core/src/test/scala/unit/kafka/server/FinalizedFeatureChangeListenerTest.scala index 324938e26eb14..e67be406921d5 100644 --- a/core/src/test/scala/unit/kafka/server/FinalizedFeatureChangeListenerTest.scala +++ b/core/src/test/scala/unit/kafka/server/FinalizedFeatureChangeListenerTest.scala @@ -33,8 +33,8 @@ class FinalizedFeatureChangeListenerTest extends ZooKeeperTestHarness { private def createBrokerFeatures(): BrokerFeatures = { val supportedFeaturesMap = Map[String, SupportedVersionRange]( - "feature_1" -> new SupportedVersionRange(1, 4), - "feature_2" -> new SupportedVersionRange(1, 3)) + "feature_1" -> new SupportedVersionRange(1, 2, 4), + "feature_2" -> new SupportedVersionRange(1, 2, 3)) val brokerFeatures = BrokerFeatures.createDefault() brokerFeatures.setSupportedFeatures(Features.supportedFeatures(supportedFeaturesMap.asJava)) brokerFeatures @@ -244,7 +244,9 @@ class FinalizedFeatureChangeListenerTest extends ZooKeeperTestHarness { 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) diff --git a/core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala b/core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala index 3929dbdae69a3..0031c1d2780d1 100644 --- a/core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala +++ b/core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala @@ -50,7 +50,7 @@ class UpdateFeaturesTest extends BaseRequestTest { } private def defaultSupportedFeatures(): Features[SupportedVersionRange] = { - Features.supportedFeatures(Utils.mkMap(Utils.mkEntry("feature_1", new SupportedVersionRange(1, 3)))) + Features.supportedFeatures(Utils.mkMap(Utils.mkEntry("feature_1", new SupportedVersionRange(1, 1, 3)))) } private def defaultFinalizedFeatures(): Features[FinalizedVersionRange] = { @@ -85,12 +85,6 @@ class UpdateFeaturesTest extends BaseRequestTest { updateSupportedFeatures(features, Set[KafkaServer]() ++ servers) } - private def updateDefaultMinVersionLevelsInAllBrokers(newMinVersionLevels: Map[String, Short]): Unit = { - servers.foreach(s => { - s.brokerFeatures.setDefaultMinVersionLevels(newMinVersionLevels) - }) - } - private def updateFeatureZNode(features: Features[FinalizedVersionRange]): Int = { val server = serverForId(0).get val newNode = new FeatureZNode(FeatureZNodeStatus.Enabled, features) @@ -112,11 +106,15 @@ class UpdateFeaturesTest extends BaseRequestTest { supported: Features[SupportedVersionRange]): FeatureMetadata = { new FeatureMetadata( finalized.features().asScala.map { - case(name, versionRange) => (name, new org.apache.kafka.clients.admin.FinalizedVersionRange(versionRange.min(), versionRange.max())) + case(name, versionRange) => + (name, new org.apache.kafka.clients.admin.FinalizedVersionRange(versionRange.min(), + versionRange.max())) }.asJava, Optional.of(epoch), supported.features().asScala.map { - case(name, versionRange) => (name, new org.apache.kafka.clients.admin.SupportedVersionRange(versionRange.min(), versionRange.max())) + case(name, versionRange) => + (name, new org.apache.kafka.clients.admin.SupportedVersionRange(versionRange.min(), + versionRange.max())) }.asJava) } @@ -182,15 +180,15 @@ class UpdateFeaturesTest extends BaseRequestTest { val versionBefore = updateFeatureZNode(defaultFinalizedFeatures()) val nodeBefore = getFeatureZNode() - val updates = new FeatureUpdateKeyCollection() - val update = new UpdateFeaturesRequestData.FeatureUpdateKey(); - update.setFeature("feature_1"); - update.setMaxVersionLevel(defaultSupportedFeatures().get("feature_1").max()) - update.setAllowDowngrade(false) - updates.add(update) + 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(updates)).build(), + new UpdateFeaturesRequest.Builder(new UpdateFeaturesRequestData().setFeatureUpdates(validUpdates)).build(), notControllerSocketServer) assertEquals(Errors.NOT_CONTROLLER, Errors.forCode(response.data.errorCode())) @@ -213,9 +211,10 @@ class UpdateFeaturesTest extends BaseRequestTest { */ @Test def testShouldFailRequestWhenDowngradeFlagIsNotSetDuringDowngrade(): Unit = { + val targetMaxVersionLevel = (defaultFinalizedFeatures().get("feature_1").max() - 1).asInstanceOf[Short] testWithInvalidFeatureUpdate[InvalidRequestException]( "feature_1", - new FeatureUpdate((defaultFinalizedFeatures().get("feature_1").max() - 1).asInstanceOf[Short],false), + new FeatureUpdate(targetMaxVersionLevel,false), ".*Can not downgrade finalized feature.*allowDowngrade.*".r) } @@ -225,9 +224,10 @@ class UpdateFeaturesTest extends BaseRequestTest { */ @Test def testShouldFailRequestWhenDowngradeToHigherVersionLevelIsAttempted(): Unit = { + val targetMaxVersionLevel = (defaultFinalizedFeatures().get("feature_1").max() + 1).asInstanceOf[Short] testWithInvalidFeatureUpdate[InvalidRequestException]( "feature_1", - new FeatureUpdate(defaultSupportedFeatures().get("feature_1").max(), true), + new FeatureUpdate(targetMaxVersionLevel, true), ".*When the allowDowngrade flag set in the request, the provided maxVersionLevel:3.*existing maxVersionLevel:2.*".r) } @@ -245,18 +245,18 @@ class UpdateFeaturesTest extends BaseRequestTest { val adminClient = createAdminClient() val nodeBefore = getFeatureZNode() - val updates + val invalidUpdates = new UpdateFeaturesRequestData.FeatureUpdateKeyCollection(); - val update = new UpdateFeaturesRequestData.FeatureUpdateKey(); - update.setFeature("feature_1") - update.setMaxVersionLevel(0) - update.setAllowDowngrade(false) - updates.add(update); + val invalidUpdate = new UpdateFeaturesRequestData.FeatureUpdateKey(); + invalidUpdate.setFeature("feature_1") + invalidUpdate.setMaxVersionLevel(0) + invalidUpdate.setAllowDowngrade(false) + invalidUpdates.add(invalidUpdate); val requestData = new UpdateFeaturesRequestData() - requestData.setFeatureUpdates(updates); + requestData.setFeatureUpdates(invalidUpdates); val response = connectAndReceive[UpdateFeaturesResponse]( - new UpdateFeaturesRequest.Builder(new UpdateFeaturesRequestData().setFeatureUpdates(updates)).build(), + new UpdateFeaturesRequest.Builder(new UpdateFeaturesRequestData().setFeatureUpdates(invalidUpdates)).build(), controllerSocketServer) assertEquals(1, response.data().results().size()) @@ -291,99 +291,111 @@ class UpdateFeaturesTest extends BaseRequestTest { */ @Test def testShouldFailRequestWhenUpgradingToSameVersionLevel(): Unit = { + val targetMaxVersionLevel = defaultFinalizedFeatures().get("feature_1").max() testWithInvalidFeatureUpdate[InvalidRequestException]( "feature_1", - new FeatureUpdate(defaultFinalizedFeatures().get("feature_1").max(), false), + new FeatureUpdate(targetMaxVersionLevel, false), ".*Can not upgrade a finalized feature.*to the same value.*".r) } - private def testShouldFailRequestWhenNewMaxVersionLevelIsBelowDefaultMinVersionLevel( - minVersionLevel: Short, - initialMaxVersionLevel: Option[Short], - allowDowngrade: Boolean + private def testShouldFailRequestWhenNewMaxVersionLevelIsBelowFirstActiveVersion( + featureName: String, + supportedVersionRange: SupportedVersionRange, + initialFinalizedVersionRange: Option[FinalizedVersionRange], + allowDowngrade: Boolean ): Unit = { TestUtils.waitUntilControllerElected(zkClient) - updateSupportedFeaturesInAllBrokers(defaultSupportedFeatures()) - val initialFinalizedFeatures = initialMaxVersionLevel.map( - maxVersionLevel => Features.finalizedFeatures( - Utils.mkMap(Utils.mkEntry("feature_1", new FinalizedVersionRange(minVersionLevel, maxVersionLevel)))) + val supportedFeatures = Features.supportedFeatures(Utils.mkMap(Utils.mkEntry(featureName, supportedVersionRange))) + updateSupportedFeaturesInAllBrokers(supportedFeatures) + val initialFinalizedFeatures = initialFinalizedVersionRange.map( + versionRange => Features.finalizedFeatures(Utils.mkMap(Utils.mkEntry(featureName, versionRange))) ).getOrElse(Features.emptyFinalizedFeatures()) - updateDefaultMinVersionLevelsInAllBrokers(Map[String, Short]("feature_1" -> minVersionLevel)) val versionBefore = updateFeatureZNode(initialFinalizedFeatures) - val newMaxVersionLevel = (minVersionLevel - 1).asInstanceOf[Short] + val newMaxVersionLevel = (supportedVersionRange.firstActiveVersion() - 1).asInstanceOf[Short] val update = new FeatureUpdate(newMaxVersionLevel, allowDowngrade) val adminClient = createAdminClient() val nodeBefore = getFeatureZNode() - val result = adminClient.updateFeatures( - Utils.mkMap(Utils.mkEntry("feature_1", update)), new UpdateFeaturesOptions()) + val result = adminClient.updateFeatures(Utils.mkMap(Utils.mkEntry(featureName, update)), new UpdateFeaturesOptions()) checkException[InvalidRequestException]( result, - Map("feature_1" -> s".*maxVersionLevel:$newMaxVersionLevel.*minVersionLevel:$minVersionLevel.*".r)) + Map(featureName -> s".*maxVersionLevel:$newMaxVersionLevel.*".r)) checkFeatures( adminClient, nodeBefore, - makeFeatureMetadata(initialFinalizedFeatures, versionBefore, defaultSupportedFeatures())) + makeFeatureMetadata(initialFinalizedFeatures, versionBefore, supportedFeatures)) } /** - * Tests that an UpdateFeatures request fails in the Controller, when, a feature version level - * downgrade is attempted to a version level thats below the default min version level for the - * feature. + * Tests that an UpdateFeatures request fails in the Controller, when, for an existing finalized + * feature, a version level DOWNGRADE is attempted to a value thats below the first active version + * for the feature. */ @Test - def testShouldFailRequestWhenDowngradingBelowMinVersionLevel(): Unit = { - testShouldFailRequestWhenNewMaxVersionLevelIsBelowDefaultMinVersionLevel(2, Some(2.asInstanceOf[Short]), true) + def testShouldFailRequestWhenDowngradingBelowFirstActiveVersionOfAnExistingFinalizedFeature(): Unit = { + testShouldFailRequestWhenNewMaxVersionLevelIsBelowFirstActiveVersion( + "feature_1", new SupportedVersionRange(1, 2, 3), Some(new FinalizedVersionRange(2, 3)), true) } /** - * Tests that an UpdateFeatures request fails in the Controller, when for a non-existing feature, - * a version level upgrade is attempted to a value thats below the default min version - * level for the feature. + * Tests that an UpdateFeatures request fails in the Controller, when, for a non-existing finalized + * feature, a version level UPGRADE is attempted to a value thats below the first active version + * for the feature. */ @Test - def testShouldFailRequestWhenUpgradingBelowDefaultMinVersionLevelOfANonExistingFinalizedFeature(): Unit = { - testShouldFailRequestWhenNewMaxVersionLevelIsBelowDefaultMinVersionLevel(2, Option.empty, false) + def testShouldFailRequestWhenUpgradingBelowFirstActiveVersionOfANonExistingFinalizedFeature(): Unit = { + testShouldFailRequestWhenNewMaxVersionLevelIsBelowFirstActiveVersion( + "feature_1", new SupportedVersionRange(1, 2, 3), Option.empty, false) } /** - * Tests that an UpdateFeatures request fails in the Controller, when for a non-existing feature, - * a version level downgrade is attempted to a value thats below the default min version + * Tests that an UpdateFeatures request fails in the Controller, when for a non-existing finalized + * feature, a version level DOWNGRADE is attempted to a value thats below the first active version * level for the feature. */ @Test - def testShouldFailRequestWhenDowngradingBelowDefaultMinVersionLevelOfANonExistingFinalizedFeature(): Unit = { - testShouldFailRequestWhenNewMaxVersionLevelIsBelowDefaultMinVersionLevel(2, Option.empty, true) + def testShouldFailRequestWhenDowngradingBelowFirstActiveVersionOfANonExistingFinalizedFeature(): Unit = { + testShouldFailRequestWhenNewMaxVersionLevelIsBelowFirstActiveVersion( + "feature_1", new SupportedVersionRange(1, 2, 3), Option.empty, true) } - - private def testShouldFailRequestDuringBrokerMaxVersionLevelIncompatibility(initialFinalizedFeatures: Features[FinalizedVersionRange]): Unit = { + 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} - val unsupportedBrokers = Set[KafkaServer](nonControllerServers.head) - val supportedBrokers = Set[KafkaServer](nonControllerServers(1), controller) + // 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) - updateSupportedFeatures(defaultSupportedFeatures(), supportedBrokers) + val supportedFeatures = Features.supportedFeatures(Utils.mkMap(Utils.mkEntry(featureName, supportedVersionRange))) + updateSupportedFeatures(supportedFeatures, versionCompatibleBrokers) - val validMinVersion = defaultSupportedFeatures().get("feature_1").min() - val unsupportedMaxVersion = - (defaultSupportedFeatures().get("feature_1").max() - 1).asInstanceOf[Short] - val badSupportedFeatures = Features.supportedFeatures( + val unsupportedMaxVersion = (supportedVersionRange.max() - 1).asInstanceOf[Short] + val supportedFeaturesWithVersionIncompatibility = Features.supportedFeatures( Utils.mkMap( Utils.mkEntry("feature_1", new SupportedVersionRange( - validMinVersion, + supportedVersionRange.min(), + supportedVersionRange.firstActiveVersion(), unsupportedMaxVersion)))) - updateSupportedFeatures(badSupportedFeatures, unsupportedBrokers) + 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(defaultSupportedFeatures().get("feature_1").max(), false) + val invalidUpdate = new FeatureUpdate(supportedVersionRange.max(), false) val nodeBefore = getFeatureZNode() val adminClient = createAdminClient() val result = adminClient.updateFeatures( @@ -394,7 +406,7 @@ class UpdateFeaturesTest extends BaseRequestTest { checkFeatures( adminClient, nodeBefore, - makeFeatureMetadata(initialFinalizedFeatures, versionBefore, defaultSupportedFeatures())) + makeFeatureMetadata(initialFinalizedFeatures, versionBefore, supportedFeatures)) } /** @@ -404,7 +416,11 @@ class UpdateFeaturesTest extends BaseRequestTest { */ @Test def testShouldFailRequestDuringBrokerMaxVersionLevelIncompatibilityForExistingFinalizedFeature(): Unit = { - testShouldFailRequestDuringBrokerMaxVersionLevelIncompatibility(defaultFinalizedFeatures()) + val feature = "feature_1" + testShouldFailRequestDuringBrokerMaxVersionLevelIncompatibility( + feature, + defaultSupportedFeatures().get(feature), + Some(defaultFinalizedFeatures().get(feature))) } /** @@ -414,7 +430,11 @@ class UpdateFeaturesTest extends BaseRequestTest { */ @Test def testShouldFailRequestDuringBrokerMaxVersionLevelIncompatibilityWithNoExistingFinalizedFeature(): Unit = { - testShouldFailRequestDuringBrokerMaxVersionLevelIncompatibility(Features.emptyFinalizedFeatures()) + val feature = "feature_1" + testShouldFailRequestDuringBrokerMaxVersionLevelIncompatibility( + feature, + defaultSupportedFeatures().get(feature), + Option.empty) } /** @@ -425,12 +445,12 @@ class UpdateFeaturesTest extends BaseRequestTest { def testSuccessfulFeatureUpgradeAndWithNoExistingFinalizedFeatures(): Unit = { TestUtils.waitUntilControllerElected(zkClient) - updateSupportedFeaturesInAllBrokers( + val supportedFeatures = Features.supportedFeatures( Utils.mkMap( - Utils.mkEntry("feature_1", new SupportedVersionRange(1, 3)), - Utils.mkEntry("feature_2", new SupportedVersionRange(2, 5))))) - updateDefaultMinVersionLevelsInAllBrokers(Map[String, Short]("feature_1" -> 1, "feature_2" -> 2)) + Utils.mkEntry("feature_1", new SupportedVersionRange(1, 1, 3)), + Utils.mkEntry("feature_2", new SupportedVersionRange(2, 2, 5)))) + updateSupportedFeaturesInAllBrokers(supportedFeatures) val versionBefore = updateFeatureZNode(Features.emptyFinalizedFeatures()) val targetFinalizedFeatures = Features.finalizedFeatures( @@ -440,13 +460,7 @@ class UpdateFeaturesTest extends BaseRequestTest { val update1 = new FeatureUpdate(targetFinalizedFeatures.get("feature_1").max(), false) val update2 = new FeatureUpdate(targetFinalizedFeatures.get("feature_2").max(), false) - val expected = makeFeatureMetadata( - targetFinalizedFeatures, - versionBefore + 1, - Features.supportedFeatures( - Utils.mkMap( - Utils.mkEntry("feature_1", new SupportedVersionRange(1, 3)), - Utils.mkEntry("feature_2", new SupportedVersionRange(2, 5))))) + val expected = makeFeatureMetadata(targetFinalizedFeatures, versionBefore + 1, supportedFeatures) val adminClient = createAdminClient() adminClient.updateFeatures( @@ -468,18 +482,20 @@ class UpdateFeaturesTest extends BaseRequestTest { def testSuccessfulFeatureUpgradeAndDowngrade(): Unit = { TestUtils.waitUntilControllerElected(zkClient) - updateSupportedFeaturesInAllBrokers( - Features.supportedFeatures( - Utils.mkMap( - Utils.mkEntry("feature_1", new SupportedVersionRange(1, 3)), - Utils.mkEntry("feature_2", new SupportedVersionRange(2, 5))))) - updateDefaultMinVersionLevelsInAllBrokers(Map[String, Short]("feature_1" -> 1, "feature_2" -> 2)) - val versionBefore = updateFeatureZNode( - Features.finalizedFeatures( - Utils.mkMap( - Utils.mkEntry("feature_1", new FinalizedVersionRange(1, 2)), - Utils.mkEntry("feature_2", new FinalizedVersionRange(2, 4))))) + val supportedFeatures = Features.supportedFeatures( + Utils.mkMap( + Utils.mkEntry("feature_1", new SupportedVersionRange(1, 1, 3)), + Utils.mkEntry("feature_2", new SupportedVersionRange(2, 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)), @@ -487,13 +503,7 @@ class UpdateFeaturesTest extends BaseRequestTest { val update1 = new FeatureUpdate(targetFinalizedFeatures.get("feature_1").max(), false) val update2 = new FeatureUpdate(targetFinalizedFeatures.get("feature_2").max(), true) - val expected = makeFeatureMetadata( - targetFinalizedFeatures, - versionBefore + 1, - Features.supportedFeatures( - Utils.mkMap( - Utils.mkEntry("feature_1", new SupportedVersionRange(1, 3)), - Utils.mkEntry("feature_2", new SupportedVersionRange(2, 5))))) + val expected = makeFeatureMetadata(targetFinalizedFeatures, versionBefore + 1, supportedFeatures) val adminClient = createAdminClient() adminClient.updateFeatures( @@ -516,18 +526,21 @@ class UpdateFeaturesTest extends BaseRequestTest { def testPartialSuccessDuringValidFeatureUpgradeAndInvalidDowngrade(): Unit = { TestUtils.waitUntilControllerElected(zkClient) - val initialSupportedFeatures = Features.supportedFeatures( + val supportedFeatures = Features.supportedFeatures( Utils.mkMap( - Utils.mkEntry("feature_1", new SupportedVersionRange(1, 3)), - Utils.mkEntry("feature_2", new SupportedVersionRange(2, 5)))) - updateSupportedFeaturesInAllBrokers(initialSupportedFeatures) - updateDefaultMinVersionLevelsInAllBrokers(Map[String, Short]("feature_1" -> 1, "feature_2" -> 2)) + Utils.mkEntry("feature_1", new SupportedVersionRange(1, 1, 3)), + Utils.mkEntry("feature_2", new SupportedVersionRange(2, 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)), @@ -552,7 +565,7 @@ class UpdateFeaturesTest extends BaseRequestTest { checkFeatures( adminClient, FeatureZNode(FeatureZNodeStatus.Enabled, expectedFeatures), - makeFeatureMetadata(expectedFeatures, versionBefore + 1, initialSupportedFeatures)) + makeFeatureMetadata(expectedFeatures, versionBefore + 1, supportedFeatures)) } /** @@ -566,22 +579,23 @@ class UpdateFeaturesTest extends BaseRequestTest { val controller = servers.filter { server => server.kafkaController.isActive}.head val nonControllerServers = servers.filter { server => !server.kafkaController.isActive} - val unsupportedBrokers = Set[KafkaServer](nonControllerServers.head) - val supportedBrokers = Set[KafkaServer](nonControllerServers(1), controller) + // 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 initialSupportedFeatures = Features.supportedFeatures( + val supportedFeatures = Features.supportedFeatures( Utils.mkMap( - Utils.mkEntry("feature_1", new SupportedVersionRange(1, 3)), - Utils.mkEntry("feature_2", new SupportedVersionRange(2, 5)))) - updateSupportedFeatures(initialSupportedFeatures, supportedBrokers) + Utils.mkEntry("feature_1", new SupportedVersionRange(1, 2, 3)), + Utils.mkEntry("feature_2", new SupportedVersionRange(2, 2, 5)))) + updateSupportedFeatures(supportedFeatures, versionCompatibleBrokers) - val badSupportedFeatures = Features.supportedFeatures( + val supportedFeaturesWithVersionIncompatibility = Features.supportedFeatures( Utils.mkMap( - Utils.mkEntry("feature_1", new SupportedVersionRange(1, 2)), - Utils.mkEntry("feature_2", new SupportedVersionRange(2, 5)))) - updateSupportedFeatures(badSupportedFeatures, unsupportedBrokers) - - updateDefaultMinVersionLevelsInAllBrokers(Map[String, Short]("feature_1" -> 1, "feature_2" -> 2)) + Utils.mkEntry("feature_1", new SupportedVersionRange(1, 2, 2)), + Utils.mkEntry("feature_2", supportedFeatures.get("feature_2")))) + updateSupportedFeatures(supportedFeaturesWithVersionIncompatibility, brokersWithVersionIncompatibility) val initialFinalizedFeatures = Features.finalizedFeatures( Utils.mkMap( @@ -589,6 +603,10 @@ class UpdateFeaturesTest extends BaseRequestTest { 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)), @@ -612,6 +630,6 @@ class UpdateFeaturesTest extends BaseRequestTest { checkFeatures( adminClient, FeatureZNode(FeatureZNodeStatus.Enabled, expectedFeatures), - makeFeatureMetadata(expectedFeatures, versionBefore + 1, initialSupportedFeatures)) + makeFeatureMetadata(expectedFeatures, versionBefore + 1, supportedFeatures)) } } diff --git a/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala b/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala index 67445004c0f8c..1fd33653e68cc 100644 --- a/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala +++ b/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala @@ -773,12 +773,12 @@ class KafkaZkClientTest extends ZooKeeperTestHarness { rack = None, features = Features.supportedFeatures( Map[String, SupportedVersionRange]( - "feature1" -> new SupportedVersionRange(1, 2)).asJava)) + "feature1" -> new SupportedVersionRange(1, 1, 2)).asJava)) val differentBrokerInfoWithSameId = createBrokerInfo( 1, "test.host2", 9995, SecurityProtocol.SSL, features = Features.supportedFeatures( Map[String, SupportedVersionRange]( - "feature2" -> new SupportedVersionRange(4, 7)).asJava)) + "feature2" -> new SupportedVersionRange(4, 4, 7)).asJava)) zkClient.registerBroker(brokerInfo) assertEquals(Some(brokerInfo.broker), zkClient.getBroker(1)) @@ -832,12 +832,12 @@ class KafkaZkClientTest extends ZooKeeperTestHarness { 0, "test.host0", 9998, SecurityProtocol.PLAINTEXT, features = Features.supportedFeatures( Map[String, SupportedVersionRange]( - "feature1" -> new SupportedVersionRange(1, 2)).asJava)) + "feature1" -> new SupportedVersionRange(1, 1, 2)).asJava)) val brokerInfo1 = createBrokerInfo( 1, "test.host1", 9999, SecurityProtocol.SSL, features = Features.supportedFeatures( Map[String, SupportedVersionRange]( - "feature2" -> new SupportedVersionRange(3, 6)).asJava)) + "feature2" -> new SupportedVersionRange(3, 3, 6)).asJava)) zkClient.registerBroker(brokerInfo1) otherZkClient.registerBroker(brokerInfo0) From 3342f147a4f9ef3a73f6e9264551519cdcaeb687 Mon Sep 17 00:00:00 2001 From: Kowshik Prakasam Date: Mon, 28 Sep 2020 01:18:47 -0700 Subject: [PATCH 31/41] Small doc change --- core/src/main/scala/kafka/server/BrokerFeatures.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/server/BrokerFeatures.scala b/core/src/main/scala/kafka/server/BrokerFeatures.scala index 4b07bf43fb97b..1592af7fa6c1c 100644 --- a/core/src/main/scala/kafka/server/BrokerFeatures.scala +++ b/core/src/main/scala/kafka/server/BrokerFeatures.scala @@ -74,7 +74,7 @@ import scala.jdk.CollectionConverters._ * * STEP 2: * After the first step is over, you may (at some point) want to permanently remove the code/logic - * for the functionality offered by the deprecated feature. This is the second step. Here a + * for the functionality offered by the deprecated feature versions. This is the second step. Here a * subsequent major Kafka release is made with another Broker code change that removes the code for * the functionality offered by the deprecated feature versions. This would completely drop support * for the deprecated versions. Such a code change needs to be supplemented by supplying a From 3c59a17771de6ecbe89a93cc8ab1410e416a93fe Mon Sep 17 00:00:00 2001 From: Kowshik Prakasam Date: Tue, 29 Sep 2020 02:23:52 -0700 Subject: [PATCH 32/41] Address comments from Jun --- .../org/apache/kafka/clients/admin/Admin.java | 1 - .../admin}/FeatureUpdate.java | 2 +- .../kafka/clients/admin/KafkaAdminClient.java | 27 +++- .../common/requests/ApiVersionsResponse.java | 2 +- .../requests/UpdateFeaturesRequest.java | 23 ---- .../common/message/ApiVersionsResponse.json | 2 +- .../clients/admin/KafkaAdminClientTest.java | 1 - .../kafka/clients/admin/MockAdminClient.java | 1 - .../kafka/controller/KafkaController.scala | 126 ++++++++++-------- .../scala/kafka/server/BrokerFeatures.scala | 42 +++--- .../FinalizedFeatureChangeListener.scala | 14 +- .../main/scala/kafka/server/KafkaApis.scala | 42 +++--- .../main/scala/kafka/server/KafkaConfig.scala | 2 +- .../main/scala/kafka/server/KafkaServer.scala | 2 +- .../kafka/server/UpdateFeaturesTest.scala | 19 +-- 15 files changed, 159 insertions(+), 147 deletions(-) rename clients/src/main/java/org/apache/kafka/{common/requests => clients/admin}/FeatureUpdate.java (98%) 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 a61c6ef1bdef5..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 @@ -40,7 +40,6 @@ 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.FeatureUpdate; import org.apache.kafka.common.requests.LeaveGroupResponse; /** diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FeatureUpdate.java b/clients/src/main/java/org/apache/kafka/clients/admin/FeatureUpdate.java similarity index 98% rename from clients/src/main/java/org/apache/kafka/common/requests/FeatureUpdate.java rename to clients/src/main/java/org/apache/kafka/clients/admin/FeatureUpdate.java index 5aea478e087cd..f43d63ed35234 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/FeatureUpdate.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/FeatureUpdate.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.kafka.common.requests; +package org.apache.kafka.clients.admin; import java.util.Objects; 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 02ea9fcf844e7..d82b0ff97fd75 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 @@ -145,6 +145,7 @@ 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; @@ -208,7 +209,6 @@ import org.apache.kafka.common.requests.ElectLeadersResponse; import org.apache.kafka.common.requests.ExpireDelegationTokenRequest; import org.apache.kafka.common.requests.ExpireDelegationTokenResponse; -import org.apache.kafka.common.requests.FeatureUpdate; import org.apache.kafka.common.requests.FindCoordinatorRequest; import org.apache.kafka.common.requests.FindCoordinatorRequest.CoordinatorType; import org.apache.kafka.common.requests.FindCoordinatorResponse; @@ -4402,8 +4402,8 @@ void handleFailure(Throwable throwable) { } @Override - public UpdateFeaturesResult updateFeatures( - final Map featureUpdates, final UpdateFeaturesOptions options) { + public UpdateFeaturesResult updateFeatures(final Map featureUpdates, + final UpdateFeaturesOptions options) { if (featureUpdates.isEmpty()) { throw new IllegalArgumentException("Feature updates can not be null or empty."); } @@ -4418,7 +4418,26 @@ public UpdateFeaturesResult updateFeatures( @Override UpdateFeaturesRequest.Builder createRequest(int timeoutMs) { - return new UpdateFeaturesRequest.Builder(UpdateFeaturesRequest.create(featureUpdates, 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 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 61be347f1144e..4b7a7a7cd5735 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 long UNKNOWN_FINALIZED_FEATURES_EPOCH = -1; + public static final long UNKNOWN_FINALIZED_FEATURES_EPOCH = -1L; public static final ApiVersionsResponse DEFAULT_API_VERSIONS_RESPONSE = createApiVersionsResponse( 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 index 008a9f37ffa09..3276c0abbe438 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesRequest.java @@ -17,7 +17,6 @@ package org.apache.kafka.common.requests; import java.nio.ByteBuffer; -import java.util.Map; import org.apache.kafka.common.message.UpdateFeaturesRequestData.FeatureUpdateKey; import org.apache.kafka.common.message.UpdateFeaturesResponseData; import org.apache.kafka.common.message.UpdateFeaturesRequestData; @@ -93,26 +92,4 @@ public static UpdateFeaturesRequest parse(ByteBuffer buffer, short version) { public static boolean isDeleteRequest(UpdateFeaturesRequestData.FeatureUpdateKey update) { return update.maxVersionLevel() < 1 && update.allowDowngrade(); } - - public static UpdateFeaturesRequestData create(Map featureUpdates, 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 UpdateFeaturesRequestData() - .setTimeoutMs(timeoutMs) - .setFeatureUpdates(featureUpdatesRequestData); - } } diff --git a/clients/src/main/resources/common/message/ApiVersionsResponse.json b/clients/src/main/resources/common/message/ApiVersionsResponse.json index e56ba7a0c4c36..ba6f01cb9434a 100644 --- a/clients/src/main/resources/common/message/ApiVersionsResponse.json +++ b/clients/src/main/resources/common/message/ApiVersionsResponse.json @@ -55,7 +55,7 @@ "about": "The maximum supported version for the feature." } ] }, - {"name": "FinalizedFeaturesEpoch", "type": "int64", "versions": "3+", + { "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", 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 0ca0dc6fa680a..74bd860c9d1d6 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 @@ -143,7 +143,6 @@ import org.apache.kafka.common.requests.DescribeLogDirsResponse; import org.apache.kafka.common.requests.DescribeUserScramCredentialsResponse; import org.apache.kafka.common.requests.ElectLeadersResponse; -import org.apache.kafka.common.requests.FeatureUpdate; import org.apache.kafka.common.requests.FindCoordinatorResponse; import org.apache.kafka.common.requests.IncrementalAlterConfigsResponse; import org.apache.kafka.common.requests.JoinGroupRequest; 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 a5bebd4c6b85e..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 @@ -51,7 +51,6 @@ import java.util.Map; import java.util.Optional; import java.util.Set; -import org.apache.kafka.common.requests.FeatureUpdate; public class MockAdminClient extends AdminClient { public static final String DEFAULT_CLUSTER_ID = "I4ZmrWqfT2e-upky_4fdPA"; diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index d9607cd97a437..340c319d1a550 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -226,7 +226,7 @@ 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 = { - setupFeatureVersioning() + maybeSetupFeatureVersioning() info("Registering handlers") @@ -329,7 +329,7 @@ class KafkaController(val config: KafkaConfig, * 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 - * default supported features as its finalized features. + * supported features as its finalized features. * * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0: * Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the @@ -342,15 +342,19 @@ class KafkaController(val config: KafkaConfig, * 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 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. + * 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. 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. + * 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 untouched. * * 3. Broker binary upgraded, with existing cluster IBP config >= KAFKA_2_7_IV0: * Imagine an existing Kafka cluster with IBP config >= KAFKA_2_7_IV0, and the broker binary @@ -359,14 +363,15 @@ class KafkaController(val config: KafkaConfig, * status and existing finalized features. In such a case, the controller needs to scan the * existing finalized features and mutate them for the purpose of version level deprecation * (if needed). - * This is how we handle this case: If an existing finalized feature is present in the default - * finalized features, then, its existing minimum version level is updated to the default - * minimum version level maintained in the BrokerFeatures object. The goal of this mutation is - * to permanently deprecate one or more feature version levels. The range of feature version - * levels deprecated are from the closed range: [existing_min_version_level, default_min_version_level]. - * NOTE: Deprecating a feature version level is an incompatible change, which requires a major - * release of Kafka. In such a release, the minimum version level maintained within the - * BrokerFeatures class is updated suitably to record the deprecation of the feature. + * This is how we handle this case: If an existing finalized feature is defined in the list of + * supported features (maintained in the BrokerFeatures object), then, the existing + * minimum version level is updated to the first active version for that feature. The goal of + * this mutation is to permanently deprecate one or more feature version levels. The range of + * feature version levels deprecated are from the closed range: + * [existing_min_version_level, first_active_version - 1]. + * NOTE: Deprecating a feature version level is a two-step process and involves incompatible + * changes. Please read the documentation for the kafka.server.BrokerFeatures class to learn + * about the deprecation process. * * 4. Broker downgrade: * Imagine that a Kafka cluster exists already and the IBP config is greater than or equal to @@ -376,55 +381,67 @@ class KafkaController(val config: KafkaConfig, * will switch the FeatureZNode status to disabled with empty features. */ private def enableFeatureVersioning(): Unit = { - val defaultFinalizedFeatures = brokerFeatures.defaultFinalizedFeatures + val supportedFeatures = brokerFeatures.supportedFeatures val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path) if (version == ZkVersion.UnknownVersion) { - val newVersion = createFeatureZNode(new FeatureZNode(FeatureZNodeStatus.Enabled, defaultFinalizedFeatures)) + val finalizedFeatures = supportedFeatures.features.asScala.map { + case(name, versionRange) => (name, new FinalizedVersionRange(versionRange.firstActiveVersion, versionRange.max)) + }.asJava + val newVersion = createFeatureZNode(new FeatureZNode(FeatureZNodeStatus.Enabled, + Features.finalizedFeatures(finalizedFeatures))) featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs) } else { val existingFeatureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get) var newFeatures: Features[FinalizedVersionRange] = Features.emptyFinalizedFeatures() if (existingFeatureZNode.status.equals(FeatureZNodeStatus.Enabled)) { newFeatures = Features.finalizedFeatures(existingFeatureZNode.features.features().asScala.map { - case (featureName, existingVersionRange) => - val brokerDefaultVersionRange = defaultFinalizedFeatures.get(featureName) - if (brokerDefaultVersionRange == null) { - warn(s"Existing finalized feature: $featureName with $existingVersionRange" - + s" is absent in default finalized $defaultFinalizedFeatures") - (featureName, existingVersionRange) - } else if (brokerDefaultVersionRange.max() >= existingVersionRange.max() && - brokerDefaultVersionRange.min() <= existingVersionRange.max()) { + case (featureName, finalizedVersionRange) => + val supportedVersionRange = supportedFeatures.get(featureName) + if (supportedVersionRange == null) { + warn(s"Existing finalized feature: $featureName with $finalizedVersionRange" + + s" is absent in supported $supportedFeatures") + (featureName, finalizedVersionRange) + } else if (supportedVersionRange.max() >= finalizedVersionRange.max() && + supportedVersionRange.firstActiveVersion() <= finalizedVersionRange.max()) { // Using the change below, we deprecate all version levels in the range: - // [existingVersionRange.min(), brokerDefaultVersionRange.min() - 1]. + // [finalizedVersionRange.min(), supportedVersionRange.firstActiveVersion() - 1]. // - // NOTE: if existingVersionRange.min() equals brokerDefaultVersionRange.min(), then - // we do not deprecate any version levels (since there is none to be deprecated). + // NOTE: if finalizedVersionRange.min() equals supportedVersionRange.firstActiveVersion(), + // then we do not deprecate any version levels (since there is none to be deprecated). // // Examples: - // 1. brokerDefaultVersionRange = [4, 7] and existingVersionRange = [1, 5]. + // + // 1. supportedVersionRange = [minVersion=1, firstActiveVersion=4, maxVersion=7] + // and + // finalizedVersionRange = [minVersionLevel=1, maxVersionLevel=5]. + // // In this case, we deprecate all version levels in the range: [1, 3]. - // 2. brokerDefaultVersionRange = [4, 7] and existingVersionRange = [4, 5]. + // + // 2. supportedVersionRange = [minVersion=1, firstActiveVersion=4, maxVersion=7] + // and + // finalizedVersionRange = [minVersionLevel=4, maxVersionLevel=5]. + // // In this case, we do not deprecate any version levels since - // brokerDefaultVersionRange.min() equals existingVersionRange.min(). - (featureName, new FinalizedVersionRange(brokerDefaultVersionRange.min(), existingVersionRange.max())) + // supportedVersionRange.min() equals finalizedVersionRange.min(). + (featureName, new FinalizedVersionRange(supportedVersionRange.min(), + finalizedVersionRange.max())) } else { // This is a serious error. We should never be reaching here, since we already // verify once during KafkaServer startup that existing finalized feature versions in // the FeatureZNode contained no incompatibilities. If we are here, it means that one // of the following is true: - // 1. The existing version levels fall completely outside the range of the default - // finalized version levels (i.e. no intersection), or - // 2. The existing version levels are incompatible with default finalized version - // levels. + // 1. The existing version levels fall completely outside the range of the supported + // version range (i.e. no intersection), or + // 2. The existing version levels are incompatible with the supported version range. // // Examples of invalid cases that can cause this exception to be triggered: - // 1. No intersection : brokerDefaultVersionRange = [4, 7] and existingVersionRange = [2, 3]. - // 2. No intersection : brokerDefaultVersionRange = [2, 3] and existingVersionRange = [4, 7]. - // 3. Incompatible versions: brokerDefaultVersionRange = [2, 3] and existingVersionRange = [1, 7]. + // 1. No intersection : supportedVersionRange = [4, 7] and finalizedVersionRange = [2, 3]. + // 2. No intersection : supportedVersionRange = [2, 3] and finalizedVersionRange = [4, 7]. + // 3. Incompatible versions : supportedVersionRange = [2, 3] and finalizedVersionRange = [1, 7]. throw new IllegalStateException( s"Can not update minimum version level in finalized feature: $featureName," - + s" since the existing $existingVersionRange is not eligible for a change" - + s" based on the default $brokerDefaultVersionRange. This should never happen" + + s" since the existing $finalizedVersionRange is not eligible for a change" + + s" based on the default $supportedVersionRange. This should never happen" + s" since feature version incompatibilities are already checked during" + s" Kafka server startup.") } @@ -466,8 +483,8 @@ class KafkaController(val config: KafkaConfig, } } - private def setupFeatureVersioning(): Unit = { - if (config.isFeatureVersioningEnabled) { + private def maybeSetupFeatureVersioning(): Unit = { + if (config.isFeatureVersioningSupported) { enableFeatureVersioning() } else { disableFeatureVersioning() @@ -1571,7 +1588,7 @@ class KafkaController(val config: KafkaConfig, // of the cache are compatible with the supported features of each broker. brokersAndEpochs.partition { case (broker, _) => - !config.isFeatureVersioningEnabled || + !config.isFeatureVersioningSupported || !featureCache.get.exists( latestFinalizedFeatures => BrokerFeatures.hasIncompatibleFeatures(broker.features, latestFinalizedFeatures.features)) @@ -1960,7 +1977,7 @@ class KafkaController(val config: KafkaConfig, * as described above. */ private def validateFeatureUpdate(update: UpdateFeaturesRequestData.FeatureUpdateKey, - existingVersionRange: Option[FinalizedVersionRange]): Either[Option[FinalizedVersionRange], ApiError] = { + 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)) @@ -2044,7 +2061,7 @@ class KafkaController(val config: KafkaConfig, // 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 ApiError(Errors.NONE). + // - 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 @@ -2055,9 +2072,10 @@ class KafkaController(val config: KafkaConfig, updates.asScala.iterator.foreach { update => validateFeatureUpdate(update, existingFeatures.get(update.feature())) match { case Left(newVersionRangeOrNone) => - newVersionRangeOrNone - .map(newVersionRange => targetFeatures += (update.feature() -> newVersionRange)) - .getOrElse(targetFeatures -= update.feature()) + 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) @@ -2072,17 +2090,17 @@ class KafkaController(val config: KafkaConfig, if (!existingFeatures.equals(targetFeatures)) { val newNode = new FeatureZNode(FeatureZNodeStatus.Enabled, Features.finalizedFeatures(targetFeatures.asJava)) val newVersion = zkClient.updateFeatureZNode(newNode) - featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs) + featureCache.waitUntilEpochOrThrow(newVersion, request.data().timeoutMs().min(config.zkConnectionTimeoutMs)) } } 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, - Errors.FEATURE_UPDATE_FAILED.message() + " Error: " + e) + errors(feature) = new ApiError(Errors.FEATURE_UPDATE_FAILED) } } } finally { diff --git a/core/src/main/scala/kafka/server/BrokerFeatures.scala b/core/src/main/scala/kafka/server/BrokerFeatures.scala index 1592af7fa6c1c..249d7880f6979 100644 --- a/core/src/main/scala/kafka/server/BrokerFeatures.scala +++ b/core/src/main/scala/kafka/server/BrokerFeatures.scala @@ -33,46 +33,54 @@ import scala.jdk.CollectionConverters._ * ================================== * * Deprecation of certain version levels of a feature is a process to stop supporting the - * functionality offered by the feature at a those version levels, across the entire Kafka cluster. + * functionality offered by the feature at those version levels, across the entire Kafka cluster. * Feature version deprecation is a simple 2-step process explained below. In each step below, an * example is provided to help understand the process better: * * STEP 1: + * ======= + * * In the first step, a major Kafka release is made with a Broker code change (explained later * below) that establishes the intent to deprecate certain versions of one or more features - * cluster-wide. When this new Kafka release is deployed to the cluster, the feature versioning - * system (via the controller) will automatically persist the new minVersionLevel for the feature in - * Zk to propagate the deprecation of certain versions. After this happens, any external client that - * queries the Broker to learn the feature versions will at some point start to see the new value - * for the finalized minVersionLevel for the feature. This makes the version deprecation permanent. + * cluster-wide. When this new Kafka release is deployed to the cluster, deprecated finalized + * feature versions are no longer advertised to the client, but they can still be used by existing + * connections. The way it works is that the feature versioning system (via the controller) will + * automatically persist the new minVersionLevel for the feature in ZK to propagate the deprecation + * of certain versions. After this happens, any external client that queries the Broker to learn the + * feature versions will at some point start to see the new value for the finalized minVersionLevel + * for the feature. The external clients are expected to stop using the deprecated versions at least + * by the time that they learn about it. * * Here is how the above code change needs to be done: * In order to deprecate feature version levels, in the supportedFeatures map you need to supply a * specific firstActiveVersion value that's higher than the minVersion for the feature. The * value for firstActiveVersion should be 1 beyond the highest version that you intend to deprecate - * for that feature. When features are finalized via the ApiKeys.UPDATE_FEATURES api, the feature - * version levels in the closed range: [minVersion, firstActiveVersion - 1] are automatically - * deprecated in ZK by the controller logic. + * for that feature. Whenever the controller is elected or the features are finalized via the + * ApiKeys.UPDATE_FEATURES api, the feature version levels in the closed range: + * [minVersion, firstActiveVersion - 1] are automatically deprecated in ZK by the controller logic. + * * Example: * - Let us assume the existing finalized feature in ZK: * { - * "feature_1" -> FinalizedVersionRange(minVersionLevel=1, maxVersionLevel=5) + * "feature" -> FinalizedVersionRange(minVersionLevel=1, maxVersionLevel=5) * } * Now, supposing you would like to deprecate feature version levels: [1, 2]. * Then, in the supportedFeatures map you should supply the following: * supportedFeatures = { - * "feature1" -> SupportedVersionRange(minVersion=1, firstActiveVersion=3, maxVersion=5) + * "feature" -> SupportedVersionRange(minVersion=1, firstActiveVersion=3, maxVersion=5) * } - * - If you do NOT want to deprecate a version level for a feature, then in the supportedFeatures + * - If you do NOT want to deprecate a version level for a feature, then, in the supportedFeatures * map you should supply the firstActiveVersion to be the same as the minVersion supplied for that * feature. * Example: * supportedFeatures = { - * "feature1" -> SupportedVersionRange(minVersion=1, firstActiveVersion=1, maxVersion=5) + * "feature" -> SupportedVersionRange(minVersion=1, firstActiveVersion=1, maxVersion=5) * } - * This indicates no intent to deprecate any version levels for the feature. + * The above indicates no intent to deprecate any version levels for the feature. * * STEP 2: + * ======= + * * After the first step is over, you may (at some point) want to permanently remove the code/logic * for the functionality offered by the deprecated feature versions. This is the second step. Here a * subsequent major Kafka release is made with another Broker code change that removes the code for @@ -81,15 +89,15 @@ import scala.jdk.CollectionConverters._ * suitable higher minVersion value for the feature in the supportedFeatures map. * Example: * - In the example above in step 1, we showed how to deprecate version levels [1, 2] for - * "feature_1". Now let us assume the following finalized feature in ZK (after the deprecation + * "feature". Now let us assume the following finalized feature in ZK (after the deprecation * has been carried out): * { - * "feature_1" -> FinalizedVersionRange(minVersionLevel=3, maxVersionLevel=5) + * "feature" -> FinalizedVersionRange(minVersionLevel=3, maxVersionLevel=5) * } * Now, supposing you would like to permanently remove support for feature versions: [1, 2]. * Then, in the supportedFeatures map you should now supply the following: * supportedFeatures = { - * "feature1" -> SupportedVersionRange(minVersion=3, firstActiveVersion=3, maxVersion=5) + * "feature" -> SupportedVersionRange(minVersion=3, firstActiveVersion=3, maxVersion=5) * } */ class BrokerFeatures private (@volatile var supportedFeatures: Features[SupportedVersionRange]) { diff --git a/core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala b/core/src/main/scala/kafka/server/FinalizedFeatureChangeListener.scala index 6dda650f11af8..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(private val featureCache: FinalizedFeatureCache, private val 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(private val featureCache: FinalizedFeatureC // a case. if (version == ZkVersion.UnknownVersion) { info(s"Feature ZK node at path: $featureZkNodePath does not exist") - featureCache.clear() + finalizedFeatureCache.clear() } else { var maybeFeatureZNode: Option[FeatureZNode] = Option.empty try { @@ -93,17 +95,17 @@ class FinalizedFeatureChangeListener(private val featureCache: FinalizedFeatureC } catch { case e: IllegalArgumentException => { error(s"Unable to deserialize feature ZK node at path: $featureZkNodePath", e) - featureCache.clear() + finalizedFeatureCache.clear() } } maybeFeatureZNode.foreach(featureZNode => { featureZNode.status match { case FeatureZNodeStatus.Disabled => { info(s"Feature ZK node at path: $featureZkNodePath is in disabled status.") - featureCache.clear() + finalizedFeatureCache.clear() } case FeatureZNodeStatus.Enabled => { - featureCache.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 66d5dc0d7a8af..e3d4bbe18c18c 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -114,7 +114,7 @@ class KafkaApis(val requestChannel: RequestChannel, time: Time, val tokenManager: DelegationTokenManager, val brokerFeatures: BrokerFeatures, - val featureCache: FinalizedFeatureCache) extends ApiRequestHandler with Logging { + val finalizedFeatureCache: FinalizedFeatureCache) extends ApiRequestHandler with Logging { type FetchResponseStats = Map[TopicPartition, RecordConversionStats] this.logIdent = "[KafkaApi-%d] ".format(brokerId) @@ -1735,19 +1735,18 @@ class KafkaApis(val requestChannel: RequestChannel, apiVersionRequest.getErrorResponse(requestThrottleMs, Errors.INVALID_REQUEST.exception) else { val supportedFeatures = brokerFeatures.supportedFeatures - val finalizedFeatures = featureCache.get - if (finalizedFeatures.isEmpty) { - ApiVersionsResponse.apiVersionsResponse( + 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) } } } @@ -3116,17 +3115,16 @@ class KafkaApis(val requestChannel: RequestChannel, def sendResponseCallback(errors: Either[ApiError, Map[String, ApiError]]): Unit = { def createResponse(throttleTimeMs: Int): UpdateFeaturesResponse = { errors match { - case Left(topLevelError) => { - val featureUpdateNoErrors = updateFeaturesRequest - .data().featureUpdates().asScala - .map(update => update.feature() -> ApiError.NONE) - .toMap.asJava - UpdateFeaturesResponse.createWithErrors(topLevelError, featureUpdateNoErrors, throttleTimeMs) - } - case Right(featureUpdateErrors) => UpdateFeaturesResponse.createWithErrors( - ApiError.NONE, - featureUpdateErrors.asJava, - throttleTimeMs) + case Left(topLevelError) => + UpdateFeaturesResponse.createWithErrors( + topLevelError, + new util.HashMap[String, ApiError](), + throttleTimeMs) + case Right(featureUpdateErrors) => + UpdateFeaturesResponse.createWithErrors( + ApiError.NONE, + featureUpdateErrors.asJava, + throttleTimeMs) } } sendResponseMaybeThrottle(request, requestThrottleMs => createResponse(requestThrottleMs)) @@ -3134,7 +3132,7 @@ class KafkaApis(val requestChannel: RequestChannel, if (!authorize(request.context, ALTER, CLUSTER, CLUSTER_NAME)) { sendResponseCallback(Left(new ApiError(Errors.CLUSTER_AUTHORIZATION_FAILED))) - } else if (!config.isFeatureVersioningEnabled) { + } else if (!config.isFeatureVersioningSupported) { sendResponseCallback(Left(new ApiError(Errors.INVALID_REQUEST, "Feature versioning system is disabled."))) } else { controller.updateFeatures(updateFeaturesRequest, sendResponseCallback) 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 1a963ee70417e..0359ed7348ac0 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -234,7 +234,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = Time.SYSTEM, threadNameP /* initialize features */ _featureChangeListener = new FinalizedFeatureChangeListener(featureCache, _zkClient) - if (config.isFeatureVersioningEnabled) { + if (config.isFeatureVersioningSupported) { _featureChangeListener.initOrThrow(config.zkConnectionTimeoutMs) } diff --git a/core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala b/core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala index 0031c1d2780d1..8031f4e3137b5 100644 --- a/core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala +++ b/core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala @@ -24,17 +24,17 @@ 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, FeatureMetadata, UpdateFeaturesOptions, UpdateFeaturesResult} +import org.apache.kafka.clients.admin.{Admin, DescribeFeaturesOptions, FeatureMetadata, 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.{FeatureUpdate, UpdateFeaturesRequest, UpdateFeaturesResponse} +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, assertNull, assertTrue} +import org.junit.Assert.{assertEquals, assertFalse, assertNotEquals, assertNotNull, assertTrue} import org.scalatest.Assertions.intercept import scala.jdk.CollectionConverters._ @@ -107,14 +107,12 @@ class UpdateFeaturesTest extends BaseRequestTest { new FeatureMetadata( finalized.features().asScala.map { case(name, versionRange) => - (name, new org.apache.kafka.clients.admin.FinalizedVersionRange(versionRange.min(), - versionRange.max())) + (name, new org.apache.kafka.clients.admin.FinalizedVersionRange(versionRange.min(), versionRange.max())) }.asJava, Optional.of(epoch), supported.features().asScala.map { case(name, versionRange) => - (name, new org.apache.kafka.clients.admin.SupportedVersionRange(versionRange.min(), - versionRange.max())) + (name, new org.apache.kafka.clients.admin.SupportedVersionRange(versionRange.min(), versionRange.max())) }.asJava) } @@ -193,12 +191,7 @@ class UpdateFeaturesTest extends BaseRequestTest { assertEquals(Errors.NOT_CONTROLLER, Errors.forCode(response.data.errorCode())) assertNotNull(response.data.errorMessage()) - - assertEquals(1, response.data.results.size) - val result = response.data.results.asScala.head - assertEquals("feature_1", result.feature) - assertEquals(Errors.NONE, Errors.forCode(result.errorCode)) - assertNull(result.errorMessage) + assertEquals(0, response.data.results.size) checkFeatures( createAdminClient(), nodeBefore, From cc378c6dabbf6ae749baaa89f268e3a2ad1fc940 Mon Sep 17 00:00:00 2001 From: Kowshik Prakasam Date: Tue, 29 Sep 2020 23:15:58 -0700 Subject: [PATCH 33/41] Address comments from Boyang --- .../clients/admin/FinalizedVersionRange.java | 2 +- .../kafka/clients/admin/KafkaAdminClient.java | 6 ++- .../clients/admin/SupportedVersionRange.java | 45 ++++++++++++++----- .../common/feature/BaseVersionRange.java | 6 +-- .../common/feature/SupportedVersionRange.java | 30 +++++++++---- .../common/requests/ApiVersionsResponse.java | 11 +++-- .../common/message/ApiVersionsResponse.json | 2 + .../clients/admin/KafkaAdminClientTest.java | 4 +- .../feature/SupportedVersionRangeTest.java | 13 ++++-- .../requests/ApiVersionsResponseTest.java | 1 + .../kafka/controller/KafkaController.scala | 8 ++-- .../kafka/server/UpdateFeaturesTest.scala | 4 +- 12 files changed, 89 insertions(+), 43 deletions(-) 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 index d4c47fd843a18..cf4fa6787ff31 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/FinalizedVersionRange.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/FinalizedVersionRange.java @@ -77,7 +77,7 @@ public int hashCode() { @Override public String toString() { return String.format( - "FinalizedVersionRange{minVersionLevel:%d, maxVersionLevel:%d}", + "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 d82b0ff97fd75..eb3a551ae26de 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 @@ -4367,7 +4367,11 @@ private FeatureMetadata createFeatureMetadata(final ApiVersionsResponse response final Map supportedFeatures = new HashMap<>(); for (final SupportedFeatureKey key : response.data().supportedFeatures().valuesSet()) { - supportedFeatures.put(key.name(), new SupportedVersionRange(key.minVersion(), key.maxVersion())); + supportedFeatures.put( + key.name(), + new SupportedVersionRange(key.minVersion(), + key.firstActiveVersion(), + key.maxVersion())); } return new FeatureMetadata(finalizedFeatures, finalizedFeaturesEpoch, supportedFeatures); 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 index 8f8eeb35224e2..e1060568da5b6 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/SupportedVersionRange.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/SupportedVersionRange.java @@ -24,25 +24,35 @@ public class SupportedVersionRange { private final short minVersion; + private final short firstActiveVersion; + private final short maxVersion; /** - * Raises an exception unless the following condition is met: - * minVersion >= 1 and maxVersion >= 1 and maxVersion >= minVersion. + * Raises an exception unless the following conditions are met: + * 1 <= minVersion <= firstActiveVersion <= maxVersion * - * @param minVersion The minimum version value. - * @param maxVersion The maximum version value. + * @param minVersion The minimum version value. + * @param firstActiveVersion The first active version value. + * @param maxVersion The maximum version value. * * @throws IllegalArgumentException Raised when the condition described above is not met. */ - public SupportedVersionRange(final short minVersion, final short maxVersion) { - if (minVersion < 1 || maxVersion < 1 || maxVersion < minVersion) { + public SupportedVersionRange(final short minVersion, final short firstActiveVersion, final short maxVersion) { + if (minVersion < 1 || + maxVersion < 1 || + firstActiveVersion < minVersion || + firstActiveVersion > maxVersion) { throw new IllegalArgumentException( String.format( - "Expected minVersion >= 1, maxVersion >= 1 and maxVersion >= minVersion," + - " but received minVersion: %d, maxVersion: %d", minVersion, maxVersion)); + "Expected 1 <= minVersion <= firstActiveVersion <= maxVersion" + + " but received minVersion:%d, firstActiveVersion:%d, maxVersion:%d.", + minVersion, + firstActiveVersion, + maxVersion)); } this.minVersion = minVersion; + this.firstActiveVersion = firstActiveVersion; this.maxVersion = maxVersion; } @@ -50,6 +60,10 @@ public short minVersion() { return minVersion; } + public short firstActiveVersion() { + return firstActiveVersion; + } + public short maxVersion() { return maxVersion; } @@ -59,22 +73,29 @@ public boolean equals(Object other) { if (this == other) { return true; } - if (!(other instanceof SupportedVersionRange)) { + + if (other == null || getClass() != other.getClass()) { return false; } final SupportedVersionRange that = (SupportedVersionRange) other; - return this.minVersion == that.minVersion && this.maxVersion == that.maxVersion; + return this.minVersion == that.minVersion && + this.firstActiveVersion == that.firstActiveVersion && + this.maxVersion == that.maxVersion; } @Override public int hashCode() { - return Objects.hash(minVersion, maxVersion); + return Objects.hash(minVersion, firstActiveVersion, maxVersion); } @Override public String toString() { - return String.format("SupportedVersionRange{minVersion:%d, maxVersion:%d}", minVersion, maxVersion); + return String.format( + "SupportedVersionRange[min_version:%d, first_active_version:%d, max_version:%d]", + minVersion, + firstActiveVersion, + maxVersion); } } 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 05244636265b0..83817c5c413e7 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 @@ -111,11 +111,7 @@ public boolean equals(Object other) { return true; } - if (other == null) { - return false; - } - - if (getClass() != other.getClass()) { + if (other == null || getClass() != other.getClass()) { return false; } 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 3c6a070d8c1d8..f58726cd270e6 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 @@ -40,14 +40,26 @@ public class SupportedVersionRange extends BaseVersionRange { private final short firstActiveVersionValue; + /** + * Raises an exception unless the following conditions are met: + * 1 <= minVersion <= firstActiveVersion <= maxVersion + * + * @param minVersion The minimum version value. + * @param firstActiveVersion The first active version. + * @param maxVersion The maximum version value. + * + * @throws IllegalArgumentException If the conditions mentioned above are not met. + */ public SupportedVersionRange(short minVersion, short firstActiveVersion, short maxVersion) { super(MIN_VERSION_KEY_LABEL, minVersion, MAX_VERSION_KEY_LABEL, maxVersion); - if (firstActiveVersion < minVersion || firstActiveVersion > maxVersion) { + if (minVersion < 1 || + maxVersion < 1 || + firstActiveVersion < minVersion || + firstActiveVersion > maxVersion) { throw new IllegalArgumentException( String.format( - "Expected firstActiveVersion >= minVersion and" + - " firstActiveVersion <= maxVersion, but received" + - " minVersion:%d, firstActiveVersion:%d, maxVersion:%d", + "Expected 1 <= minVersion <= firstActiveVersion <= maxVersion," + + " but received minVersion:%d, firstActiveVersion:%d, maxVersion:%d.", minVersion, firstActiveVersion, maxVersion)); @@ -55,6 +67,10 @@ public SupportedVersionRange(short minVersion, short firstActiveVersion, short m this.firstActiveVersionValue = firstActiveVersion; } + public SupportedVersionRange(short minVersion, short maxVersion) { + this(minVersion, minVersion, maxVersion); + } + public short firstActiveVersion() { return firstActiveVersionValue; } @@ -71,11 +87,7 @@ public boolean equals(Object other) { return true; } - if (other == null) { - return false; - } - - if (getClass() != other.getClass()) { + if (other == null || getClass() != other.getClass()) { return false; } 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 4b7a7a7cd5735..447ebd34c234d 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 @@ -220,9 +220,11 @@ private static SupportedFeatureKeyCollection createSupportedFeatureKeys( SupportedFeatureKeyCollection converted = new SupportedFeatureKeyCollection(); for (Map.Entry feature : latestSupportedFeatures.features().entrySet()) { SupportedFeatureKey key = new SupportedFeatureKey(); + SupportedVersionRange versionRange = feature.getValue(); key.setName(feature.getKey()); - key.setMinVersion(feature.getValue().min()); - key.setMaxVersion(feature.getValue().max()); + key.setMinVersion(versionRange.min()); + key.setFirstActiveVersion(versionRange.firstActiveVersion()); + key.setMaxVersion(versionRange.max()); converted.add(key); } @@ -234,9 +236,10 @@ private static FinalizedFeatureKeyCollection createFinalizedFeatureKeys( FinalizedFeatureKeyCollection converted = new FinalizedFeatureKeyCollection(); for (Map.Entry feature : finalizedFeatures.features().entrySet()) { FinalizedFeatureKey key = new FinalizedFeatureKey(); + 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/resources/common/message/ApiVersionsResponse.json b/clients/src/main/resources/common/message/ApiVersionsResponse.json index ba6f01cb9434a..4dd28cc6e0f9a 100644 --- a/clients/src/main/resources/common/message/ApiVersionsResponse.json +++ b/clients/src/main/resources/common/message/ApiVersionsResponse.json @@ -51,6 +51,8 @@ "about": "The name of the feature." }, { "name": "MinVersion", "type": "int16", "versions": "3+", "about": "The minimum supported version for the feature." }, + { "name": "FirstActiveVersion", "type": "int16", "versions": "3+", + "about": "The first active version for the feature." }, { "name": "MaxVersion", "type": "int16", "versions": "3+", "about": "The maximum supported version for the feature." } ] 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 74bd860c9d1d6..e8665d64cd2ca 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 @@ -485,7 +485,7 @@ 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)))); + Utils.mkMap(Utils.mkEntry("test_feature_1", new SupportedVersionRange((short) 1, (short) 2, (short) 5)))); } private static Features convertSupportedFeaturesMap(Map features) { @@ -495,7 +495,7 @@ private static Features c featuresMap.put( entry.getKey(), new org.apache.kafka.common.feature.SupportedVersionRange(versionRange.minVersion(), - versionRange.minVersion(), + versionRange.firstActiveVersion(), versionRange.maxVersion())); } diff --git a/clients/src/test/java/org/apache/kafka/common/feature/SupportedVersionRangeTest.java b/clients/src/test/java/org/apache/kafka/common/feature/SupportedVersionRangeTest.java index 8658afa1e02ef..54147affc3ad7 100644 --- a/clients/src/test/java/org/apache/kafka/common/feature/SupportedVersionRangeTest.java +++ b/clients/src/test/java/org/apache/kafka/common/feature/SupportedVersionRangeTest.java @@ -187,9 +187,14 @@ public void testEquals() { @Test public void testAttributes() { - SupportedVersionRange versionRange = new SupportedVersionRange((short) 1, (short) 2, (short) 3); - assertEquals(1, versionRange.min()); - assertEquals(2, versionRange.firstActiveVersion()); - assertEquals(3, versionRange.max()); + SupportedVersionRange versionRange1 = new SupportedVersionRange((short) 1, (short) 2, (short) 3); + assertEquals(1, versionRange1.min()); + assertEquals(2, versionRange1.firstActiveVersion()); + assertEquals(3, versionRange1.max()); + + SupportedVersionRange versionRange2 = new SupportedVersionRange((short) 2, (short) 3); + assertEquals(2, versionRange2.min()); + assertEquals(versionRange2.firstActiveVersion(), versionRange2.min()); + assertEquals(3, versionRange2.max()); } } diff --git a/clients/src/test/java/org/apache/kafka/common/requests/ApiVersionsResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/ApiVersionsResponseTest.java index 0debaa154102b..41b8143039f4e 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/ApiVersionsResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/ApiVersionsResponseTest.java @@ -118,6 +118,7 @@ public void shouldReturnFeatureKeysWhenMagicIsCurrentValueAndThrottleMsIsDefault SupportedFeatureKey sKey = response.data.supportedFeatures().find("feature"); assertNotNull(sKey); assertEquals(1, sKey.minVersion()); + assertEquals(2, sKey.firstActiveVersion()); assertEquals(4, sKey.maxVersion()); assertEquals(1, response.data.finalizedFeatures().size()); diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 340c319d1a550..e5b78ed6ea05e 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -423,7 +423,7 @@ class KafkaController(val config: KafkaConfig, // // In this case, we do not deprecate any version levels since // supportedVersionRange.min() equals finalizedVersionRange.min(). - (featureName, new FinalizedVersionRange(supportedVersionRange.min(), + (featureName, new FinalizedVersionRange(supportedVersionRange.firstActiveVersion(), finalizedVersionRange.max())) } else { // This is a serious error. We should never be reaching here, since we already @@ -435,9 +435,9 @@ class KafkaController(val config: KafkaConfig, // 2. The existing version levels are incompatible with the supported version range. // // Examples of invalid cases that can cause this exception to be triggered: - // 1. No intersection : supportedVersionRange = [4, 7] and finalizedVersionRange = [2, 3]. - // 2. No intersection : supportedVersionRange = [2, 3] and finalizedVersionRange = [4, 7]. - // 3. Incompatible versions : supportedVersionRange = [2, 3] and finalizedVersionRange = [1, 7]. + // 1. No intersection : supportedVersionRange = [minVersion=1, firstActiveVersion=2, maxVersion=3] and finalizedVersionRange = [minVersionLevel=4, maxVersionLevel=7]. + // 2. No intersection : supportedVersionRange = [minVersion=1, firstActiveVersion=4, maxVersion=7] and finalizedVersionRange = [minVersionLevel=2, maxVersionLevel=3]. + // 3. Incompatible versions : supportedVersionRange = [minVersion=1, firstActiveVersion=2, maxVersion=3] and finalizedVersionRange = [minVersionLevel=1, maxVersionLevel=7]. throw new IllegalStateException( s"Can not update minimum version level in finalized feature: $featureName," + s" since the existing $finalizedVersionRange is not eligible for a change" diff --git a/core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala b/core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala index 8031f4e3137b5..a2d635840f97d 100644 --- a/core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala +++ b/core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala @@ -112,7 +112,9 @@ class UpdateFeaturesTest extends BaseRequestTest { Optional.of(epoch), supported.features().asScala.map { case(name, versionRange) => - (name, new org.apache.kafka.clients.admin.SupportedVersionRange(versionRange.min(), versionRange.max())) + (name, new org.apache.kafka.clients.admin.SupportedVersionRange(versionRange.min(), + versionRange.firstActiveVersion(), + versionRange.max())) }.asJava) } From 116352bcdd0aa9485db59cbd4e36cf7d03c81ffe Mon Sep 17 00:00:00 2001 From: Kowshik Prakasam Date: Thu, 1 Oct 2020 02:45:43 -0700 Subject: [PATCH 34/41] Address comments from Jun --- .../kafka/clients/admin/FeatureMetadata.java | 2 +- .../clients/admin/FinalizedVersionRange.java | 2 +- .../clients/admin/SupportedVersionRange.java | 2 +- .../apache/kafka/common/protocol/Errors.java | 2 +- .../kafka/controller/KafkaController.scala | 94 ++++++------- .../main/scala/kafka/server/KafkaApis.scala | 2 +- .../main/scala/kafka/server/KafkaServer.scala | 2 +- .../ControllerIntegrationTest.scala | 123 ++++++++++++++++++ .../kafka/server/UpdateFeaturesTest.scala | 81 +++++++----- 9 files changed, 216 insertions(+), 94 deletions(-) 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 index c7be80f1a1c37..815f9e3b97ca6 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/FeatureMetadata.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/FeatureMetadata.java @@ -35,7 +35,7 @@ public class FeatureMetadata { private final Map supportedFeatures; - public FeatureMetadata(final Map finalizedFeatures, + FeatureMetadata(final Map finalizedFeatures, final Optional finalizedFeaturesEpoch, final Map supportedFeatures) { this.finalizedFeatures = new HashMap<>(finalizedFeatures); 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 index cf4fa6787ff31..aa0401a8a86eb 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/FinalizedVersionRange.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/FinalizedVersionRange.java @@ -35,7 +35,7 @@ public class FinalizedVersionRange { * * @throws IllegalArgumentException Raised when the condition described above is not met. */ - public FinalizedVersionRange(final short minVersionLevel, final short maxVersionLevel) { + FinalizedVersionRange(final short minVersionLevel, final short maxVersionLevel) { if (minVersionLevel < 1 || maxVersionLevel < 1 || maxVersionLevel < minVersionLevel) { throw new IllegalArgumentException( String.format( 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 index e1060568da5b6..376464028a7e0 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/SupportedVersionRange.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/SupportedVersionRange.java @@ -38,7 +38,7 @@ public class SupportedVersionRange { * * @throws IllegalArgumentException Raised when the condition described above is not met. */ - public SupportedVersionRange(final short minVersion, final short firstActiveVersion, final short maxVersion) { + SupportedVersionRange(final short minVersion, final short firstActiveVersion, final short maxVersion) { if (minVersion < 1 || maxVersion < 1 || firstActiveVersion < minVersion || 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 74142c4fa5a57..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 @@ -338,7 +338,7 @@ public enum Errors { 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), - FEATURE_UPDATE_FAILED(96, "Unable to update finalized features.", FeatureUpdateFailedException::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/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index e5b78ed6ea05e..ad43d47eeb2be 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -141,6 +141,11 @@ class KafkaController(val config: KafkaConfig, newGauge("TopicsIneligibleToDeleteCount", () => ineligibleTopicsToDeleteCount) newGauge("ReplicasIneligibleToDeleteCount", () => ineligibleReplicasToDeleteCount) + class IncompatibleFeatureException(message: String, t: Throwable) extends RuntimeException(message, t) { + def this(message: String) = this(message, null) + def this(t: Throwable) = this("", t) + } + /** * Returns true if this broker is the current controller. */ @@ -392,60 +397,34 @@ class KafkaController(val config: KafkaConfig, featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs) } else { val existingFeatureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get) - var newFeatures: Features[FinalizedVersionRange] = Features.emptyFinalizedFeatures() - if (existingFeatureZNode.status.equals(FeatureZNodeStatus.Enabled)) { - newFeatures = Features.finalizedFeatures(existingFeatureZNode.features.features().asScala.map { - case (featureName, finalizedVersionRange) => - val supportedVersionRange = supportedFeatures.get(featureName) - if (supportedVersionRange == null) { - warn(s"Existing finalized feature: $featureName with $finalizedVersionRange" + - s" is absent in supported $supportedFeatures") - (featureName, finalizedVersionRange) - } else if (supportedVersionRange.max() >= finalizedVersionRange.max() && - supportedVersionRange.firstActiveVersion() <= finalizedVersionRange.max()) { - // Using the change below, we deprecate all version levels in the range: - // [finalizedVersionRange.min(), supportedVersionRange.firstActiveVersion() - 1]. - // - // NOTE: if finalizedVersionRange.min() equals supportedVersionRange.firstActiveVersion(), - // then we do not deprecate any version levels (since there is none to be deprecated). - // - // Examples: - // - // 1. supportedVersionRange = [minVersion=1, firstActiveVersion=4, maxVersion=7] - // and - // finalizedVersionRange = [minVersionLevel=1, maxVersionLevel=5]. - // - // In this case, we deprecate all version levels in the range: [1, 3]. - // - // 2. supportedVersionRange = [minVersion=1, firstActiveVersion=4, maxVersion=7] - // and - // finalizedVersionRange = [minVersionLevel=4, maxVersionLevel=5]. - // - // In this case, we do not deprecate any version levels since - // supportedVersionRange.min() equals finalizedVersionRange.min(). - (featureName, new FinalizedVersionRange(supportedVersionRange.firstActiveVersion(), - finalizedVersionRange.max())) - } else { - // This is a serious error. We should never be reaching here, since we already - // verify once during KafkaServer startup that existing finalized feature versions in - // the FeatureZNode contained no incompatibilities. If we are here, it means that one - // of the following is true: - // 1. The existing version levels fall completely outside the range of the supported - // version range (i.e. no intersection), or - // 2. The existing version levels are incompatible with the supported version range. - // - // Examples of invalid cases that can cause this exception to be triggered: - // 1. No intersection : supportedVersionRange = [minVersion=1, firstActiveVersion=2, maxVersion=3] and finalizedVersionRange = [minVersionLevel=4, maxVersionLevel=7]. - // 2. No intersection : supportedVersionRange = [minVersion=1, firstActiveVersion=4, maxVersion=7] and finalizedVersionRange = [minVersionLevel=2, maxVersionLevel=3]. - // 3. Incompatible versions : supportedVersionRange = [minVersion=1, firstActiveVersion=2, maxVersion=3] and finalizedVersionRange = [minVersionLevel=1, maxVersionLevel=7]. - throw new IllegalStateException( - s"Can not update minimum version level in finalized feature: $featureName," - + s" since the existing $finalizedVersionRange is not eligible for a change" - + s" based on the default $supportedVersionRange. This should never happen" - + s" since feature version incompatibilities are already checked during" - + s" Kafka server startup.") - } - }.asJava) + val newFeatures: Features[FinalizedVersionRange] = existingFeatureZNode.status match { + case FeatureZNodeStatus.Disabled => Features.emptyFinalizedFeatures() + case FeatureZNodeStatus.Enabled => + Features.finalizedFeatures(existingFeatureZNode.features.features().asScala.map { + case (featureName, finalizedVersionRange) => + val supportedVersionRange = supportedFeatures.get(featureName) + if (finalizedVersionRange.isIncompatibleWith(supportedVersionRange)) { + // This is a serious error. We should never be reaching here. + throw new IncompatibleFeatureException( + s"Can not update minimum version level in finalized feature: $featureName," + + s" since the existing $finalizedVersionRange is incompatible with" + + s" the $supportedVersionRange. This should never happen since feature version" + + s" incompatibilities are already checked during Kafka server startup.") + } else { + // If supportedVersionRange.firstActiveVersion() > finalizedVersionRange.min(), then + // we go ahead and deprecate all version levels in the range: + // [finalizedVersionRange.min(), supportedVersionRange.firstActiveVersion() - 1]. + // + // Example: + // supportedVersionRange = [minVersion=1, firstActiveVersion=4, maxVersion=7] + // and + // finalizedVersionRange = [minVersionLevel=1, maxVersionLevel=5]. + // + // In this case, we deprecate all version levels in the range: [1, 3]. + val newMinVersionLevel = supportedVersionRange.firstActiveVersion().max(finalizedVersionRange.min()) + (featureName, new FinalizedVersionRange(newMinVersionLevel, finalizedVersionRange.max())) + } + }.asJava) } val newFeatureZNode = new FeatureZNode(FeatureZNodeStatus.Enabled, newFeatures) if (!newFeatureZNode.equals(existingFeatureZNode)) { @@ -1563,11 +1542,14 @@ 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) triggerControllerMove() + if (t.isInstanceOf[IncompatibleFeatureException]) { + fatal("Feature version incompatibility found. The controller will eventually exit.") + Exit.exit(1) + } } } @@ -2090,7 +2072,7 @@ class KafkaController(val config: KafkaConfig, if (!existingFeatures.equals(targetFeatures)) { val newNode = new FeatureZNode(FeatureZNodeStatus.Enabled, Features.finalizedFeatures(targetFeatures.asJava)) val newVersion = zkClient.updateFeatureZNode(newNode) - featureCache.waitUntilEpochOrThrow(newVersion, request.data().timeoutMs().min(config.zkConnectionTimeoutMs)) + featureCache.waitUntilEpochOrThrow(newVersion, request.data().timeoutMs()) } } catch { // For all features that correspond to valid FeatureUpdate (i.e. error is Errors.NONE), diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index e3d4bbe18c18c..5077c3209f948 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -3118,7 +3118,7 @@ class KafkaApis(val requestChannel: RequestChannel, case Left(topLevelError) => UpdateFeaturesResponse.createWithErrors( topLevelError, - new util.HashMap[String, ApiError](), + Collections.emptyMap(), throttleTimeMs) case Right(featureUpdateErrors) => UpdateFeaturesResponse.createWithErrors( diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 0359ed7348ac0..0f2cf3fad7155 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -471,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) diff --git a/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala b/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala index a59a20c1a89d4..97e69849b6473 100644 --- a/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala @@ -24,11 +24,16 @@ import com.yammer.metrics.core.Timer 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.TestUtils.waitUntilTrue import kafka.utils.{LogCaptureAppender, TestUtils} import kafka.zk._ +import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, DescribeFeaturesOptions, FeatureUpdate, UpdateFeaturesOptions} import org.apache.kafka.common.errors.{ControllerMovedException, StaleBrokerEpochException} +import org.apache.kafka.common.feature.{Features, SupportedVersionRange} import org.apache.kafka.common.metrics.KafkaMetric +import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.protocol.Errors +import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.{ElectionType, TopicPartition} import org.apache.log4j.Level import org.junit.Assert.{assertEquals, assertTrue} @@ -605,6 +610,124 @@ class ControllerIntegrationTest extends ZooKeeperTestHarness { testControllerFeatureZNodeSetup(KAFKA_2_6_IV0) } + private def createAdminClient(): Admin = { + val props = new Properties + props.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, + TestUtils.bootstrapServers(servers, + ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT))) + Admin.create(props) + } + + private def updateSupportedFeatures(features: Features[SupportedVersionRange]): Unit = { + servers.foreach(s => { + s.brokerFeatures.setSupportedFeatures(features) + s.zkClient.updateBrokerInfo(s.createBrokerInfo) + }) + + // Wait until updates to supported features in all BrokerZNode propagate to the controller. + waitUntilTrue( + () => servers.exists(s => { + if (s.kafkaController.isActive) { + s.kafkaController.controllerContext.liveOrShuttingDownBrokers + .forall(b => { + b.features.equals(features) + }) + } else { + false + } + }), + "Controller did not get broker supported features updates for too long") + } + + private def finalizedFeaturesEpoch(client: Admin): Long = { + client + .describeFeatures(new DescribeFeaturesOptions().sendRequestToController(true)) + .featureMetadata() + .get() + .finalizedFeaturesEpoch() + .get() + } + + @Test + def testFeatureVersionDeprecation(): Unit = { + servers = makeServers(3, interBrokerProtocolVersion = Some(KAFKA_2_7_IV0)) + assertTrue(servers.forall(s => s.config.isFeatureVersioningSupported)) + TestUtils.waitUntilControllerElected(zkClient) + + val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path) + assertEquals(0, version) + val featureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get) + assertEquals(FeatureZNodeStatus.Enabled, featureZNode.status) + assertTrue(featureZNode.features.empty) + + var client: Admin = null + try { + // 1. Setup supported features ahead of finalizing them. + val supportedFeatures = Features.supportedFeatures( + new java.util.HashMap[String, SupportedVersionRange] { + put("deprecation_target", new SupportedVersionRange(1, 5)) + put("not_deprecation_target", new SupportedVersionRange(1, 6)) + }) + updateSupportedFeatures(supportedFeatures) + + // 2. Finalize the supported features at the max level. + client = createAdminClient() + client.updateFeatures( + new java.util.HashMap[String, FeatureUpdate] { + put("deprecation_target", new FeatureUpdate(5, false)) + put("not_deprecation_target", new FeatureUpdate(6, false)) + }, + new UpdateFeaturesOptions() + ).all().get() + val epochBeforeControllerFailover = finalizedFeaturesEpoch(client) + + // 3. Setup supported features once again, this time deprecating feature versions [1, 2] + // for the feature "deprecation_target". + val supportedFeaturesWithDeprecatedVersions = Features.supportedFeatures( + new java.util.HashMap[String, SupportedVersionRange] { + put("deprecation_target", new SupportedVersionRange(1, 3, 5)) + put("not_deprecation_target", new SupportedVersionRange(1, 6)) + }) + updateSupportedFeatures(supportedFeaturesWithDeprecatedVersions) + + // 4. Trigger controller failover + val controller = getController().kafkaController + zkClient.deleteController(controller.controllerContext.epochZkVersion) + + // 5. Expect the controller to update the finalized features + val epochAfterControllerFailover = epochBeforeControllerFailover + 1 + TestUtils.waitUntilTrue( + () => { + getController().kafkaController.isActive && + finalizedFeaturesEpoch(client) == epochAfterControllerFailover + }, + "Finalized features are not updated for too long after controller failover") + + // 6. Expect the finalized feature: "deprecation_target" to be updated to [3, 5] (due to + // deprecation of versions [1, 2]) + val featureMetadata = client + .describeFeatures(new DescribeFeaturesOptions().sendRequestToController(true)) + .featureMetadata() + .get() + assertTrue(featureMetadata.finalizedFeaturesEpoch().isPresent) + assertEquals(epochAfterControllerFailover, featureMetadata.finalizedFeaturesEpoch().get()) + + assertEquals(2, featureMetadata.finalizedFeatures().size()) + assertTrue(featureMetadata.finalizedFeatures().containsKey("deprecation_target")) + val deprecationTargetVersionRange = featureMetadata.finalizedFeatures().get("deprecation_target") + assertEquals(3, deprecationTargetVersionRange.minVersionLevel()) + assertEquals(5, deprecationTargetVersionRange.maxVersionLevel()) + + assertTrue(featureMetadata.finalizedFeatures().containsKey("not_deprecation_target")) + val notDeprecationTargetVersionRange = featureMetadata.finalizedFeatures().get("not_deprecation_target") + assertEquals(1, notDeprecationTargetVersionRange.minVersionLevel()) + assertEquals(6, notDeprecationTargetVersionRange.maxVersionLevel()) + } finally { + if (client != null) + client.close() + } + } + @Test def testControllerDetectsBouncedBrokers(): Unit = { servers = makeServers(2, enableControlledShutdown = false) diff --git a/core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala b/core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala index a2d635840f97d..3d71f70ea5ec2 100644 --- a/core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala +++ b/core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala @@ -24,7 +24,7 @@ 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, FeatureMetadata, FeatureUpdate, UpdateFeaturesOptions, UpdateFeaturesResult} +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} @@ -101,28 +101,31 @@ class UpdateFeaturesTest extends BaseRequestTest { FeatureZNode.decode(mayBeFeatureZNodeBytes.get) } - private def makeFeatureMetadata(finalized: Features[FinalizedVersionRange], - epoch: Long, - supported: Features[SupportedVersionRange]): FeatureMetadata = { - new FeatureMetadata( - finalized.features().asScala.map { - case(name, versionRange) => - (name, new org.apache.kafka.clients.admin.FinalizedVersionRange(versionRange.min(), versionRange.max())) - }.asJava, - Optional.of(epoch), - supported.features().asScala.map { - case(name, versionRange) => - (name, new org.apache.kafka.clients.admin.SupportedVersionRange(versionRange.min(), - versionRange.firstActiveVersion(), - versionRange.max())) - }.asJava) + 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 checkFeatures(client: Admin, expectedNode: FeatureZNode, expectedMetadata: FeatureMetadata): Unit = { + 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.firstActiveVersion(), 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(expectedMetadata, featureMetadata) + 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, @@ -166,7 +169,9 @@ class UpdateFeaturesTest extends BaseRequestTest { checkFeatures( adminClient, nodeBefore, - makeFeatureMetadata(defaultFinalizedFeatures(), versionBefore, defaultSupportedFeatures())) + defaultFinalizedFeatures(), + versionBefore, + defaultSupportedFeatures()) } /** @@ -197,7 +202,9 @@ class UpdateFeaturesTest extends BaseRequestTest { checkFeatures( createAdminClient(), nodeBefore, - makeFeatureMetadata(defaultFinalizedFeatures(), versionBefore, defaultSupportedFeatures())) + defaultFinalizedFeatures(), + versionBefore, + defaultSupportedFeatures()) } /** @@ -265,7 +272,9 @@ class UpdateFeaturesTest extends BaseRequestTest { checkFeatures( adminClient, nodeBefore, - makeFeatureMetadata(defaultFinalizedFeatures(), versionBefore, defaultSupportedFeatures())) + defaultFinalizedFeatures(), + versionBefore, + defaultSupportedFeatures()) } /** @@ -321,7 +330,9 @@ class UpdateFeaturesTest extends BaseRequestTest { checkFeatures( adminClient, nodeBefore, - makeFeatureMetadata(initialFinalizedFeatures, versionBefore, supportedFeatures)) + initialFinalizedFeatures, + versionBefore, + supportedFeatures) } /** @@ -401,7 +412,9 @@ class UpdateFeaturesTest extends BaseRequestTest { checkFeatures( adminClient, nodeBefore, - makeFeatureMetadata(initialFinalizedFeatures, versionBefore, supportedFeatures)) + initialFinalizedFeatures, + versionBefore, + supportedFeatures) } /** @@ -455,8 +468,6 @@ class UpdateFeaturesTest extends BaseRequestTest { val update1 = new FeatureUpdate(targetFinalizedFeatures.get("feature_1").max(), false) val update2 = new FeatureUpdate(targetFinalizedFeatures.get("feature_2").max(), false) - val expected = makeFeatureMetadata(targetFinalizedFeatures, versionBefore + 1, supportedFeatures) - val adminClient = createAdminClient() adminClient.updateFeatures( Utils.mkMap(Utils.mkEntry("feature_1", update1), Utils.mkEntry("feature_2", update2)), @@ -466,7 +477,9 @@ class UpdateFeaturesTest extends BaseRequestTest { checkFeatures( adminClient, new FeatureZNode(FeatureZNodeStatus.Enabled, targetFinalizedFeatures), - expected) + targetFinalizedFeatures, + versionBefore + 1, + supportedFeatures) } /** @@ -498,8 +511,6 @@ class UpdateFeaturesTest extends BaseRequestTest { val update1 = new FeatureUpdate(targetFinalizedFeatures.get("feature_1").max(), false) val update2 = new FeatureUpdate(targetFinalizedFeatures.get("feature_2").max(), true) - val expected = makeFeatureMetadata(targetFinalizedFeatures, versionBefore + 1, supportedFeatures) - val adminClient = createAdminClient() adminClient.updateFeatures( Utils.mkMap(Utils.mkEntry("feature_1", update1), Utils.mkEntry("feature_2", update2)), @@ -509,7 +520,9 @@ class UpdateFeaturesTest extends BaseRequestTest { checkFeatures( adminClient, new FeatureZNode(FeatureZNodeStatus.Enabled, targetFinalizedFeatures), - expected) + targetFinalizedFeatures, + versionBefore + 1, + supportedFeatures) } /** @@ -560,7 +573,9 @@ class UpdateFeaturesTest extends BaseRequestTest { checkFeatures( adminClient, FeatureZNode(FeatureZNodeStatus.Enabled, expectedFeatures), - makeFeatureMetadata(expectedFeatures, versionBefore + 1, supportedFeatures)) + expectedFeatures, + versionBefore + 1, + supportedFeatures) } /** @@ -625,6 +640,8 @@ class UpdateFeaturesTest extends BaseRequestTest { checkFeatures( adminClient, FeatureZNode(FeatureZNodeStatus.Enabled, expectedFeatures), - makeFeatureMetadata(expectedFeatures, versionBefore + 1, supportedFeatures)) + expectedFeatures, + versionBefore + 1, + supportedFeatures) } } From 50f53dd4060bd70d0ffd22cf916fcbd359e8db1b Mon Sep 17 00:00:00 2001 From: Kowshik Prakasam Date: Fri, 2 Oct 2020 01:59:23 -0700 Subject: [PATCH 35/41] Address comment from Jun: Revert firstActiveVersion change --- .../kafka/clients/admin/KafkaAdminClient.java | 6 +- .../clients/admin/SupportedVersionRange.java | 31 +--- .../common/feature/BaseVersionRange.java | 8 - .../common/feature/FinalizedVersionRange.java | 7 +- .../common/feature/SupportedVersionRange.java | 73 +------- .../common/requests/ApiVersionsResponse.java | 1 - .../common/message/ApiVersionsResponse.json | 2 - .../clients/admin/KafkaAdminClientTest.java | 3 +- .../kafka/common/feature/FeaturesTest.java | 36 ++-- .../feature/FinalizedVersionRangeTest.java | 21 +-- .../feature/SupportedVersionRangeTest.java | 108 +++--------- .../requests/ApiVersionsResponseTest.java | 3 +- .../kafka/controller/KafkaController.scala | 98 +++-------- .../scala/kafka/server/BrokerFeatures.scala | 77 +-------- .../kafka/cluster/BrokerEndPointTest.scala | 4 +- .../ControllerIntegrationTest.scala | 161 ++++-------------- .../kafka/server/BrokerFeaturesTest.scala | 45 ++--- .../server/FinalizedFeatureCacheTest.scala | 14 +- .../FinalizedFeatureChangeListenerTest.scala | 4 +- .../kafka/server/UpdateFeaturesTest.scala | 89 ++-------- .../unit/kafka/zk/KafkaZkClientTest.scala | 8 +- 21 files changed, 159 insertions(+), 640 deletions(-) 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 eb3a551ae26de..d82b0ff97fd75 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 @@ -4367,11 +4367,7 @@ private FeatureMetadata createFeatureMetadata(final ApiVersionsResponse response final Map supportedFeatures = new HashMap<>(); for (final SupportedFeatureKey key : response.data().supportedFeatures().valuesSet()) { - supportedFeatures.put( - key.name(), - new SupportedVersionRange(key.minVersion(), - key.firstActiveVersion(), - key.maxVersion())); + supportedFeatures.put(key.name(), new SupportedVersionRange(key.minVersion(), key.maxVersion())); } return new FeatureMetadata(finalizedFeatures, finalizedFeaturesEpoch, supportedFeatures); 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 index 376464028a7e0..9440ee627c455 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/SupportedVersionRange.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/SupportedVersionRange.java @@ -24,35 +24,28 @@ public class SupportedVersionRange { private final short minVersion; - private final short firstActiveVersion; - private final short maxVersion; /** * Raises an exception unless the following conditions are met: - * 1 <= minVersion <= firstActiveVersion <= maxVersion + * 1 <= minVersion <= maxVersion. * * @param minVersion The minimum version value. - * @param firstActiveVersion The first active 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 firstActiveVersion, final short maxVersion) { + SupportedVersionRange(final short minVersion, final short maxVersion) { if (minVersion < 1 || maxVersion < 1 || - firstActiveVersion < minVersion || - firstActiveVersion > maxVersion) { + maxVersion < minVersion) { throw new IllegalArgumentException( String.format( - "Expected 1 <= minVersion <= firstActiveVersion <= maxVersion" + - " but received minVersion:%d, firstActiveVersion:%d, maxVersion:%d.", + "Expected 1 <= minVersion <= maxVersion but received minVersion:%d, maxVersion:%d.", minVersion, - firstActiveVersion, maxVersion)); } this.minVersion = minVersion; - this.firstActiveVersion = firstActiveVersion; this.maxVersion = maxVersion; } @@ -60,10 +53,6 @@ public short minVersion() { return minVersion; } - public short firstActiveVersion() { - return firstActiveVersion; - } - public short maxVersion() { return maxVersion; } @@ -79,23 +68,17 @@ public boolean equals(Object other) { } final SupportedVersionRange that = (SupportedVersionRange) other; - return this.minVersion == that.minVersion && - this.firstActiveVersion == that.firstActiveVersion && - this.maxVersion == that.maxVersion; + return this.minVersion == that.minVersion && this.maxVersion == that.maxVersion; } @Override public int hashCode() { - return Objects.hash(minVersion, firstActiveVersion, maxVersion); + return Objects.hash(minVersion, maxVersion); } @Override public String toString() { - return String.format( - "SupportedVersionRange[min_version:%d, first_active_version:%d, max_version:%d]", - minVersion, - firstActiveVersion, - maxVersion); + return String.format("SupportedVersionRange[min_version:%d, max_version:%d]", minVersion, maxVersion); } } 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 83817c5c413e7..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 @@ -134,12 +134,4 @@ public static short valueOrThrow(String key, Map versionRangeMap) } return value; } - - protected String minKeyLabel() { - return minKeyLabel; - } - - protected String maxKeyLabel() { - return maxKeyLabel; - } } 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 545e42169ddcd..ff778422af089 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, first_active_version, max] range of the provided SupportedVersionRange parameter. + * [min, max] range of the provided SupportedVersionRange parameter. * * @param supportedVersionRange the SupportedVersionRange to be checked * @@ -49,7 +49,6 @@ public static FinalizedVersionRange fromMap(Map versionRangeMap) */ public boolean isIncompatibleWith(SupportedVersionRange supportedVersionRange) { return min() < supportedVersionRange.min() || - max() > supportedVersionRange.max() || - max() < supportedVersionRange.firstActiveVersion(); + max() > supportedVersionRange.max(); } } 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 f58726cd270e6..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 @@ -17,16 +17,9 @@ package org.apache.kafka.common.feature; import java.util.Map; -import java.util.Objects; -import org.apache.kafka.common.utils.Utils; /** - * An extended {@link BaseVersionRange} representing the min, max and first active versions for a - * supported feature: - * - minVersion: This is the minimum supported version for the feature. - * - maxVersion: This the maximum supported version for the feature. - * - firstActiveVersion: This is the first active version for the feature. Versions in the range - * [minVersion, firstActiveVersion - 1] are considered to be deprecated. + * 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. @@ -35,75 +28,17 @@ 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"; - // Label for the first active version key, that's used only to convert to/from a map. - private static final String FIRST_ACTIVE_VERSION_KEY_LABEL = "first_active_version"; - - private final short firstActiveVersionValue; - - /** - * Raises an exception unless the following conditions are met: - * 1 <= minVersion <= firstActiveVersion <= maxVersion - * - * @param minVersion The minimum version value. - * @param firstActiveVersion The first active version. - * @param maxVersion The maximum version value. - * - * @throws IllegalArgumentException If the conditions mentioned above are not met. - */ - public SupportedVersionRange(short minVersion, short firstActiveVersion, short maxVersion) { - super(MIN_VERSION_KEY_LABEL, minVersion, MAX_VERSION_KEY_LABEL, maxVersion); - if (minVersion < 1 || - maxVersion < 1 || - firstActiveVersion < minVersion || - firstActiveVersion > maxVersion) { - throw new IllegalArgumentException( - String.format( - "Expected 1 <= minVersion <= firstActiveVersion <= maxVersion," + - " but received minVersion:%d, firstActiveVersion:%d, maxVersion:%d.", - minVersion, - firstActiveVersion, - maxVersion)); - } - this.firstActiveVersionValue = firstActiveVersion; - } - public SupportedVersionRange(short minVersion, short maxVersion) { - this(minVersion, minVersion, maxVersion); - } - - public short firstActiveVersion() { - return firstActiveVersionValue; - } - - public Map toMap() { - return Utils.mkMap(Utils.mkEntry(minKeyLabel(), min()), - Utils.mkEntry(FIRST_ACTIVE_VERSION_KEY_LABEL, firstActiveVersionValue), - Utils.mkEntry(maxKeyLabel(), max())); - } - - @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 super.equals(other) && this.firstActiveVersionValue == that.firstActiveVersionValue; + super(MIN_VERSION_KEY_LABEL, minVersion, MAX_VERSION_KEY_LABEL, maxVersion); } - @Override - public int hashCode() { - return Objects.hash(super.hashCode(), firstActiveVersionValue); + public SupportedVersionRange(short maxVersion) { + this((short) 1, maxVersion); } public static SupportedVersionRange fromMap(Map versionRangeMap) { return new SupportedVersionRange( BaseVersionRange.valueOrThrow(MIN_VERSION_KEY_LABEL, versionRangeMap), - BaseVersionRange.valueOrThrow(FIRST_ACTIVE_VERSION_KEY_LABEL, versionRangeMap), BaseVersionRange.valueOrThrow(MAX_VERSION_KEY_LABEL, versionRangeMap)); } } 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 447ebd34c234d..19d5593042747 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 @@ -223,7 +223,6 @@ private static SupportedFeatureKeyCollection createSupportedFeatureKeys( SupportedVersionRange versionRange = feature.getValue(); key.setName(feature.getKey()); key.setMinVersion(versionRange.min()); - key.setFirstActiveVersion(versionRange.firstActiveVersion()); key.setMaxVersion(versionRange.max()); converted.add(key); } diff --git a/clients/src/main/resources/common/message/ApiVersionsResponse.json b/clients/src/main/resources/common/message/ApiVersionsResponse.json index 4dd28cc6e0f9a..ba6f01cb9434a 100644 --- a/clients/src/main/resources/common/message/ApiVersionsResponse.json +++ b/clients/src/main/resources/common/message/ApiVersionsResponse.json @@ -51,8 +51,6 @@ "about": "The name of the feature." }, { "name": "MinVersion", "type": "int16", "versions": "3+", "about": "The minimum supported version for the feature." }, - { "name": "FirstActiveVersion", "type": "int16", "versions": "3+", - "about": "The first active version for the feature." }, { "name": "MaxVersion", "type": "int16", "versions": "3+", "about": "The maximum supported version for the feature." } ] 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 e8665d64cd2ca..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 @@ -485,7 +485,7 @@ 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) 2, (short) 5)))); + Utils.mkMap(Utils.mkEntry("test_feature_1", new SupportedVersionRange((short) 1, (short) 5)))); } private static Features convertSupportedFeaturesMap(Map features) { @@ -495,7 +495,6 @@ private static Features c featuresMap.put( entry.getKey(), new org.apache.kafka.common.feature.SupportedVersionRange(versionRange.minVersion(), - versionRange.firstActiveVersion(), versionRange.maxVersion())); } diff --git a/clients/src/test/java/org/apache/kafka/common/feature/FeaturesTest.java b/clients/src/test/java/org/apache/kafka/common/feature/FeaturesTest.java index 81db1ef096ec6..896196dc550a4 100644 --- a/clients/src/test/java/org/apache/kafka/common/feature/FeaturesTest.java +++ b/clients/src/test/java/org/apache/kafka/common/feature/FeaturesTest.java @@ -59,8 +59,8 @@ public void testNullFeatures() { @Test public void testGetAllFeaturesAPI() { - SupportedVersionRange v1 = new SupportedVersionRange((short) 1, (short) 2, (short) 3); - SupportedVersionRange v2 = new SupportedVersionRange((short) 3, (short) 4, (short) 5); + SupportedVersionRange v1 = new SupportedVersionRange((short) 1, (short) 2); + SupportedVersionRange v2 = new SupportedVersionRange((short) 3, (short) 4); Map allFeatures = mkMap(mkEntry("feature_1", v1), mkEntry("feature_2", v2)); Features features = Features.supportedFeatures(allFeatures); @@ -69,8 +69,8 @@ public void testGetAllFeaturesAPI() { @Test public void testGetAPI() { - SupportedVersionRange v1 = new SupportedVersionRange((short) 1, (short) 2, (short) 3); - SupportedVersionRange v2 = new SupportedVersionRange((short) 3, (short) 4, (short) 5); + SupportedVersionRange v1 = new SupportedVersionRange((short) 1, (short) 2); + SupportedVersionRange v2 = new SupportedVersionRange((short) 3, (short) 4); Map allFeatures = mkMap(mkEntry("feature_1", v1), mkEntry("feature_2", v2)); Features features = Features.supportedFeatures(allFeatures); assertEquals(v1, features.get("feature_1")); @@ -80,19 +80,15 @@ public void testGetAPI() { @Test public void testFromFeaturesMapToFeaturesMap() { - SupportedVersionRange v1 = new SupportedVersionRange((short) 1, (short) 2, (short) 3); - SupportedVersionRange v2 = new SupportedVersionRange((short) 3, (short) 4, (short) 5); + SupportedVersionRange v1 = new SupportedVersionRange((short) 1, (short) 2); + SupportedVersionRange v2 = new SupportedVersionRange((short) 3, (short) 4); Map allFeatures = mkMap(mkEntry("feature_1", v1), mkEntry("feature_2", v2)); Features features = Features.supportedFeatures(allFeatures); Map> expected = mkMap( - mkEntry("feature_1", mkMap(mkEntry("min_version", (short) 1), - mkEntry("first_active_version", (short) 2), - mkEntry("max_version", (short) 3))), - mkEntry("feature_2", mkMap(mkEntry("min_version", (short) 3), - mkEntry("first_active_version", (short) 4), - mkEntry("max_version", (short) 5)))); + mkEntry("feature_1", mkMap(mkEntry("min_version", (short) 1), mkEntry("max_version", (short) 2))), + mkEntry("feature_2", mkMap(mkEntry("min_version", (short) 3), mkEntry("max_version", (short) 4)))); assertEquals(expected, features.toMap()); assertEquals(features, Features.fromSupportedFeaturesMap(expected)); } @@ -121,23 +117,21 @@ public void testToStringFinalizedFeatures() { Features features = Features.finalizedFeatures(allFeatures); assertEquals( - "Features{(feature_1 -> FinalizedVersionRange[min_version_level:1, max_version_level:2])," + - " (feature_2 -> FinalizedVersionRange[min_version_level:3, max_version_level:4])}", + "Features{(feature_1 -> FinalizedVersionRange[min_version_level:1, max_version_level:2]), (feature_2 -> FinalizedVersionRange[min_version_level:3, max_version_level:4])}", features.toString()); } @Test public void testToStringSupportedFeatures() { - SupportedVersionRange v1 = new SupportedVersionRange((short) 1, (short) 2, (short) 3); - SupportedVersionRange v2 = new SupportedVersionRange((short) 3, (short) 4, (short) 5); + SupportedVersionRange v1 = new SupportedVersionRange((short) 1, (short) 2); + SupportedVersionRange v2 = new SupportedVersionRange((short) 3, (short) 4); Map allFeatures = mkMap(mkEntry("feature_1", v1), mkEntry("feature_2", v2)); Features features = Features.supportedFeatures(allFeatures); assertEquals( - "Features{(feature_1 -> SupportedVersionRange[min_version:1, first_active_version:2, max_version:3])," + - " (feature_2 -> SupportedVersionRange[min_version:3, first_active_version:4, max_version:5])}", + "Features{(feature_1 -> SupportedVersionRange[min_version:1, max_version:2]), (feature_2 -> SupportedVersionRange[min_version:3, max_version:4])}", features.toString()); } @@ -145,7 +139,7 @@ public void testToStringSupportedFeatures() { public void testSuppportedFeaturesFromMapFailureWithInvalidMissingMaxVersion() { // This is invalid because 'max_version' key is missing. Map> invalidFeatures = mkMap( - mkEntry("feature_1", mkMap(mkEntry("min_version", (short) 1), mkEntry("first_active_version", (short) 2)))); + mkEntry("feature_1", mkMap(mkEntry("min_version", (short) 1)))); assertThrows( IllegalArgumentException.class, () -> Features.fromSupportedFeaturesMap(invalidFeatures)); @@ -163,13 +157,13 @@ public void testFinalizedFeaturesFromMapFailureWithInvalidMissingMaxVersionLevel @Test public void testEquals() { - SupportedVersionRange v1 = new SupportedVersionRange((short) 1, (short) 2, (short) 3); + SupportedVersionRange v1 = new SupportedVersionRange((short) 1, (short) 2); Map allFeatures = mkMap(mkEntry("feature_1", v1)); Features features = Features.supportedFeatures(allFeatures); Features featuresClone = Features.supportedFeatures(allFeatures); assertTrue(features.equals(featuresClone)); - SupportedVersionRange v2 = new SupportedVersionRange((short) 1, (short) 3, (short) 4); + SupportedVersionRange v2 = new SupportedVersionRange((short) 1, (short) 3); Map allFeaturesDifferent = mkMap(mkEntry("feature_1", v2)); Features featuresDifferent = Features.supportedFeatures(allFeaturesDifferent); assertFalse(features.equals(featuresDifferent)); diff --git a/clients/src/test/java/org/apache/kafka/common/feature/FinalizedVersionRangeTest.java b/clients/src/test/java/org/apache/kafka/common/feature/FinalizedVersionRangeTest.java index 32aeaec23b46e..3d62a8fdbad8a 100644 --- a/clients/src/test/java/org/apache/kafka/common/feature/FinalizedVersionRangeTest.java +++ b/clients/src/test/java/org/apache/kafka/common/feature/FinalizedVersionRangeTest.java @@ -61,21 +61,14 @@ public void testToString() { } @Test - public void testIsIncompatibleWith() { - assertFalse(new FinalizedVersionRange((short) 1, (short) 1).isIncompatibleWith(new SupportedVersionRange((short) 1, (short) 1, (short) 1))); - assertFalse(new FinalizedVersionRange((short) 2, (short) 3).isIncompatibleWith(new SupportedVersionRange((short) 1, (short) 1, (short) 4))); - assertFalse(new FinalizedVersionRange((short) 2, (short) 3).isIncompatibleWith(new SupportedVersionRange((short) 1, (short) 2, (short) 4))); - assertFalse(new FinalizedVersionRange((short) 2, (short) 3).isIncompatibleWith(new SupportedVersionRange((short) 1, (short) 3, (short) 4))); - assertFalse(new FinalizedVersionRange((short) 1, (short) 4).isIncompatibleWith(new SupportedVersionRange((short) 1, (short) 1, (short) 4))); + public void testIsCompatibleWith() { + assertFalse(new FinalizedVersionRange((short) 1, (short) 1).isIncompatibleWith(new SupportedVersionRange((short) 1, (short) 1))); + assertFalse(new FinalizedVersionRange((short) 2, (short) 3).isIncompatibleWith(new SupportedVersionRange((short) 1, (short) 4))); + assertFalse(new FinalizedVersionRange((short) 1, (short) 4).isIncompatibleWith(new SupportedVersionRange((short) 1, (short) 4))); - // minVersion and maxVersion are incompatible. - assertTrue(new FinalizedVersionRange((short) 1, (short) 4).isIncompatibleWith(new SupportedVersionRange((short) 2, (short) 2, (short) 3))); - // Only minVersion is incompatible. - assertTrue(new FinalizedVersionRange((short) 1, (short) 4).isIncompatibleWith(new SupportedVersionRange((short) 2, (short) 2, (short) 4))); - // Only maxVersion is incompatible. - assertTrue(new FinalizedVersionRange((short) 2, (short) 4).isIncompatibleWith(new SupportedVersionRange((short) 2, (short) 2, (short) 3))); - // Only firstActiveVersion is incompatible. - assertTrue(new FinalizedVersionRange((short) 1, (short) 4).isIncompatibleWith(new SupportedVersionRange((short) 1, (short) 5, (short) 5))); + assertTrue(new FinalizedVersionRange((short) 1, (short) 4).isIncompatibleWith(new SupportedVersionRange((short) 2, (short) 3))); + assertTrue(new FinalizedVersionRange((short) 1, (short) 4).isIncompatibleWith(new SupportedVersionRange((short) 2, (short) 4))); + assertTrue(new FinalizedVersionRange((short) 2, (short) 4).isIncompatibleWith(new SupportedVersionRange((short) 2, (short) 3))); } @Test diff --git a/clients/src/test/java/org/apache/kafka/common/feature/SupportedVersionRangeTest.java b/clients/src/test/java/org/apache/kafka/common/feature/SupportedVersionRangeTest.java index 54147affc3ad7..4c7b5591dad0f 100644 --- a/clients/src/test/java/org/apache/kafka/common/feature/SupportedVersionRangeTest.java +++ b/clients/src/test/java/org/apache/kafka/common/feature/SupportedVersionRangeTest.java @@ -38,52 +38,35 @@ public void testFailDueToInvalidParams() { // min and max can't be < 1. assertThrows( IllegalArgumentException.class, - () -> new SupportedVersionRange((short) 0, (short) 1, (short) 0)); + () -> new SupportedVersionRange((short) 0, (short) 0)); // min can't be < 1. assertThrows( IllegalArgumentException.class, - () -> new SupportedVersionRange((short) 0, (short) 1, (short) 1)); + () -> new SupportedVersionRange((short) 0, (short) 1)); // max can't be < 1. assertThrows( IllegalArgumentException.class, - () -> new SupportedVersionRange((short) 1, (short) 1, (short) 0)); - // firstActiveVersion can't be < 1. - assertThrows( - IllegalArgumentException.class, - () -> new SupportedVersionRange((short) 1, (short) 0, (short) 1)); + () -> new SupportedVersionRange((short) 1, (short) 0)); // min can't be > max. assertThrows( IllegalArgumentException.class, - () -> new SupportedVersionRange((short) 2, (short) 1, (short) 1)); - // firstActiveVersion can't be < min. - assertThrows( - IllegalArgumentException.class, - () -> new SupportedVersionRange((short) 2, (short) 1, (short) 2)); - // firstActiveVersion can't be > max. - assertThrows( - IllegalArgumentException.class, - () -> new SupportedVersionRange((short) 1, (short) 3, (short) 2)); + () -> new SupportedVersionRange((short) 2, (short) 1)); } @Test public void testFromToMap() { - SupportedVersionRange versionRange = new SupportedVersionRange((short) 1, (short) 2, (short) 3); + SupportedVersionRange versionRange = new SupportedVersionRange((short) 1, (short) 2); assertEquals(1, versionRange.min()); - assertEquals(2, versionRange.firstActiveVersion()); - assertEquals(3, versionRange.max()); + assertEquals(2, versionRange.max()); Map versionRangeMap = versionRange.toMap(); assertEquals( - mkMap(mkEntry("min_version", versionRange.min()), - mkEntry("first_active_version", versionRange.firstActiveVersion()), - mkEntry("max_version", versionRange.max())), + mkMap(mkEntry("min_version", versionRange.min()), mkEntry("max_version", versionRange.max())), versionRangeMap); SupportedVersionRange newVersionRange = SupportedVersionRange.fromMap(versionRangeMap); assertEquals(1, newVersionRange.min()); - assertEquals(2, newVersionRange.firstActiveVersion()); - assertEquals(3, newVersionRange.max()); - + assertEquals(2, newVersionRange.max()); assertEquals(versionRange, newVersionRange); } @@ -91,110 +74,69 @@ public void testFromToMap() { public void testFromMapFailure() { // min_version can't be < 1. Map invalidWithBadMinVersion = - mkMap(mkEntry("min_version", (short) 0), - mkEntry("first_active_version", (short) 1), - mkEntry("max_version", (short) 1)); + mkMap(mkEntry("min_version", (short) 0), mkEntry("max_version", (short) 1)); assertThrows( IllegalArgumentException.class, () -> SupportedVersionRange.fromMap(invalidWithBadMinVersion)); // max_version can't be < 1. Map invalidWithBadMaxVersion = - mkMap(mkEntry("min_version", (short) 1), - mkEntry("first_active_version", (short) 1), - mkEntry("max_version", (short) 0)); + mkMap(mkEntry("min_version", (short) 1), mkEntry("max_version", (short) 0)); assertThrows( IllegalArgumentException.class, () -> SupportedVersionRange.fromMap(invalidWithBadMaxVersion)); // min_version and max_version can't be < 1. Map invalidWithBadMinMaxVersion = - mkMap(mkEntry("min_version", (short) 0), - mkEntry("first_active_version", (short) 1), - mkEntry("max_version", (short) 0)); + mkMap(mkEntry("min_version", (short) 0), mkEntry("max_version", (short) 0)); assertThrows( IllegalArgumentException.class, () -> SupportedVersionRange.fromMap(invalidWithBadMinMaxVersion)); // min_version can't be > max_version. Map invalidWithLowerMaxVersion = - mkMap(mkEntry("min_version", (short) 2), - mkEntry("first_active_version", (short) 1), - mkEntry("max_version", (short) 1)); + mkMap(mkEntry("min_version", (short) 2), mkEntry("max_version", (short) 1)); assertThrows( IllegalArgumentException.class, () -> SupportedVersionRange.fromMap(invalidWithLowerMaxVersion)); - // first_active_version can't be < min_version - Map invalidWithLowerFirstActiveVersion = - mkMap(mkEntry("min_version", (short) 2), - mkEntry("first_active_version", (short) 1), - mkEntry("max_version", (short) 3)); - assertThrows( - IllegalArgumentException.class, - () -> SupportedVersionRange.fromMap(invalidWithLowerFirstActiveVersion)); - - // first_active_version can't be > max_version - Map invalidWithHigherFirstActiveVersion = - mkMap(mkEntry("min_version", (short) 1), - mkEntry("first_active_version", (short) 3), - mkEntry("max_version", (short) 2)); - assertThrows( - IllegalArgumentException.class, - () -> SupportedVersionRange.fromMap(invalidWithHigherFirstActiveVersion)); - // min_version key missing. Map invalidWithMinKeyMissing = - mkMap(mkEntry("first_active_version", (short) 1), - mkEntry("max_version", (short) 1)); + mkMap(mkEntry("max_version", (short) 1)); assertThrows( IllegalArgumentException.class, () -> SupportedVersionRange.fromMap(invalidWithMinKeyMissing)); // max_version key missing. Map invalidWithMaxKeyMissing = - mkMap(mkEntry("min_version", (short) 1), - mkEntry("first_active_version", (short) 1)); + mkMap(mkEntry("min_version", (short) 1)); assertThrows( IllegalArgumentException.class, () -> SupportedVersionRange.fromMap(invalidWithMaxKeyMissing)); - - // first_active_version key missing. - Map invalidWithFirstActiveVersionKeyMissing = - mkMap(mkEntry("min_version", (short) 1), - mkEntry("max_version", (short) 1)); - assertThrows( - IllegalArgumentException.class, - () -> SupportedVersionRange.fromMap(invalidWithFirstActiveVersionKeyMissing)); } @Test public void testToString() { assertEquals( - "SupportedVersionRange[min_version:1, first_active_version:2, max_version:3]", - new SupportedVersionRange((short) 1, (short) 2, (short) 3).toString()); + "SupportedVersionRange[min_version:1, max_version:1]", + new SupportedVersionRange((short) 1, (short) 1).toString()); + assertEquals( + "SupportedVersionRange[min_version:1, max_version:2]", + new SupportedVersionRange((short) 1, (short) 2).toString()); } @Test public void testEquals() { - SupportedVersionRange tested = new SupportedVersionRange((short) 1, (short) 2, (short) 4); + SupportedVersionRange tested = new SupportedVersionRange((short) 1, (short) 1); assertTrue(tested.equals(tested)); - assertFalse(tested.equals(new SupportedVersionRange(tested.firstActiveVersion(), tested.firstActiveVersion(), tested.max()))); - assertFalse(tested.equals(new SupportedVersionRange(tested.min(), (short) (tested.firstActiveVersion() + 1), tested.max()))); - assertFalse(tested.equals(new SupportedVersionRange(tested.min(), tested.firstActiveVersion(), (short) (tested.max() + 1)))); + assertFalse(tested.equals(new SupportedVersionRange((short) 1, (short) 2))); assertFalse(tested.equals(null)); } @Test - public void testAttributes() { - SupportedVersionRange versionRange1 = new SupportedVersionRange((short) 1, (short) 2, (short) 3); - assertEquals(1, versionRange1.min()); - assertEquals(2, versionRange1.firstActiveVersion()); - assertEquals(3, versionRange1.max()); - - SupportedVersionRange versionRange2 = new SupportedVersionRange((short) 2, (short) 3); - assertEquals(2, versionRange2.min()); - assertEquals(versionRange2.firstActiveVersion(), versionRange2.min()); - assertEquals(3, versionRange2.max()); + public void testMinMax() { + SupportedVersionRange versionRange = new SupportedVersionRange((short) 1, (short) 2); + assertEquals(1, versionRange.min()); + assertEquals(2, versionRange.max()); } } diff --git a/clients/src/test/java/org/apache/kafka/common/requests/ApiVersionsResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/ApiVersionsResponseTest.java index 41b8143039f4e..fbb2d0c5ced51 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/ApiVersionsResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/ApiVersionsResponseTest.java @@ -108,7 +108,7 @@ public void shouldReturnFeatureKeysWhenMagicIsCurrentValueAndThrottleMsIsDefault ApiVersionsResponse response = ApiVersionsResponse.apiVersionsResponse( 10, RecordBatch.MAGIC_VALUE_V1, - Features.supportedFeatures(Utils.mkMap(Utils.mkEntry("feature", new SupportedVersionRange((short) 1, (short) 2, (short) 4)))), + Features.supportedFeatures(Utils.mkMap(Utils.mkEntry("feature", new SupportedVersionRange((short) 1, (short) 4)))), Features.finalizedFeatures(Utils.mkMap(Utils.mkEntry("feature", new FinalizedVersionRange((short) 2, (short) 3)))), 10); verifyApiKeysForMagic(response, RecordBatch.MAGIC_VALUE_V1); @@ -118,7 +118,6 @@ public void shouldReturnFeatureKeysWhenMagicIsCurrentValueAndThrottleMsIsDefault SupportedFeatureKey sKey = response.data.supportedFeatures().find("feature"); assertNotNull(sKey); assertEquals(1, sKey.minVersion()); - assertEquals(2, sKey.firstActiveVersion()); assertEquals(4, sKey.maxVersion()); assertEquals(1, response.data.finalizedFeatures().size()); diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index ad43d47eeb2be..43eddcc5cfd2c 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -141,11 +141,6 @@ class KafkaController(val config: KafkaConfig, newGauge("TopicsIneligibleToDeleteCount", () => ineligibleTopicsToDeleteCount) newGauge("ReplicasIneligibleToDeleteCount", () => ineligibleReplicasToDeleteCount) - class IncompatibleFeatureException(message: String, t: Throwable) extends RuntimeException(message, t) { - def this(message: String) = this(message, null) - def this(t: Throwable) = this("", t) - } - /** * Returns true if this broker is the current controller. */ @@ -306,9 +301,9 @@ class KafkaController(val config: KafkaConfig, * * 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 + * 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. * @@ -337,14 +332,14 @@ class KafkaController(val config: KafkaConfig, * supported features as its finalized features. * * 2. Broker binary upgraded, but IBP config set to lower than KAFKA_2_7_IV0: - * Imagine there is an existing Kafka cluster with IBP config less than KAFKA_2_7_IV0, and the - * broker binary has been upgraded to a newer version that 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 - * 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. + * 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. @@ -359,24 +354,14 @@ class KafkaController(val config: KafkaConfig, * 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 untouched. + * the node umodified. * * 3. Broker binary upgraded, with existing cluster IBP config >= KAFKA_2_7_IV0: - * Imagine 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 needs to scan the - * existing finalized features and mutate them for the purpose of version level deprecation - * (if needed). - * This is how we handle this case: If an existing finalized feature is defined in the list of - * supported features (maintained in the BrokerFeatures object), then, the existing - * minimum version level is updated to the first active version for that feature. The goal of - * this mutation is to permanently deprecate one or more feature version levels. The range of - * feature version levels deprecated are from the closed range: - * [existing_min_version_level, first_active_version - 1]. - * NOTE: Deprecating a feature version level is a two-step process and involves incompatible - * changes. Please read the documentation for the kafka.server.BrokerFeatures class to learn - * about the deprecation process. + * 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 @@ -386,49 +371,16 @@ class KafkaController(val config: KafkaConfig, * will switch the FeatureZNode status to disabled with empty features. */ private def enableFeatureVersioning(): Unit = { - val supportedFeatures = brokerFeatures.supportedFeatures val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path) if (version == ZkVersion.UnknownVersion) { - val finalizedFeatures = supportedFeatures.features.asScala.map { - case(name, versionRange) => (name, new FinalizedVersionRange(versionRange.firstActiveVersion, versionRange.max)) - }.asJava val newVersion = createFeatureZNode(new FeatureZNode(FeatureZNodeStatus.Enabled, - Features.finalizedFeatures(finalizedFeatures))) + brokerFeatures.defaultFinalizedFeatures)) featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs) } else { val existingFeatureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get) - val newFeatures: Features[FinalizedVersionRange] = existingFeatureZNode.status match { - case FeatureZNodeStatus.Disabled => Features.emptyFinalizedFeatures() - case FeatureZNodeStatus.Enabled => - Features.finalizedFeatures(existingFeatureZNode.features.features().asScala.map { - case (featureName, finalizedVersionRange) => - val supportedVersionRange = supportedFeatures.get(featureName) - if (finalizedVersionRange.isIncompatibleWith(supportedVersionRange)) { - // This is a serious error. We should never be reaching here. - throw new IncompatibleFeatureException( - s"Can not update minimum version level in finalized feature: $featureName," - + s" since the existing $finalizedVersionRange is incompatible with" - + s" the $supportedVersionRange. This should never happen since feature version" - + s" incompatibilities are already checked during Kafka server startup.") - } else { - // If supportedVersionRange.firstActiveVersion() > finalizedVersionRange.min(), then - // we go ahead and deprecate all version levels in the range: - // [finalizedVersionRange.min(), supportedVersionRange.firstActiveVersion() - 1]. - // - // Example: - // supportedVersionRange = [minVersion=1, firstActiveVersion=4, maxVersion=7] - // and - // finalizedVersionRange = [minVersionLevel=1, maxVersionLevel=5]. - // - // In this case, we deprecate all version levels in the range: [1, 3]. - val newMinVersionLevel = supportedVersionRange.firstActiveVersion().max(finalizedVersionRange.min()) - (featureName, new FinalizedVersionRange(newMinVersionLevel, finalizedVersionRange.max())) - } - }.asJava) - } - val newFeatureZNode = new FeatureZNode(FeatureZNodeStatus.Enabled, newFeatures) - if (!newFeatureZNode.equals(existingFeatureZNode)) { - val newVersion = updateFeatureZNode(newFeatureZNode) + if (!existingFeatureZNode.status.equals(FeatureZNodeStatus.Enabled)) { + val newVersion = updateFeatureZNode(new FeatureZNode(FeatureZNodeStatus.Enabled, + existingFeatureZNode.features)) featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs) } } @@ -1546,10 +1498,6 @@ class KafkaController(val config: KafkaConfig, error(s"Error while electing or becoming controller on broker ${config.brokerId}. " + s"Trigger controller movement immediately", t) triggerControllerMove() - if (t.isInstanceOf[IncompatibleFeatureException]) { - fatal("Feature version incompatibility found. The controller will eventually exit.") - Exit.exit(1) - } } } @@ -1914,7 +1862,7 @@ class KafkaController(val config: KafkaConfig, } else { var newVersionRange: FinalizedVersionRange = null try { - newVersionRange = new FinalizedVersionRange(supportedVersionRange.firstActiveVersion, update.maxVersionLevel) + newVersionRange = new FinalizedVersionRange(supportedVersionRange.min, update.maxVersionLevel) } catch { case _: IllegalArgumentException => { // This exception means the provided maxVersionLevel is invalid. It is handled below @@ -1925,7 +1873,7 @@ class KafkaController(val config: KafkaConfig, Right(new ApiError(Errors.INVALID_REQUEST, "Could not apply finalized feature update because the provided" + s" maxVersionLevel:${update.maxVersionLevel} is lower than the" + - s" first active version:${supportedVersionRange.firstActiveVersion}.")) + s" supported minVersion:${supportedVersionRange.min}.")) } else { val newFinalizedFeature = Features.finalizedFeatures(Utils.mkMap(Utils.mkEntry(update.feature, newVersionRange))) diff --git a/core/src/main/scala/kafka/server/BrokerFeatures.scala b/core/src/main/scala/kafka/server/BrokerFeatures.scala index 249d7880f6979..dd84f9e73e70f 100644 --- a/core/src/main/scala/kafka/server/BrokerFeatures.scala +++ b/core/src/main/scala/kafka/server/BrokerFeatures.scala @@ -26,79 +26,8 @@ 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. - * The class also enables feature version level deprecation, as explained below. This class is - * immutable in production. It provides few APIs to mutate state only for the purpose of testing. - * - * Feature version level deprecation: - * ================================== - * - * Deprecation of certain version levels of a feature is a process to stop supporting the - * functionality offered by the feature at those version levels, across the entire Kafka cluster. - * Feature version deprecation is a simple 2-step process explained below. In each step below, an - * example is provided to help understand the process better: - * - * STEP 1: - * ======= - * - * In the first step, a major Kafka release is made with a Broker code change (explained later - * below) that establishes the intent to deprecate certain versions of one or more features - * cluster-wide. When this new Kafka release is deployed to the cluster, deprecated finalized - * feature versions are no longer advertised to the client, but they can still be used by existing - * connections. The way it works is that the feature versioning system (via the controller) will - * automatically persist the new minVersionLevel for the feature in ZK to propagate the deprecation - * of certain versions. After this happens, any external client that queries the Broker to learn the - * feature versions will at some point start to see the new value for the finalized minVersionLevel - * for the feature. The external clients are expected to stop using the deprecated versions at least - * by the time that they learn about it. - * - * Here is how the above code change needs to be done: - * In order to deprecate feature version levels, in the supportedFeatures map you need to supply a - * specific firstActiveVersion value that's higher than the minVersion for the feature. The - * value for firstActiveVersion should be 1 beyond the highest version that you intend to deprecate - * for that feature. Whenever the controller is elected or the features are finalized via the - * ApiKeys.UPDATE_FEATURES api, the feature version levels in the closed range: - * [minVersion, firstActiveVersion - 1] are automatically deprecated in ZK by the controller logic. - * - * Example: - * - Let us assume the existing finalized feature in ZK: - * { - * "feature" -> FinalizedVersionRange(minVersionLevel=1, maxVersionLevel=5) - * } - * Now, supposing you would like to deprecate feature version levels: [1, 2]. - * Then, in the supportedFeatures map you should supply the following: - * supportedFeatures = { - * "feature" -> SupportedVersionRange(minVersion=1, firstActiveVersion=3, maxVersion=5) - * } - * - If you do NOT want to deprecate a version level for a feature, then, in the supportedFeatures - * map you should supply the firstActiveVersion to be the same as the minVersion supplied for that - * feature. - * Example: - * supportedFeatures = { - * "feature" -> SupportedVersionRange(minVersion=1, firstActiveVersion=1, maxVersion=5) - * } - * The above indicates no intent to deprecate any version levels for the feature. - * - * STEP 2: - * ======= - * - * After the first step is over, you may (at some point) want to permanently remove the code/logic - * for the functionality offered by the deprecated feature versions. This is the second step. Here a - * subsequent major Kafka release is made with another Broker code change that removes the code for - * the functionality offered by the deprecated feature versions. This would completely drop support - * for the deprecated versions. Such a code change needs to be supplemented by supplying a - * suitable higher minVersion value for the feature in the supportedFeatures map. - * Example: - * - In the example above in step 1, we showed how to deprecate version levels [1, 2] for - * "feature". Now let us assume the following finalized feature in ZK (after the deprecation - * has been carried out): - * { - * "feature" -> FinalizedVersionRange(minVersionLevel=3, maxVersionLevel=5) - * } - * Now, supposing you would like to permanently remove support for feature versions: [1, 2]. - * Then, in the supportedFeatures map you should now supply the following: - * supportedFeatures = { - * "feature" -> SupportedVersionRange(minVersion=3, firstActiveVersion=3, maxVersion=5) - * } + * 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. @@ -114,7 +43,7 @@ class BrokerFeatures private (@volatile var supportedFeatures: Features[Supporte Features.finalizedFeatures( supportedFeatures.features.asScala.map { case(name, versionRange) => ( - name, new FinalizedVersionRange(versionRange.firstActiveVersion, versionRange.max)) + name, new FinalizedVersionRange(versionRange.min, versionRange.max)) }.asJava) } diff --git a/core/src/test/scala/unit/kafka/cluster/BrokerEndPointTest.scala b/core/src/test/scala/unit/kafka/cluster/BrokerEndPointTest.scala index 2f3850b802bdd..d3b52ae5cf5f9 100644 --- a/core/src/test/scala/unit/kafka/cluster/BrokerEndPointTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/BrokerEndPointTest.scala @@ -195,8 +195,8 @@ class BrokerEndPointTest { assertEquals(Some("dc1"), broker.rack) assertEquals(Features.supportedFeatures( Map[String, SupportedVersionRange]( - "feature1" -> new SupportedVersionRange(1, 1, 2), - "feature2" -> new SupportedVersionRange(2, 2, 4)).asJava), + "feature1" -> new SupportedVersionRange(1, 2), + "feature2" -> new SupportedVersionRange(2, 4)).asJava), broker.features) } diff --git a/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala b/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala index 97e69849b6473..29998260806ba 100644 --- a/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala @@ -24,16 +24,12 @@ import com.yammer.metrics.core.Timer 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.TestUtils.waitUntilTrue import kafka.utils.{LogCaptureAppender, TestUtils} -import kafka.zk._ -import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, DescribeFeaturesOptions, FeatureUpdate, UpdateFeaturesOptions} +import kafka.zk.{FeatureZNodeStatus, _} import org.apache.kafka.common.errors.{ControllerMovedException, StaleBrokerEpochException} -import org.apache.kafka.common.feature.{Features, SupportedVersionRange} +import org.apache.kafka.common.feature.Features import org.apache.kafka.common.metrics.KafkaMetric -import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.protocol.Errors -import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.{ElectionType, TopicPartition} import org.apache.log4j.Level import org.junit.Assert.{assertEquals, assertTrue} @@ -601,131 +597,23 @@ class ControllerIntegrationTest extends ZooKeeperTestHarness { } @Test - def testControllerFeatureZNodeSetupWhenFeatureVersioningIsEnabled(): Unit = { - testControllerFeatureZNodeSetup(KAFKA_2_7_IV0) + def testControllerFeatureZNodeSetupWhenFeatureVersioningIsEnabledWithDisabledExistingNode(): Unit = { + testControllerFeatureZNodeSetup(new FeatureZNode(FeatureZNodeStatus.Disabled, Features.emptyFinalizedFeatures()), KAFKA_2_7_IV0) } @Test - def testControllerFeatureZNodeSetupWhenFeatureVersioningIsDisabled(): Unit = { - testControllerFeatureZNodeSetup(KAFKA_2_6_IV0) + def testControllerFeatureZNodeSetupWhenFeatureVersioningIsEnabledWithEnabledExistingNode(): Unit = { + testControllerFeatureZNodeSetup(new FeatureZNode(FeatureZNodeStatus.Enabled, Features.emptyFinalizedFeatures()), KAFKA_2_7_IV0) } - private def createAdminClient(): Admin = { - val props = new Properties - props.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, - TestUtils.bootstrapServers(servers, - ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT))) - Admin.create(props) - } - - private def updateSupportedFeatures(features: Features[SupportedVersionRange]): Unit = { - servers.foreach(s => { - s.brokerFeatures.setSupportedFeatures(features) - s.zkClient.updateBrokerInfo(s.createBrokerInfo) - }) - - // Wait until updates to supported features in all BrokerZNode propagate to the controller. - waitUntilTrue( - () => servers.exists(s => { - if (s.kafkaController.isActive) { - s.kafkaController.controllerContext.liveOrShuttingDownBrokers - .forall(b => { - b.features.equals(features) - }) - } else { - false - } - }), - "Controller did not get broker supported features updates for too long") - } - - private def finalizedFeaturesEpoch(client: Admin): Long = { - client - .describeFeatures(new DescribeFeaturesOptions().sendRequestToController(true)) - .featureMetadata() - .get() - .finalizedFeaturesEpoch() - .get() + @Test + def testControllerFeatureZNodeSetupWhenFeatureVersioningIsDisabledWithDisabledExistingNode(): Unit = { + testControllerFeatureZNodeSetup(new FeatureZNode(FeatureZNodeStatus.Disabled, Features.emptyFinalizedFeatures()), KAFKA_2_6_IV0) } @Test - def testFeatureVersionDeprecation(): Unit = { - servers = makeServers(3, interBrokerProtocolVersion = Some(KAFKA_2_7_IV0)) - assertTrue(servers.forall(s => s.config.isFeatureVersioningSupported)) - TestUtils.waitUntilControllerElected(zkClient) - - val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path) - assertEquals(0, version) - val featureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get) - assertEquals(FeatureZNodeStatus.Enabled, featureZNode.status) - assertTrue(featureZNode.features.empty) - - var client: Admin = null - try { - // 1. Setup supported features ahead of finalizing them. - val supportedFeatures = Features.supportedFeatures( - new java.util.HashMap[String, SupportedVersionRange] { - put("deprecation_target", new SupportedVersionRange(1, 5)) - put("not_deprecation_target", new SupportedVersionRange(1, 6)) - }) - updateSupportedFeatures(supportedFeatures) - - // 2. Finalize the supported features at the max level. - client = createAdminClient() - client.updateFeatures( - new java.util.HashMap[String, FeatureUpdate] { - put("deprecation_target", new FeatureUpdate(5, false)) - put("not_deprecation_target", new FeatureUpdate(6, false)) - }, - new UpdateFeaturesOptions() - ).all().get() - val epochBeforeControllerFailover = finalizedFeaturesEpoch(client) - - // 3. Setup supported features once again, this time deprecating feature versions [1, 2] - // for the feature "deprecation_target". - val supportedFeaturesWithDeprecatedVersions = Features.supportedFeatures( - new java.util.HashMap[String, SupportedVersionRange] { - put("deprecation_target", new SupportedVersionRange(1, 3, 5)) - put("not_deprecation_target", new SupportedVersionRange(1, 6)) - }) - updateSupportedFeatures(supportedFeaturesWithDeprecatedVersions) - - // 4. Trigger controller failover - val controller = getController().kafkaController - zkClient.deleteController(controller.controllerContext.epochZkVersion) - - // 5. Expect the controller to update the finalized features - val epochAfterControllerFailover = epochBeforeControllerFailover + 1 - TestUtils.waitUntilTrue( - () => { - getController().kafkaController.isActive && - finalizedFeaturesEpoch(client) == epochAfterControllerFailover - }, - "Finalized features are not updated for too long after controller failover") - - // 6. Expect the finalized feature: "deprecation_target" to be updated to [3, 5] (due to - // deprecation of versions [1, 2]) - val featureMetadata = client - .describeFeatures(new DescribeFeaturesOptions().sendRequestToController(true)) - .featureMetadata() - .get() - assertTrue(featureMetadata.finalizedFeaturesEpoch().isPresent) - assertEquals(epochAfterControllerFailover, featureMetadata.finalizedFeaturesEpoch().get()) - - assertEquals(2, featureMetadata.finalizedFeatures().size()) - assertTrue(featureMetadata.finalizedFeatures().containsKey("deprecation_target")) - val deprecationTargetVersionRange = featureMetadata.finalizedFeatures().get("deprecation_target") - assertEquals(3, deprecationTargetVersionRange.minVersionLevel()) - assertEquals(5, deprecationTargetVersionRange.maxVersionLevel()) - - assertTrue(featureMetadata.finalizedFeatures().containsKey("not_deprecation_target")) - val notDeprecationTargetVersionRange = featureMetadata.finalizedFeatures().get("not_deprecation_target") - assertEquals(1, notDeprecationTargetVersionRange.minVersionLevel()) - assertEquals(6, notDeprecationTargetVersionRange.maxVersionLevel()) - } finally { - if (client != null) - client.close() - } + def testControllerFeatureZNodeSetupWhenFeatureVersioningIsDisabledWithEnabledExistingNode(): Unit = { + testControllerFeatureZNodeSetup(new FeatureZNode(FeatureZNodeStatus.Enabled, Features.emptyFinalizedFeatures()), KAFKA_2_6_IV0) } @Test @@ -852,19 +740,32 @@ class ControllerIntegrationTest extends ZooKeeperTestHarness { controller.shutdown() } - private def testControllerFeatureZNodeSetup(interBrokerProtocolVersion: ApiVersion): Unit = { + private def testControllerFeatureZNodeSetup(initialZNode: FeatureZNode, + interBrokerProtocolVersion: ApiVersion): Unit = { + zkClient.updateFeatureZNode(initialZNode) + val (_, versionBefore) = zkClient.getDataAndVersion(FeatureZNode.path) servers = makeServers(1, interBrokerProtocolVersion = Some(interBrokerProtocolVersion)) TestUtils.waitUntilControllerElected(zkClient) - val (mayBeFeatureZNodeBytes, version) = zkClient.getDataAndVersion(FeatureZNode.path) - assertEquals(0, version) - val featureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get) + val (mayBeFeatureZNodeBytes, versionAfter) = zkClient.getDataAndVersion(FeatureZNode.path) + val newZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get) if (interBrokerProtocolVersion >= KAFKA_2_7_IV0) { - assertEquals(FeatureZNodeStatus.Enabled, featureZNode.status) + if (initialZNode.status == FeatureZNodeStatus.Enabled) { + assertEquals(versionBefore, versionAfter) + assertEquals(initialZNode, newZNode) + } else if (initialZNode.status == FeatureZNodeStatus.Disabled) { + assertEquals(versionBefore + 1, versionAfter) + assertEquals(new FeatureZNode(FeatureZNodeStatus.Enabled, Features.emptyFinalizedFeatures()), newZNode) + } } else { - assertEquals(FeatureZNodeStatus.Disabled, featureZNode.status) + if (initialZNode.status == FeatureZNodeStatus.Enabled) { + assertEquals(versionBefore + 1, versionAfter) + assertEquals(new FeatureZNode(FeatureZNodeStatus.Disabled, Features.emptyFinalizedFeatures()), newZNode) + } else if (initialZNode.status == FeatureZNodeStatus.Disabled) { + assertEquals(versionBefore, versionAfter) + assertEquals(initialZNode, newZNode) + } } - assertTrue(featureZNode.features.empty) } @Test diff --git a/core/src/test/scala/unit/kafka/server/BrokerFeaturesTest.scala b/core/src/test/scala/unit/kafka/server/BrokerFeaturesTest.scala index bc04cd9bfb266..2fe08fb3e1317 100644 --- a/core/src/test/scala/unit/kafka/server/BrokerFeaturesTest.scala +++ b/core/src/test/scala/unit/kafka/server/BrokerFeaturesTest.scala @@ -34,8 +34,8 @@ class BrokerFeaturesTest { def testIncompatibilitiesDueToAbsentFeature(): Unit = { val brokerFeatures = BrokerFeatures.createDefault() val supportedFeatures = Features.supportedFeatures(Map[String, SupportedVersionRange]( - "test_feature_1" -> new SupportedVersionRange(1, 1, 4), - "test_feature_2" -> new SupportedVersionRange(1, 1, 3)).asJava) + "test_feature_1" -> new SupportedVersionRange(1, 4), + "test_feature_2" -> new SupportedVersionRange(1, 3)).asJava) brokerFeatures.setSupportedFeatures(supportedFeatures) val compatibleFeatures = Map[String, FinalizedVersionRange]( @@ -55,8 +55,8 @@ class BrokerFeaturesTest { def testIncompatibilitiesDueToIncompatibleFeature(): Unit = { val brokerFeatures = BrokerFeatures.createDefault() val supportedFeatures = Features.supportedFeatures(Map[String, SupportedVersionRange]( - "test_feature_1" -> new SupportedVersionRange(1, 1, 4), - "test_feature_2" -> new SupportedVersionRange(1, 1, 3)).asJava) + "test_feature_1" -> new SupportedVersionRange(1, 4), + "test_feature_2" -> new SupportedVersionRange(1, 3)).asJava) brokerFeatures.setSupportedFeatures(supportedFeatures) val compatibleFeatures = Map[String, FinalizedVersionRange]( @@ -72,33 +72,12 @@ class BrokerFeaturesTest { assertTrue(BrokerFeatures.hasIncompatibleFeatures(supportedFeatures, finalizedFeatures)) } - @Test - def testIncompatibilitiesWithFirstActiveVersion(): Unit = { - val brokerFeatures = BrokerFeatures.createDefault() - val supportedFeatures = Features.supportedFeatures(Map[String, SupportedVersionRange]( - "test_feature_1" -> new SupportedVersionRange(1, 2, 4), - "test_feature_2" -> new SupportedVersionRange(1, 3, 4)).asJava) - brokerFeatures.setSupportedFeatures(supportedFeatures) - - val compatibleFeatures = Map[String, FinalizedVersionRange]( - "test_feature_1" -> new FinalizedVersionRange(1, 2)) - val inCompatibleFeatures = Map[String, FinalizedVersionRange]( - "test_feature_2" -> new FinalizedVersionRange(1, 2)) - 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, 2, 4), - "test_feature_2" -> new SupportedVersionRange(1, 3, 3)).asJava) + "test_feature_1" -> new SupportedVersionRange(1, 4), + "test_feature_2" -> new SupportedVersionRange(1, 3)).asJava) brokerFeatures.setSupportedFeatures(supportedFeatures) val compatibleFeatures = Map[String, FinalizedVersionRange]( @@ -113,15 +92,15 @@ class BrokerFeaturesTest { def testDefaultFinalizedFeatures(): Unit = { val brokerFeatures = BrokerFeatures.createDefault() val supportedFeatures = Features.supportedFeatures(Map[String, SupportedVersionRange]( - "test_feature_1" -> new SupportedVersionRange(1, 2, 4), - "test_feature_2" -> new SupportedVersionRange(1, 3, 3), - "test_feature_3" -> new SupportedVersionRange(3, 5, 7)).asJava) + "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(2, 4), - "test_feature_2" -> new FinalizedVersionRange(3, 3), - "test_feature_3" -> new FinalizedVersionRange(5, 7)) + "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 eb888f9487a47..2f0f70ee0bad3 100644 --- a/core/src/test/scala/unit/kafka/server/FinalizedFeatureCacheTest.scala +++ b/core/src/test/scala/unit/kafka/server/FinalizedFeatureCacheTest.scala @@ -33,7 +33,7 @@ class FinalizedFeatureCacheTest { @Test def testUpdateOrThrowFailedDueToInvalidEpoch(): Unit = { val supportedFeatures = Map[String, SupportedVersionRange]( - "feature_1" -> new SupportedVersionRange(1, 2, 4)) + "feature_1" -> new SupportedVersionRange(1, 4)) val brokerFeatures = BrokerFeatures.createDefault() brokerFeatures.setSupportedFeatures(Features.supportedFeatures(supportedFeatures.asJava)) @@ -59,8 +59,8 @@ class FinalizedFeatureCacheTest { @Test def testUpdateOrThrowFailedDueToInvalidFeatures(): Unit = { - val supportedFeatures = Map[String, SupportedVersionRange]( - "feature_1" -> new SupportedVersionRange(1, 1, 1)) + val supportedFeatures = + Map[String, SupportedVersionRange]("feature_1" -> new SupportedVersionRange(1, 1)) val brokerFeatures = BrokerFeatures.createDefault() brokerFeatures.setSupportedFeatures(Features.supportedFeatures(supportedFeatures.asJava)) @@ -79,8 +79,8 @@ class FinalizedFeatureCacheTest { @Test def testUpdateOrThrowSuccess(): Unit = { - val supportedFeatures = Map[String, SupportedVersionRange]( - "feature_1" -> new SupportedVersionRange(1, 2, 4)) + val supportedFeatures = + Map[String, SupportedVersionRange]("feature_1" -> new SupportedVersionRange(1, 4)) val brokerFeatures = BrokerFeatures.createDefault() brokerFeatures.setSupportedFeatures(Features.supportedFeatures(supportedFeatures.asJava)) @@ -97,8 +97,8 @@ class FinalizedFeatureCacheTest { @Test def testClear(): Unit = { - val supportedFeatures = Map[String, SupportedVersionRange]( - "feature_1" -> new SupportedVersionRange(1, 2, 4)) + val supportedFeatures = + Map[String, SupportedVersionRange]("feature_1" -> new SupportedVersionRange(1, 4)) val brokerFeatures = BrokerFeatures.createDefault() brokerFeatures.setSupportedFeatures(Features.supportedFeatures(supportedFeatures.asJava)) diff --git a/core/src/test/scala/unit/kafka/server/FinalizedFeatureChangeListenerTest.scala b/core/src/test/scala/unit/kafka/server/FinalizedFeatureChangeListenerTest.scala index e67be406921d5..cb8a661317794 100644 --- a/core/src/test/scala/unit/kafka/server/FinalizedFeatureChangeListenerTest.scala +++ b/core/src/test/scala/unit/kafka/server/FinalizedFeatureChangeListenerTest.scala @@ -33,8 +33,8 @@ class FinalizedFeatureChangeListenerTest extends ZooKeeperTestHarness { private def createBrokerFeatures(): BrokerFeatures = { val supportedFeaturesMap = Map[String, SupportedVersionRange]( - "feature_1" -> new SupportedVersionRange(1, 2, 4), - "feature_2" -> new SupportedVersionRange(1, 2, 3)) + "feature_1" -> new SupportedVersionRange(1, 4), + "feature_2" -> new SupportedVersionRange(1, 3)) val brokerFeatures = BrokerFeatures.createDefault() brokerFeatures.setSupportedFeatures(Features.supportedFeatures(supportedFeaturesMap.asJava)) brokerFeatures diff --git a/core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala b/core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala index 3d71f70ea5ec2..3822c4238ff7a 100644 --- a/core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala +++ b/core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala @@ -50,7 +50,7 @@ class UpdateFeaturesTest extends BaseRequestTest { } private def defaultSupportedFeatures(): Features[SupportedVersionRange] = { - Features.supportedFeatures(Utils.mkMap(Utils.mkEntry("feature_1", new SupportedVersionRange(1, 1, 3)))) + Features.supportedFeatures(Utils.mkMap(Utils.mkEntry("feature_1", new SupportedVersionRange(1, 3)))) } private def defaultFinalizedFeatures(): Features[FinalizedVersionRange] = { @@ -111,7 +111,7 @@ class UpdateFeaturesTest extends BaseRequestTest { 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.firstActiveVersion(), versionRange.maxVersion())) + (name, new SupportedVersionRange(versionRange.minVersion(), versionRange.maxVersion())) }.asJava) } @@ -302,72 +302,6 @@ class UpdateFeaturesTest extends BaseRequestTest { ".*Can not upgrade a finalized feature.*to the same value.*".r) } - private def testShouldFailRequestWhenNewMaxVersionLevelIsBelowFirstActiveVersion( - featureName: String, - supportedVersionRange: SupportedVersionRange, - initialFinalizedVersionRange: Option[FinalizedVersionRange], - allowDowngrade: Boolean - ): Unit = { - TestUtils.waitUntilControllerElected(zkClient) - - val supportedFeatures = Features.supportedFeatures(Utils.mkMap(Utils.mkEntry(featureName, supportedVersionRange))) - updateSupportedFeaturesInAllBrokers(supportedFeatures) - val initialFinalizedFeatures = initialFinalizedVersionRange.map( - versionRange => Features.finalizedFeatures(Utils.mkMap(Utils.mkEntry(featureName, versionRange))) - ).getOrElse(Features.emptyFinalizedFeatures()) - val versionBefore = updateFeatureZNode(initialFinalizedFeatures) - - val newMaxVersionLevel = (supportedVersionRange.firstActiveVersion() - 1).asInstanceOf[Short] - val update = new FeatureUpdate(newMaxVersionLevel, allowDowngrade) - val adminClient = createAdminClient() - val nodeBefore = getFeatureZNode() - - val result = adminClient.updateFeatures(Utils.mkMap(Utils.mkEntry(featureName, update)), new UpdateFeaturesOptions()) - - checkException[InvalidRequestException]( - result, - Map(featureName -> s".*maxVersionLevel:$newMaxVersionLevel.*".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 DOWNGRADE is attempted to a value thats below the first active version - * for the feature. - */ - @Test - def testShouldFailRequestWhenDowngradingBelowFirstActiveVersionOfAnExistingFinalizedFeature(): Unit = { - testShouldFailRequestWhenNewMaxVersionLevelIsBelowFirstActiveVersion( - "feature_1", new SupportedVersionRange(1, 2, 3), Some(new FinalizedVersionRange(2, 3)), true) - } - - /** - * Tests that an UpdateFeatures request fails in the Controller, when, for a non-existing finalized - * feature, a version level UPGRADE is attempted to a value thats below the first active version - * for the feature. - */ - @Test - def testShouldFailRequestWhenUpgradingBelowFirstActiveVersionOfANonExistingFinalizedFeature(): Unit = { - testShouldFailRequestWhenNewMaxVersionLevelIsBelowFirstActiveVersion( - "feature_1", new SupportedVersionRange(1, 2, 3), Option.empty, false) - } - - /** - * Tests that an UpdateFeatures request fails in the Controller, when for a non-existing finalized - * feature, a version level DOWNGRADE is attempted to a value thats below the first active version - * level for the feature. - */ - @Test - def testShouldFailRequestWhenDowngradingBelowFirstActiveVersionOfANonExistingFinalizedFeature(): Unit = { - testShouldFailRequestWhenNewMaxVersionLevelIsBelowFirstActiveVersion( - "feature_1", new SupportedVersionRange(1, 2, 3), Option.empty, true) - } - private def testShouldFailRequestDuringBrokerMaxVersionLevelIncompatibility( featureName: String, supportedVersionRange: SupportedVersionRange, @@ -392,7 +326,6 @@ class UpdateFeaturesTest extends BaseRequestTest { Utils.mkEntry("feature_1", new SupportedVersionRange( supportedVersionRange.min(), - supportedVersionRange.firstActiveVersion(), unsupportedMaxVersion)))) updateSupportedFeatures(supportedFeaturesWithVersionIncompatibility, brokersWithVersionIncompatibility) @@ -456,8 +389,8 @@ class UpdateFeaturesTest extends BaseRequestTest { val supportedFeatures = Features.supportedFeatures( Utils.mkMap( - Utils.mkEntry("feature_1", new SupportedVersionRange(1, 1, 3)), - Utils.mkEntry("feature_2", new SupportedVersionRange(2, 2, 5)))) + Utils.mkEntry("feature_1", new SupportedVersionRange(1, 3)), + Utils.mkEntry("feature_2", new SupportedVersionRange(2, 5)))) updateSupportedFeaturesInAllBrokers(supportedFeatures) val versionBefore = updateFeatureZNode(Features.emptyFinalizedFeatures()) @@ -492,8 +425,8 @@ class UpdateFeaturesTest extends BaseRequestTest { val supportedFeatures = Features.supportedFeatures( Utils.mkMap( - Utils.mkEntry("feature_1", new SupportedVersionRange(1, 1, 3)), - Utils.mkEntry("feature_2", new SupportedVersionRange(2, 2, 5)))) + Utils.mkEntry("feature_1", new SupportedVersionRange(1, 3)), + Utils.mkEntry("feature_2", new SupportedVersionRange(2, 5)))) updateSupportedFeaturesInAllBrokers(supportedFeatures) val initialFinalizedFeatures = Features.finalizedFeatures( Utils.mkMap( @@ -536,8 +469,8 @@ class UpdateFeaturesTest extends BaseRequestTest { val supportedFeatures = Features.supportedFeatures( Utils.mkMap( - Utils.mkEntry("feature_1", new SupportedVersionRange(1, 1, 3)), - Utils.mkEntry("feature_2", new SupportedVersionRange(2, 2, 5)))) + Utils.mkEntry("feature_1", new SupportedVersionRange(1, 3)), + Utils.mkEntry("feature_2", new SupportedVersionRange(2, 5)))) updateSupportedFeaturesInAllBrokers(supportedFeatures) val initialFinalizedFeatures = Features.finalizedFeatures( Utils.mkMap( @@ -597,13 +530,13 @@ class UpdateFeaturesTest extends BaseRequestTest { val supportedFeatures = Features.supportedFeatures( Utils.mkMap( - Utils.mkEntry("feature_1", new SupportedVersionRange(1, 2, 3)), - Utils.mkEntry("feature_2", new SupportedVersionRange(2, 2, 5)))) + 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, 2)), + Utils.mkEntry("feature_1", new SupportedVersionRange(1, 2)), Utils.mkEntry("feature_2", supportedFeatures.get("feature_2")))) updateSupportedFeatures(supportedFeaturesWithVersionIncompatibility, brokersWithVersionIncompatibility) diff --git a/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala b/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala index 1fd33653e68cc..67445004c0f8c 100644 --- a/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala +++ b/core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala @@ -773,12 +773,12 @@ class KafkaZkClientTest extends ZooKeeperTestHarness { rack = None, features = Features.supportedFeatures( Map[String, SupportedVersionRange]( - "feature1" -> new SupportedVersionRange(1, 1, 2)).asJava)) + "feature1" -> new SupportedVersionRange(1, 2)).asJava)) val differentBrokerInfoWithSameId = createBrokerInfo( 1, "test.host2", 9995, SecurityProtocol.SSL, features = Features.supportedFeatures( Map[String, SupportedVersionRange]( - "feature2" -> new SupportedVersionRange(4, 4, 7)).asJava)) + "feature2" -> new SupportedVersionRange(4, 7)).asJava)) zkClient.registerBroker(brokerInfo) assertEquals(Some(brokerInfo.broker), zkClient.getBroker(1)) @@ -832,12 +832,12 @@ class KafkaZkClientTest extends ZooKeeperTestHarness { 0, "test.host0", 9998, SecurityProtocol.PLAINTEXT, features = Features.supportedFeatures( Map[String, SupportedVersionRange]( - "feature1" -> new SupportedVersionRange(1, 1, 2)).asJava)) + "feature1" -> new SupportedVersionRange(1, 2)).asJava)) val brokerInfo1 = createBrokerInfo( 1, "test.host1", 9999, SecurityProtocol.SSL, features = Features.supportedFeatures( Map[String, SupportedVersionRange]( - "feature2" -> new SupportedVersionRange(3, 3, 6)).asJava)) + "feature2" -> new SupportedVersionRange(3, 6)).asJava)) zkClient.registerBroker(brokerInfo1) otherZkClient.registerBroker(brokerInfo0) From 0ba831d3fcf0a7b6c3099e87491114fe17d4d634 Mon Sep 17 00:00:00 2001 From: Kowshik Prakasam Date: Fri, 2 Oct 2020 02:17:39 -0700 Subject: [PATCH 36/41] Minor change to code format --- .../unit/kafka/controller/ControllerIntegrationTest.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala b/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala index 29998260806ba..604c2c6b0ab28 100644 --- a/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala @@ -700,7 +700,6 @@ class ControllerIntegrationTest extends ZooKeeperTestHarness { controller.eventManager.thread = spyThread val processedEvent = new MockEvent(ControllerState.TopicChange) { override def process(): Unit = latch.await() - override def preempt(): Unit = {} } val tp0 = new TopicPartition("t", 0) @@ -719,7 +718,7 @@ class ControllerIntegrationTest extends ZooKeeperTestHarness { case Failure(e) => assertEquals(classOf[ControllerMovedException], e.getClass) }) - val event3 = ApiPartitionReassignment(Map(tp0 -> None, tp1 -> None), { + val event3 = ApiPartitionReassignment(Map(tp0 -> None, tp1 -> None), { case Left(_) => fail("api partition reassignment should error") case Right(e) => assertEquals(Errors.NOT_CONTROLLER, e.error()) }) From c821e8501793a40e40f461de7c96126840f4a02d Mon Sep 17 00:00:00 2001 From: Kowshik Prakasam Date: Fri, 2 Oct 2020 11:33:34 -0700 Subject: [PATCH 37/41] Fix ControllerIntegrationTest --- .../scala/unit/kafka/controller/ControllerIntegrationTest.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala b/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala index 604c2c6b0ab28..5286164698eac 100644 --- a/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala @@ -741,7 +741,7 @@ class ControllerIntegrationTest extends ZooKeeperTestHarness { private def testControllerFeatureZNodeSetup(initialZNode: FeatureZNode, interBrokerProtocolVersion: ApiVersion): Unit = { - zkClient.updateFeatureZNode(initialZNode) + zkClient.createFeatureZNode(initialZNode) val (_, versionBefore) = zkClient.getDataAndVersion(FeatureZNode.path) servers = makeServers(1, interBrokerProtocolVersion = Some(interBrokerProtocolVersion)) TestUtils.waitUntilControllerElected(zkClient) From 5e3fc96c48794d80ccaf56e6dfbfdde0c30313c5 Mon Sep 17 00:00:00 2001 From: Kowshik Prakasam Date: Fri, 2 Oct 2020 13:17:59 -0700 Subject: [PATCH 38/41] Minor cosmetic changes --- .../apache/kafka/clients/admin/KafkaAdminClient.java | 3 ++- .../kafka/common/feature/FinalizedVersionRange.java | 3 +-- .../kafka/common/requests/ApiVersionsResponse.java | 10 +++++----- 3 files changed, 8 insertions(+), 8 deletions(-) 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 d82b0ff97fd75..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 @@ -4349,7 +4349,7 @@ public DescribeFeaturesResult describeFeatures(final DescribeFeaturesOptions opt final NodeProvider provider = options.sendRequestToController() ? new ControllerNodeProvider() : new LeastLoadedNodeProvider(); - Call call = new Call( + final Call call = new Call( "describeFeatures", calcDeadlineMs(now, options.timeoutMs()), provider) { private FeatureMetadata createFeatureMetadata(final ApiVersionsResponse response) { @@ -4412,6 +4412,7 @@ public UpdateFeaturesResult updateFeatures(final Map feat 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()) { 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 ff778422af089..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 @@ -48,7 +48,6 @@ public static FinalizedVersionRange fromMap(Map versionRangeMap) * - false otherwise */ public boolean isIncompatibleWith(SupportedVersionRange supportedVersionRange) { - return min() < supportedVersionRange.min() || - max() > supportedVersionRange.max(); + return min() < supportedVersionRange.min() || max() > supportedVersionRange.max(); } } 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 19d5593042747..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 @@ -204,7 +204,7 @@ public static ApiVersionsResponseData createApiVersionsResponseData( final Features finalizedFeatures, final long finalizedFeaturesEpoch ) { - ApiVersionsResponseData data = new ApiVersionsResponseData(); + final ApiVersionsResponseData data = new ApiVersionsResponseData(); data.setThrottleTimeMs(throttleTimeMs); data.setErrorCode(error.code()); data.setApiKeys(apiKeys); @@ -219,8 +219,8 @@ private static SupportedFeatureKeyCollection createSupportedFeatureKeys( Features latestSupportedFeatures) { SupportedFeatureKeyCollection converted = new SupportedFeatureKeyCollection(); for (Map.Entry feature : latestSupportedFeatures.features().entrySet()) { - SupportedFeatureKey key = new SupportedFeatureKey(); - SupportedVersionRange versionRange = feature.getValue(); + final SupportedFeatureKey key = new SupportedFeatureKey(); + final SupportedVersionRange versionRange = feature.getValue(); key.setName(feature.getKey()); key.setMinVersion(versionRange.min()); key.setMaxVersion(versionRange.max()); @@ -234,8 +234,8 @@ private static FinalizedFeatureKeyCollection createFinalizedFeatureKeys( Features finalizedFeatures) { FinalizedFeatureKeyCollection converted = new FinalizedFeatureKeyCollection(); for (Map.Entry feature : finalizedFeatures.features().entrySet()) { - FinalizedFeatureKey key = new FinalizedFeatureKey(); - FinalizedVersionRange versionLevelRange = feature.getValue(); + final FinalizedFeatureKey key = new FinalizedFeatureKey(); + final FinalizedVersionRange versionLevelRange = feature.getValue(); key.setName(feature.getKey()); key.setMinVersionLevel(versionLevelRange.min()); key.setMaxVersionLevel(versionLevelRange.max()); From 3c5c04f5910c50dd7f3e02b4a1f33c50d42fb43f Mon Sep 17 00:00:00 2001 From: Kowshik Prakasam Date: Fri, 2 Oct 2020 18:19:10 -0700 Subject: [PATCH 39/41] Address comments from Jun --- .../clients/admin/DescribeFeaturesResult.java | 2 +- .../clients/admin/UpdateFeaturesResult.java | 2 +- .../kafka/controller/KafkaController.scala | 37 +++++++-- .../kafka/server/FinalizedFeatureCache.scala | 4 +- .../ControllerIntegrationTest.scala | 80 +++++++++++++------ 5 files changed, 90 insertions(+), 35 deletions(-) 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 index 9245d9b86b33a..c48dc19143077 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/DescribeFeaturesResult.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/DescribeFeaturesResult.java @@ -27,7 +27,7 @@ public class DescribeFeaturesResult { private final KafkaFuture future; - public DescribeFeaturesResult(KafkaFuture future) { + DescribeFeaturesResult(KafkaFuture future) { this.future = future; } 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 index d0d27aedb6298..6c484dc24d95b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/UpdateFeaturesResult.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/UpdateFeaturesResult.java @@ -31,7 +31,7 @@ public class UpdateFeaturesResult { * @param futures a map from feature name to future, which can be used to check the status of * individual feature updates. */ - public UpdateFeaturesResult(final Map> futures) { + UpdateFeaturesResult(final Map> futures) { this.futures = futures; } diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 43eddcc5cfd2c..b13971ff73c31 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -378,9 +378,18 @@ class KafkaController(val config: KafkaConfig, featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs) } else { val existingFeatureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get) - if (!existingFeatureZNode.status.equals(FeatureZNodeStatus.Enabled)) { - val newVersion = updateFeatureZNode(new FeatureZNode(FeatureZNodeStatus.Enabled, - existingFeatureZNode.features)) + 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" + + " contains non-empty features.") + } + Features.emptyFinalizedFeatures + } + val newFeatureZNode = new FeatureZNode(FeatureZNodeStatus.Enabled, newFeatures) + if (!newFeatureZNode.equals(existingFeatureZNode)) { + val newVersion = updateFeatureZNode(newFeatureZNode) featureCache.waitUntilEpochOrThrow(newVersion, config.zkConnectionTimeoutMs) } } @@ -408,7 +417,12 @@ class KafkaController(val config: KafkaConfig, createFeatureZNode(newNode) } else { val existingFeatureZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get) - if (!existingFeatureZNode.status.equals(FeatureZNodeStatus.Disabled)) { + if (existingFeatureZNode.status == FeatureZNodeStatus.Disabled && + !existingFeatureZNode.features.empty()) { + warn(s"FeatureZNode at path: ${FeatureZNode.path} with disabled status" + + " contains non-empty features.") + } + if (!newNode.equals(existingFeatureZNode)) { updateFeatureZNode(newNode) } } @@ -878,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) @@ -1554,9 +1573,10 @@ class KafkaController(val config: KafkaConfig, if (newBrokerIds.nonEmpty) { val (newCompatibleBrokerAndEpochs, newIncompatibleBrokerAndEpochs) = partitionOnFeatureCompatibility(newBrokerAndEpochs) - if (!newIncompatibleBrokerAndEpochs.isEmpty) + 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) } @@ -1565,9 +1585,10 @@ class KafkaController(val config: KafkaConfig, onBrokerFailure(bouncedBrokerIdsSorted) val (bouncedCompatibleBrokerAndEpochs, bouncedIncompatibleBrokerAndEpochs) = partitionOnFeatureCompatibility(bouncedBrokerAndEpochs) - if (!bouncedIncompatibleBrokerAndEpochs.isEmpty) + 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) } @@ -2019,7 +2040,7 @@ class KafkaController(val config: KafkaConfig, try { if (!existingFeatures.equals(targetFeatures)) { val newNode = new FeatureZNode(FeatureZNodeStatus.Enabled, Features.finalizedFeatures(targetFeatures.asJava)) - val newVersion = zkClient.updateFeatureZNode(newNode) + val newVersion = updateFeatureZNode(newNode) featureCache.waitUntilEpochOrThrow(newVersion, request.data().timeoutMs()) } } catch { diff --git a/core/src/main/scala/kafka/server/FinalizedFeatureCache.scala b/core/src/main/scala/kafka/server/FinalizedFeatureCache.scala index 32fb5cdbe580c..898657d732e4e 100644 --- a/core/src/main/scala/kafka/server/FinalizedFeatureCache.scala +++ b/core/src/main/scala/kafka/server/FinalizedFeatureCache.scala @@ -145,7 +145,7 @@ class FinalizedFeatureCache(private val brokerFeatures: BrokerFeatures) extends if(timeoutMs < 0L) { throw new IllegalArgumentException(s"Expected timeoutMs >= 0, but $timeoutMs was provided.") } - val waitEndTimeNanos = System.nanoTime() + (timeoutMs * 1_000_000) + val waitEndTimeNanos = System.nanoTime() + (timeoutMs * 1000000) synchronized { while (!waitCondition()) { val nowNanos = System.nanoTime() @@ -154,7 +154,7 @@ class FinalizedFeatureCache(private val brokerFeatures: BrokerFeatures) extends 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) / 1_000_000) + val sleepTimeMs = max(1L, (waitEndTimeNanos - nowNanos) / 1000000) wait(sleepTimeMs) } } diff --git a/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala b/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala index 5286164698eac..9ceedf80cb0ed 100644 --- a/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala @@ -596,24 +596,35 @@ 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 testControllerFeatureZNodeSetupWhenFeatureVersioningIsEnabledWithDisabledExistingNode(): Unit = { - testControllerFeatureZNodeSetup(new FeatureZNode(FeatureZNodeStatus.Disabled, Features.emptyFinalizedFeatures()), KAFKA_2_7_IV0) + def testControllerFeatureZNodeSetupWhenFeatureVersioningIsEnabledWithEnabledExistingFeatureZNode(): Unit = { + testControllerFeatureZNodeSetup(Some(new FeatureZNode(FeatureZNodeStatus.Enabled, Features.emptyFinalizedFeatures())), KAFKA_2_7_IV0) } @Test - def testControllerFeatureZNodeSetupWhenFeatureVersioningIsEnabledWithEnabledExistingNode(): Unit = { - testControllerFeatureZNodeSetup(new FeatureZNode(FeatureZNodeStatus.Enabled, Features.emptyFinalizedFeatures()), KAFKA_2_7_IV0) + def testControllerFeatureZNodeSetupWhenFeatureVersioningIsDisabledWithNonExistingFeatureZNode(): Unit = { + testControllerFeatureZNodeSetup(Option.empty, KAFKA_2_6_IV0) } @Test - def testControllerFeatureZNodeSetupWhenFeatureVersioningIsDisabledWithDisabledExistingNode(): Unit = { - testControllerFeatureZNodeSetup(new FeatureZNode(FeatureZNodeStatus.Disabled, Features.emptyFinalizedFeatures()), KAFKA_2_6_IV0) + def testControllerFeatureZNodeSetupWhenFeatureVersioningIsDisabledWithDisabledExistingFeatureZNode(): Unit = { + testControllerFeatureZNodeSetup(Some(new FeatureZNode(FeatureZNodeStatus.Disabled, Features.emptyFinalizedFeatures())), KAFKA_2_6_IV0) } @Test - def testControllerFeatureZNodeSetupWhenFeatureVersioningIsDisabledWithEnabledExistingNode(): Unit = { - testControllerFeatureZNodeSetup(new FeatureZNode(FeatureZNodeStatus.Enabled, Features.emptyFinalizedFeatures()), KAFKA_2_6_IV0) + def testControllerFeatureZNodeSetupWhenFeatureVersioningIsDisabledWithEnabledExistingFeatureZNode(): Unit = { + testControllerFeatureZNodeSetup(Some(new FeatureZNode(FeatureZNodeStatus.Enabled, Features.emptyFinalizedFeatures())), KAFKA_2_6_IV0) } @Test @@ -739,30 +750,53 @@ class ControllerIntegrationTest extends ZooKeeperTestHarness { controller.shutdown() } - private def testControllerFeatureZNodeSetup(initialZNode: FeatureZNode, + private def testControllerFeatureZNodeSetup(initialZNode: Option[FeatureZNode], interBrokerProtocolVersion: ApiVersion): Unit = { - zkClient.createFeatureZNode(initialZNode) - val (_, versionBefore) = zkClient.getDataAndVersion(FeatureZNode.path) + 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) val (mayBeFeatureZNodeBytes, versionAfter) = zkClient.getDataAndVersion(FeatureZNode.path) val newZNode = FeatureZNode.decode(mayBeFeatureZNodeBytes.get) if (interBrokerProtocolVersion >= KAFKA_2_7_IV0) { - if (initialZNode.status == FeatureZNodeStatus.Enabled) { - assertEquals(versionBefore, versionAfter) - assertEquals(initialZNode, newZNode) - } else if (initialZNode.status == FeatureZNodeStatus.Disabled) { - assertEquals(versionBefore + 1, versionAfter) - assertEquals(new FeatureZNode(FeatureZNodeStatus.Enabled, Features.emptyFinalizedFeatures()), newZNode) + 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 { - if (initialZNode.status == FeatureZNodeStatus.Enabled) { - assertEquals(versionBefore + 1, versionAfter) - assertEquals(new FeatureZNode(FeatureZNodeStatus.Disabled, Features.emptyFinalizedFeatures()), newZNode) - } else if (initialZNode.status == FeatureZNodeStatus.Disabled) { - assertEquals(versionBefore, versionAfter) - assertEquals(initialZNode, newZNode) + 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) } } } From b69f7fe3da3b6f7a46480a0902f15db39bd8f946 Mon Sep 17 00:00:00 2001 From: Kowshik Prakasam Date: Fri, 2 Oct 2020 19:26:53 -0700 Subject: [PATCH 40/41] Minor formatting change --- .../org/apache/kafka/clients/admin/SupportedVersionRange.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) 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 index 9440ee627c455..d71da31fb8200 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/SupportedVersionRange.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/SupportedVersionRange.java @@ -36,9 +36,7 @@ public class SupportedVersionRange { * @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) { + if (minVersion < 1 || maxVersion < 1 || maxVersion < minVersion) { throw new IllegalArgumentException( String.format( "Expected 1 <= minVersion <= maxVersion but received minVersion:%d, maxVersion:%d.", From e1c79cee2ab243d95647935d2b3e7abe371bf6ea Mon Sep 17 00:00:00 2001 From: Kowshik Prakasam Date: Mon, 5 Oct 2020 13:31:01 -0700 Subject: [PATCH 41/41] Address comments from Jun --- .../scala/kafka/controller/KafkaController.scala | 4 ++-- core/src/main/scala/kafka/zk/ZkData.scala | 3 ++- .../unit/kafka/cluster/BrokerEndPointTest.scala | 2 +- .../controller/ControllerIntegrationTest.scala | 15 ++++++++++++++- .../unit/kafka/server/UpdateFeaturesTest.scala | 9 +++++---- 5 files changed, 24 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index b13971ff73c31..37b5c4d77399c 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -383,7 +383,7 @@ class KafkaController(val config: KafkaConfig, case FeatureZNodeStatus.Disabled => if (!existingFeatureZNode.features.empty()) { warn(s"FeatureZNode at path: ${FeatureZNode.path} with disabled status" + - " contains non-empty features.") + s" contains non-empty features: ${existingFeatureZNode.features}") } Features.emptyFinalizedFeatures } @@ -420,7 +420,7 @@ class KafkaController(val config: KafkaConfig, if (existingFeatureZNode.status == FeatureZNodeStatus.Disabled && !existingFeatureZNode.features.empty()) { warn(s"FeatureZNode at path: ${FeatureZNode.path} with disabled status" + - " contains non-empty features.") + s" contains non-empty features: ${existingFeatureZNode.features}") } if (!newNode.equals(existingFeatureZNode)) { updateFeatureZNode(newNode) diff --git a/core/src/main/scala/kafka/zk/ZkData.scala b/core/src/main/scala/kafka/zk/ZkData.scala index 18f364a8bcaac..67fbef39321f6 100644 --- a/core/src/main/scala/kafka/zk/ZkData.scala +++ b/core/src/main/scala/kafka/zk/ZkData.scala @@ -799,6 +799,7 @@ object DelegationTokenInfoZNode { * is less than KAFKA_2_7_IV0. */ object FeatureZNodeStatus extends Enumeration { + type FeatureZNodeStatus = Value val Disabled, Enabled = Value def withNameOpt(value: Int): Option[Value] = { @@ -812,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/cluster/BrokerEndPointTest.scala b/core/src/test/scala/unit/kafka/cluster/BrokerEndPointTest.scala index d3b52ae5cf5f9..3708f73aedb64 100644 --- a/core/src/test/scala/unit/kafka/cluster/BrokerEndPointTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/BrokerEndPointTest.scala @@ -185,7 +185,7 @@ class BrokerEndPointTest { "endpoints":["CLIENT://host1:9092", "REPLICATION://host1:9093"], "listener_security_protocol_map":{"CLIENT":"SSL", "REPLICATION":"PLAINTEXT"}, "rack":"dc1", - "features": {"feature1": {"min_version": 1, "first_active_version": 1, "max_version": 2}, "feature2": {"min_version": 2, "first_active_version": 2, "max_version": 4}} + "features": {"feature1": {"min_version": 1, "max_version": 2}, "feature2": {"min_version": 2, "max_version": 4}} }""" val broker = parseBrokerJson(1, json) assertEquals(1, broker.id) diff --git a/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala b/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala index 9ceedf80cb0ed..47c271fca0b29 100644 --- a/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/controller/ControllerIntegrationTest.scala @@ -596,7 +596,6 @@ class ControllerIntegrationTest extends ZooKeeperTestHarness { testControllerMove(() => zkClient.createPartitionReassignment(reassignment)) } - @Test def testControllerFeatureZNodeSetupWhenFeatureVersioningIsEnabledWithNonExistingFeatureZNode(): Unit = { testControllerFeatureZNodeSetup(Option.empty, KAFKA_2_7_IV0) @@ -761,6 +760,20 @@ class ControllerIntegrationTest extends ZooKeeperTestHarness { } 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) diff --git a/core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala b/core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala index 3822c4238ff7a..a16ff30d543e5 100644 --- a/core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala +++ b/core/src/test/scala/unit/kafka/server/UpdateFeaturesTest.scala @@ -148,6 +148,7 @@ class UpdateFeaturesTest extends BaseRequestTest { * 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 @@ -222,7 +223,7 @@ class UpdateFeaturesTest extends BaseRequestTest { /** * Tests that an UpdateFeatures request fails in the Controller, when, for a feature the downgrade - * is attempted to a max version level thats higher than the existing max version level. + * is attempted to a max version level higher than the existing max version level. */ @Test def testShouldFailRequestWhenDowngradeToHigherVersionLevelIsAttempted(): Unit = { @@ -285,13 +286,13 @@ class UpdateFeaturesTest extends BaseRequestTest { def testShouldFailRequestDuringDeletionOfNonExistingFeature(): Unit = { testWithInvalidFeatureUpdate[InvalidRequestException]( "feature_non_existing", - new FeatureUpdate(0, true), - ".*Can not delete non-existing finalized feature.*".r) + 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 thats the same as the existing max version level. + * upgrade is attempted to a version level same as the existing max version level. */ @Test def testShouldFailRequestWhenUpgradingToSameVersionLevel(): Unit = {