diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java b/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java index 53ed92afd8888..96620df17fba3 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java @@ -37,6 +37,7 @@ import org.apache.kafka.common.acl.AclBindingFilter; import org.apache.kafka.common.annotation.InterfaceStability; import org.apache.kafka.common.config.ConfigResource; +import org.apache.kafka.common.errors.FeatureUpdateFailedException; import org.apache.kafka.common.quota.ClientQuotaAlteration; import org.apache.kafka.common.quota.ClientQuotaFilter; import org.apache.kafka.common.requests.LeaveGroupResponse; @@ -1306,6 +1307,73 @@ default AlterUserScramCredentialsResult alterUserScramCredentials(List alterations, AlterUserScramCredentialsOptions options); + /** + * Describes finalized as well as supported features. By default, the request is issued to any + * broker. It can be optionally directed only to the controller via DescribeFeaturesOptions + * parameter. This is particularly useful if the user requires strongly consistent reads of + * finalized features. + *

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

+ *

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

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

+ *

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

+ *

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