mirror of https://github.com/apache/kafka.git
KAFKA-16538; Enable upgrading kraft version for existing clusters (#19416)
This change implements upgrading the kraft version from 0 to 1 in existing clusters. Previously, clusters were formatted with either version 0 or version 1, and could not be moved between them. The kraft version for the cluster metadata partition is recorded using the KRaftVersion control record. If there is no KRaftVersion control record the default kraft version is 0. The kraft version is upgraded using the UpdateFeatures RPC. These RPCs are handled by the QuorumController and FeatureControlManager. This change adds special handling in the FeatureControlManager so that upgrades to the kraft.version are directed to RaftClient#upgradeKRaftVersion. To allow the FeatureControlManager to call RaftClient#upgradeKRaftVersion is a non-blocking fashion, the kraft version upgrade uses optimistic locking. The call to RaftClient#upgradeKRaftVersion does validations of the version change. If the validations succeeds, it generates the necessary control records and adds them to the BatchAccumulator. Before the kraft version can be upgraded to version 1, all of the brokers and controllers in the cluster need to support kraft version 1. The check that all brokers support kraft version 1 is done by the FeatureControlManager. The check that all of the controllers support kraft version is done by KafkaRaftClient and LeaderState. When the kraft version is 0, the kraft leader starts by assuming that all voters do not support kraft version 1. The leader discovers which voters support kraft version 1 through the UpdateRaftVoter RPC. The KRaft leader handles UpdateRaftVoter RPCs by storing the updated information in-memory until the kraft version is upgraded to version 1. This state is stored in LeaderState and contains the latest directory id, endpoints and supported kraft version for each voter. Only when the KRaft leader has received an UpdateRaftVoter RPC from all of the voters will it allow the upgrade from kraft.version 0 to 1. Reviewers: Alyssa Huang <ahuang@confluent.io>, Colin P. McCabe <cmccabe@apache.org>
This commit is contained in:
parent
5af3547c6e
commit
b97a130c08
|
@ -665,15 +665,17 @@ public class ConfigurationControlManager {
|
|||
* @param updates The user-requested updates.
|
||||
* @param upgradeTypes The user-requested upgrade types.
|
||||
* @param validateOnly True if we should validate the request but not make changes.
|
||||
* @param currentClaimEpoch the currently claimed epoch
|
||||
*
|
||||
* @return The result.
|
||||
*/
|
||||
ControllerResult<ApiError> updateFeatures(
|
||||
Map<String, Short> updates,
|
||||
Map<String, FeatureUpdate.UpgradeType> upgradeTypes,
|
||||
boolean validateOnly
|
||||
boolean validateOnly,
|
||||
int currentClaimEpoch
|
||||
) {
|
||||
ControllerResult<ApiError> result = featureControl.updateFeatures(updates, upgradeTypes, validateOnly);
|
||||
ControllerResult<ApiError> result = featureControl.updateFeatures(updates, upgradeTypes, validateOnly, currentClaimEpoch);
|
||||
if (result.response().isSuccess() &&
|
||||
!validateOnly &&
|
||||
updates.getOrDefault(EligibleLeaderReplicasVersion.FEATURE_NAME, (short) 0) > 0
|
||||
|
|
|
@ -18,7 +18,9 @@
|
|||
package org.apache.kafka.controller;
|
||||
|
||||
import org.apache.kafka.clients.admin.FeatureUpdate;
|
||||
import org.apache.kafka.common.errors.ApiException;
|
||||
import org.apache.kafka.common.metadata.FeatureLevelRecord;
|
||||
import org.apache.kafka.common.metadata.NoOpRecord;
|
||||
import org.apache.kafka.common.protocol.Errors;
|
||||
import org.apache.kafka.common.requests.ApiError;
|
||||
import org.apache.kafka.common.utils.LogContext;
|
||||
|
@ -55,6 +57,8 @@ public class FeatureControlManager {
|
|||
private LogContext logContext = null;
|
||||
private SnapshotRegistry snapshotRegistry = null;
|
||||
private QuorumFeatures quorumFeatures = null;
|
||||
private KRaftVersionAccessor kraftVersionAccessor = null;
|
||||
|
||||
private ClusterFeatureSupportDescriber clusterSupportDescriber = new ClusterFeatureSupportDescriber() {
|
||||
@Override
|
||||
public Iterator<Entry<Integer, Map<String, VersionRange>>> brokerSupported() {
|
||||
|
@ -87,6 +91,11 @@ public class FeatureControlManager {
|
|||
return this;
|
||||
}
|
||||
|
||||
Builder setKRaftVersionAccessor(KRaftVersionAccessor kraftVersionAccessor) {
|
||||
this.kraftVersionAccessor = kraftVersionAccessor;
|
||||
return this;
|
||||
}
|
||||
|
||||
public FeatureControlManager build() {
|
||||
if (logContext == null) logContext = new LogContext();
|
||||
if (snapshotRegistry == null) snapshotRegistry = new SnapshotRegistry(logContext);
|
||||
|
@ -97,11 +106,30 @@ public class FeatureControlManager {
|
|||
MetadataVersion.latestProduction().featureLevel()));
|
||||
quorumFeatures = new QuorumFeatures(0, localSupportedFeatures, List.of(0));
|
||||
}
|
||||
if (kraftVersionAccessor == null) {
|
||||
kraftVersionAccessor = new KRaftVersionAccessor() {
|
||||
private KRaftVersion version = KRaftVersion.LATEST_PRODUCTION;
|
||||
|
||||
@Override
|
||||
public KRaftVersion kraftVersion() {
|
||||
return version;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void upgradeKRaftVersion(int epoch, KRaftVersion version, boolean validateOnly) {
|
||||
if (!validateOnly) {
|
||||
this.version = version;
|
||||
}
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
return new FeatureControlManager(
|
||||
logContext,
|
||||
quorumFeatures,
|
||||
snapshotRegistry,
|
||||
clusterSupportDescriber
|
||||
clusterSupportDescriber,
|
||||
kraftVersionAccessor
|
||||
);
|
||||
}
|
||||
}
|
||||
|
@ -128,23 +156,31 @@ public class FeatureControlManager {
|
|||
*/
|
||||
private final ClusterFeatureSupportDescriber clusterSupportDescriber;
|
||||
|
||||
/**
|
||||
* The interface for reading and upgrading the kraft version.
|
||||
*/
|
||||
private final KRaftVersionAccessor kraftVersionAccessor;
|
||||
|
||||
private FeatureControlManager(
|
||||
LogContext logContext,
|
||||
QuorumFeatures quorumFeatures,
|
||||
SnapshotRegistry snapshotRegistry,
|
||||
ClusterFeatureSupportDescriber clusterSupportDescriber
|
||||
ClusterFeatureSupportDescriber clusterSupportDescriber,
|
||||
KRaftVersionAccessor kraftVersionAccessor
|
||||
) {
|
||||
this.log = logContext.logger(FeatureControlManager.class);
|
||||
this.quorumFeatures = quorumFeatures;
|
||||
this.finalizedVersions = new TimelineHashMap<>(snapshotRegistry, 0);
|
||||
this.metadataVersion = new TimelineObject<>(snapshotRegistry, Optional.empty());
|
||||
this.clusterSupportDescriber = clusterSupportDescriber;
|
||||
this.kraftVersionAccessor = kraftVersionAccessor;
|
||||
}
|
||||
|
||||
ControllerResult<ApiError> updateFeatures(
|
||||
Map<String, Short> updates,
|
||||
Map<String, FeatureUpdate.UpgradeType> upgradeTypes,
|
||||
boolean validateOnly
|
||||
boolean validateOnly,
|
||||
int currentClaimedEpoch
|
||||
) {
|
||||
List<ApiMessageAndVersion> records =
|
||||
BoundedList.newArrayBacked(MAX_RECORDS_PER_USER_OP);
|
||||
|
@ -154,8 +190,15 @@ public class FeatureControlManager {
|
|||
proposedUpdatedVersions.putAll(updates);
|
||||
|
||||
for (Entry<String, Short> entry : updates.entrySet()) {
|
||||
ApiError error = updateFeature(entry.getKey(), entry.getValue(),
|
||||
upgradeTypes.getOrDefault(entry.getKey(), FeatureUpdate.UpgradeType.UPGRADE), records, proposedUpdatedVersions);
|
||||
ApiError error = updateFeature(
|
||||
entry.getKey(),
|
||||
entry.getValue(),
|
||||
upgradeTypes.getOrDefault(entry.getKey(), FeatureUpdate.UpgradeType.UPGRADE),
|
||||
records,
|
||||
proposedUpdatedVersions,
|
||||
validateOnly,
|
||||
currentClaimedEpoch
|
||||
);
|
||||
if (!error.error().equals(Errors.NONE)) {
|
||||
return ControllerResult.of(List.of(), error);
|
||||
}
|
||||
|
@ -181,12 +224,15 @@ public class FeatureControlManager {
|
|||
new IllegalStateException("Unknown metadata version for FeatureControlManager"));
|
||||
}
|
||||
|
||||
@SuppressWarnings({ "CyclomaticComplexity" })
|
||||
private ApiError updateFeature(
|
||||
String featureName,
|
||||
short newVersion,
|
||||
FeatureUpdate.UpgradeType upgradeType,
|
||||
List<ApiMessageAndVersion> records,
|
||||
Map<String, Short> proposedUpdatedVersions
|
||||
Map<String, Short> proposedUpdatedVersions,
|
||||
boolean validateOnly,
|
||||
int currentClaimedEpoch
|
||||
) {
|
||||
if (upgradeType.equals(FeatureUpdate.UpgradeType.UNKNOWN)) {
|
||||
return invalidUpdateVersion(featureName, newVersion,
|
||||
|
@ -196,6 +242,8 @@ public class FeatureControlManager {
|
|||
final short currentVersion;
|
||||
if (featureName.equals(MetadataVersion.FEATURE_NAME)) {
|
||||
currentVersion = metadataVersionOrThrow().featureLevel();
|
||||
} else if (featureName.equals(KRaftVersion.FEATURE_NAME)) {
|
||||
currentVersion = kraftVersionAccessor.kraftVersion().featureLevel();
|
||||
} else {
|
||||
currentVersion = finalizedVersions.getOrDefault(featureName, (short) 0);
|
||||
}
|
||||
|
@ -225,6 +273,34 @@ public class FeatureControlManager {
|
|||
if (featureName.equals(MetadataVersion.FEATURE_NAME)) {
|
||||
// Perform additional checks if we're updating metadata.version
|
||||
return updateMetadataVersion(newVersion, upgradeType.equals(FeatureUpdate.UpgradeType.UNSAFE_DOWNGRADE), records::add);
|
||||
} else if (featureName.equals(KRaftVersion.FEATURE_NAME)) {
|
||||
if (upgradeType.equals(FeatureUpdate.UpgradeType.UPGRADE)) {
|
||||
try {
|
||||
kraftVersionAccessor.upgradeKRaftVersion(
|
||||
currentClaimedEpoch,
|
||||
KRaftVersion.fromFeatureLevel(newVersion),
|
||||
validateOnly
|
||||
);
|
||||
/* Add the noop record so that there is at least one offset to wait on to
|
||||
* complete the upgrade RPC
|
||||
*/
|
||||
records.add(new ApiMessageAndVersion(new NoOpRecord(), (short) 0));
|
||||
return ApiError.NONE;
|
||||
} catch (ApiException e) {
|
||||
return ApiError.fromThrowable(e);
|
||||
} catch (IllegalArgumentException e) {
|
||||
return invalidUpdateVersion(featureName, newVersion, e.getMessage());
|
||||
}
|
||||
} else if (newVersion != currentVersion) {
|
||||
return invalidUpdateVersion(
|
||||
featureName,
|
||||
newVersion,
|
||||
"Can't downgrade the version of this feature."
|
||||
);
|
||||
} else {
|
||||
// Version didn't change
|
||||
return ApiError.NONE;
|
||||
}
|
||||
} else {
|
||||
// Validate dependencies for features that are not metadata.version
|
||||
try {
|
||||
|
|
|
@ -0,0 +1,42 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.kafka.controller;
|
||||
|
||||
import org.apache.kafka.server.common.KRaftVersion;
|
||||
|
||||
/**
|
||||
* Type for upgrading and reading the kraft version.
|
||||
*/
|
||||
interface KRaftVersionAccessor {
|
||||
/**
|
||||
* Returns the latest kraft version.
|
||||
*
|
||||
* The latest version may be uncommitted.
|
||||
*/
|
||||
KRaftVersion kraftVersion();
|
||||
|
||||
/**
|
||||
* Upgrade the kraft version.
|
||||
*
|
||||
* @param epoch the current epoch
|
||||
* @param newVersion the new kraft version to upgrade to
|
||||
* @param validateOnly whether to just validate the change and not persist it
|
||||
* @throws ApiException when the upgrade fails to validate
|
||||
*/
|
||||
void upgradeKRaftVersion(int epoch, KRaftVersion newVersion, boolean validateOnly);
|
||||
}
|
|
@ -1519,6 +1519,7 @@ public final class QuorumController implements Controller {
|
|||
setQuorumFeatures(quorumFeatures).
|
||||
setSnapshotRegistry(snapshotRegistry).
|
||||
setClusterFeatureSupportDescriber(clusterSupportDescriber).
|
||||
setKRaftVersionAccessor(new RaftClientKRaftVersionAccessor(raftClient)).
|
||||
build();
|
||||
this.clusterControl = new ClusterControlManager.Builder().
|
||||
setLogContext(logContext).
|
||||
|
@ -2041,7 +2042,7 @@ public final class QuorumController implements Controller {
|
|||
upgradeTypes.put(featureName, FeatureUpdate.UpgradeType.fromCode(featureUpdate.upgradeType()));
|
||||
updates.put(featureName, featureUpdate.maxVersionLevel());
|
||||
});
|
||||
return configurationControl.updateFeatures(updates, upgradeTypes, request.validateOnly());
|
||||
return configurationControl.updateFeatures(updates, upgradeTypes, request.validateOnly(), curClaimEpoch);
|
||||
}).thenApply(result -> {
|
||||
UpdateFeaturesResponseData responseData = new UpdateFeaturesResponseData();
|
||||
|
||||
|
|
|
@ -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.controller;
|
||||
|
||||
import org.apache.kafka.raft.RaftClient;
|
||||
import org.apache.kafka.server.common.KRaftVersion;
|
||||
|
||||
public final class RaftClientKRaftVersionAccessor implements KRaftVersionAccessor {
|
||||
private final RaftClient<?> raftClient;
|
||||
|
||||
public RaftClientKRaftVersionAccessor(RaftClient<?> raftClient) {
|
||||
this.raftClient = raftClient;
|
||||
}
|
||||
|
||||
public KRaftVersion kraftVersion() {
|
||||
return raftClient.kraftVersion();
|
||||
}
|
||||
|
||||
public void upgradeKRaftVersion(int epoch, KRaftVersion version, boolean validateOnly) {
|
||||
raftClient.upgradeKRaftVersion(epoch, version, validateOnly);
|
||||
}
|
||||
}
|
|
@ -489,7 +489,8 @@ public class ConfigurationControlManagerTest {
|
|||
EligibleLeaderReplicasVersion.ELRV_1.featureLevel()),
|
||||
Map.of(EligibleLeaderReplicasVersion.FEATURE_NAME,
|
||||
FeatureUpdate.UpgradeType.UPGRADE),
|
||||
false);
|
||||
false,
|
||||
0);
|
||||
assertNotNull(result.response());
|
||||
assertEquals(Errors.NONE, result.response().error());
|
||||
RecordTestUtils.replayAll(manager, result.records());
|
||||
|
@ -540,7 +541,8 @@ public class ConfigurationControlManagerTest {
|
|||
EligibleLeaderReplicasVersion.ELRV_1.featureLevel()),
|
||||
Map.of(EligibleLeaderReplicasVersion.FEATURE_NAME,
|
||||
FeatureUpdate.UpgradeType.UPGRADE),
|
||||
false);
|
||||
false,
|
||||
0);
|
||||
assertNotNull(result.response());
|
||||
if (isMetadataVersionElrEnabled) {
|
||||
assertEquals(Errors.NONE, result.response().error());
|
||||
|
|
|
@ -107,10 +107,10 @@ public class FeatureControlManagerTest {
|
|||
"Invalid update version 3 for feature " + TestFeatureVersion.FEATURE_NAME + ". Local controller 0 only supports versions 0-2")),
|
||||
manager.updateFeatures(updateMap(TestFeatureVersion.FEATURE_NAME, 3),
|
||||
Map.of(TestFeatureVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.SAFE_DOWNGRADE),
|
||||
false));
|
||||
false, 0));
|
||||
ControllerResult<ApiError> result = manager.updateFeatures(
|
||||
updateMap(TestFeatureVersion.FEATURE_NAME, 1, "bar", 1), Map.of(),
|
||||
false);
|
||||
false, 0);
|
||||
ApiError expectedError = new ApiError(Errors.INVALID_UPDATE_VERSION,
|
||||
"Invalid update version 1 for feature bar. Local controller 0 does not support this feature.");
|
||||
List<ApiMessageAndVersion> expectedMessages = new ArrayList<>();
|
||||
|
@ -119,7 +119,7 @@ public class FeatureControlManagerTest {
|
|||
|
||||
result = manager.updateFeatures(
|
||||
updateMap(TestFeatureVersion.FEATURE_NAME, 1), Map.of(),
|
||||
false);
|
||||
false, 0);
|
||||
expectedError = ApiError.NONE;
|
||||
assertEquals(expectedError, result.response());
|
||||
expectedMessages = new ArrayList<>();
|
||||
|
@ -208,10 +208,10 @@ public class FeatureControlManagerTest {
|
|||
"Invalid update version 3 for feature foo. Broker 5 does not support this feature.")),
|
||||
manager.updateFeatures(updateMap("foo", 3),
|
||||
Map.of("foo", FeatureUpdate.UpgradeType.SAFE_DOWNGRADE),
|
||||
false));
|
||||
false, 0));
|
||||
|
||||
ControllerResult<ApiError> result = manager.updateFeatures(
|
||||
updateMap(TransactionVersion.FEATURE_NAME, 2), Map.of(), false);
|
||||
updateMap(TransactionVersion.FEATURE_NAME, 2), Map.of(), false, 0);
|
||||
assertEquals(ApiError.NONE, result.response());
|
||||
manager.replay((FeatureLevelRecord) result.records().get(0).message());
|
||||
snapshotRegistry.idempotentCreateSnapshot(3);
|
||||
|
@ -219,7 +219,7 @@ public class FeatureControlManagerTest {
|
|||
assertEquals(ControllerResult.of(List.of(), new ApiError(Errors.INVALID_UPDATE_VERSION,
|
||||
"Invalid update version 1 for feature " + TransactionVersion.FEATURE_NAME + "." +
|
||||
" Can't downgrade the version of this feature without setting the upgrade type to either safe or unsafe downgrade.")),
|
||||
manager.updateFeatures(updateMap(TransactionVersion.FEATURE_NAME, 1), Map.of(), false));
|
||||
manager.updateFeatures(updateMap(TransactionVersion.FEATURE_NAME, 1), Map.of(), false, 0));
|
||||
|
||||
assertEquals(
|
||||
ControllerResult.atomicOf(
|
||||
|
@ -236,7 +236,8 @@ public class FeatureControlManagerTest {
|
|||
manager.updateFeatures(
|
||||
updateMap(TransactionVersion.FEATURE_NAME, 1),
|
||||
Map.of(TransactionVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.SAFE_DOWNGRADE),
|
||||
false)
|
||||
false,
|
||||
0)
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -251,7 +252,7 @@ public class FeatureControlManagerTest {
|
|||
build();
|
||||
manager.replay(new FeatureLevelRecord().setName(MetadataVersion.FEATURE_NAME).setFeatureLevel(MetadataVersion.MINIMUM_VERSION.featureLevel()));
|
||||
ControllerResult<ApiError> result = manager.
|
||||
updateFeatures(updateMap(TestFeatureVersion.FEATURE_NAME, 1, TransactionVersion.FEATURE_NAME, 2), Map.of(), false);
|
||||
updateFeatures(updateMap(TestFeatureVersion.FEATURE_NAME, 1, TransactionVersion.FEATURE_NAME, 2), Map.of(), false, 0);
|
||||
RecordTestUtils.replayAll(manager, result.records());
|
||||
assertEquals(MetadataVersion.MINIMUM_VERSION, manager.metadataVersionOrThrow());
|
||||
assertEquals(Optional.of((short) 1), manager.finalizedFeatures(Long.MAX_VALUE).get(TestFeatureVersion.FEATURE_NAME));
|
||||
|
@ -289,7 +290,8 @@ public class FeatureControlManagerTest {
|
|||
manager.updateFeatures(
|
||||
Map.of(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_5_IV0.featureLevel()),
|
||||
Map.of(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.SAFE_DOWNGRADE),
|
||||
true));
|
||||
true,
|
||||
0));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -301,7 +303,8 @@ public class FeatureControlManagerTest {
|
|||
manager.updateFeatures(
|
||||
Map.of(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_5_IV0.featureLevel()),
|
||||
Map.of(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.UNSAFE_DOWNGRADE),
|
||||
true));
|
||||
true,
|
||||
0));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -318,7 +321,8 @@ public class FeatureControlManagerTest {
|
|||
manager.updateFeatures(
|
||||
Map.of(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_5_IV0.featureLevel()),
|
||||
Map.of(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.UPGRADE),
|
||||
true));
|
||||
true,
|
||||
0));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -328,7 +332,8 @@ public class FeatureControlManagerTest {
|
|||
manager.updateFeatures(
|
||||
Map.of(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_5_IV0.featureLevel()),
|
||||
Map.of(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.UPGRADE),
|
||||
true));
|
||||
true,
|
||||
0));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -340,7 +345,8 @@ public class FeatureControlManagerTest {
|
|||
manager.updateFeatures(
|
||||
Map.of(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_3_IV3.featureLevel()),
|
||||
Map.of(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.SAFE_DOWNGRADE),
|
||||
true));
|
||||
true,
|
||||
0));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -349,9 +355,10 @@ public class FeatureControlManagerTest {
|
|||
assertEquals(ControllerResult.of(List.of(), new ApiError(Errors.INVALID_UPDATE_VERSION,
|
||||
"Invalid metadata.version 7. Unsafe metadata downgrade is not supported in this version.")),
|
||||
manager.updateFeatures(
|
||||
Map.of(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_3_IV3.featureLevel()),
|
||||
Map.of(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.UNSAFE_DOWNGRADE),
|
||||
true));
|
||||
Map.of(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_3_IV3.featureLevel()),
|
||||
Map.of(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.UNSAFE_DOWNGRADE),
|
||||
true,
|
||||
0));
|
||||
}
|
||||
|
||||
@Disabled
|
||||
|
@ -362,7 +369,8 @@ public class FeatureControlManagerTest {
|
|||
manager.updateFeatures(
|
||||
Map.of(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_3_IV3.featureLevel()),
|
||||
Map.of(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.UNSAFE_DOWNGRADE),
|
||||
true));
|
||||
true,
|
||||
0));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -376,7 +384,8 @@ public class FeatureControlManagerTest {
|
|||
manager.updateFeatures(
|
||||
Map.of(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_4_IV0.featureLevel()),
|
||||
Map.of(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.SAFE_DOWNGRADE),
|
||||
true));
|
||||
true,
|
||||
0));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -391,7 +400,8 @@ public class FeatureControlManagerTest {
|
|||
manager.updateFeatures(
|
||||
Map.of(MetadataVersion.FEATURE_NAME, MetadataVersionTestUtils.IBP_3_3_IV2_FEATURE_LEVEL),
|
||||
Map.of(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.UNSAFE_DOWNGRADE),
|
||||
true));
|
||||
true,
|
||||
0));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -410,7 +420,8 @@ public class FeatureControlManagerTest {
|
|||
ControllerResult<ApiError> result = manager.updateFeatures(
|
||||
Map.of(Feature.TEST_VERSION.featureName(), (short) 1),
|
||||
Map.of(Feature.TEST_VERSION.featureName(), FeatureUpdate.UpgradeType.UPGRADE),
|
||||
false);
|
||||
false,
|
||||
0);
|
||||
assertEquals(ControllerResult.atomicOf(List.of(new ApiMessageAndVersion(
|
||||
new FeatureLevelRecord().setName(Feature.TEST_VERSION.featureName()).setFeatureLevel((short) 1), (short) 0)),
|
||||
ApiError.NONE), result);
|
||||
|
@ -420,7 +431,8 @@ public class FeatureControlManagerTest {
|
|||
ControllerResult<ApiError> result2 = manager.updateFeatures(
|
||||
Map.of(Feature.TEST_VERSION.featureName(), (short) 0),
|
||||
Map.of(Feature.TEST_VERSION.featureName(), FeatureUpdate.UpgradeType.UNSAFE_DOWNGRADE),
|
||||
false);
|
||||
false,
|
||||
0);
|
||||
assertEquals(ControllerResult.atomicOf(List.of(new ApiMessageAndVersion(
|
||||
new FeatureLevelRecord().setName(Feature.TEST_VERSION.featureName()).setFeatureLevel((short) 0), (short) 0)),
|
||||
ApiError.NONE), result2);
|
||||
|
@ -444,7 +456,8 @@ public class FeatureControlManagerTest {
|
|||
ControllerResult<ApiError> result = manager.updateFeatures(
|
||||
Map.of(Feature.ELIGIBLE_LEADER_REPLICAS_VERSION.featureName(), (short) 1),
|
||||
Map.of(Feature.ELIGIBLE_LEADER_REPLICAS_VERSION.featureName(), FeatureUpdate.UpgradeType.UPGRADE),
|
||||
false);
|
||||
false,
|
||||
0);
|
||||
assertTrue(result.response().isSuccess());
|
||||
assertEquals(List.of(new ApiMessageAndVersion(
|
||||
new FeatureLevelRecord().
|
||||
|
|
|
@ -126,6 +126,11 @@ public class SnapshotEmitterTest {
|
|||
return KRaftVersion.KRAFT_VERSION_0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void upgradeKRaftVersion(int epoch, KRaftVersion version, boolean validateOnly) {
|
||||
// nothing to do
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws Exception {
|
||||
// nothing to do
|
||||
|
|
|
@ -415,7 +415,7 @@ public final class LocalLogManager implements RaftClient<ApiMessageAndVersion>,
|
|||
/**
|
||||
* The latest kraft version used by this local log manager.
|
||||
*/
|
||||
private final KRaftVersion lastKRaftVersion;
|
||||
private KRaftVersion lastKRaftVersion;
|
||||
|
||||
/**
|
||||
* Whether this LocalLogManager has been shut down.
|
||||
|
@ -789,4 +789,11 @@ public final class LocalLogManager implements RaftClient<ApiMessageAndVersion>,
|
|||
public KRaftVersion kraftVersion() {
|
||||
return lastKRaftVersion;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void upgradeKRaftVersion(int epoch, KRaftVersion version, boolean validateOnly) {
|
||||
if (!validateOnly) {
|
||||
lastKRaftVersion = version;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -30,6 +30,8 @@ import java.util.OptionalLong;
|
|||
import java.util.Set;
|
||||
|
||||
public class FollowerState implements EpochState {
|
||||
private final Logger log;
|
||||
|
||||
private final int fetchTimeoutMs;
|
||||
private final int epoch;
|
||||
private final int leaderId;
|
||||
|
@ -38,19 +40,23 @@ public class FollowerState implements EpochState {
|
|||
private final Set<Integer> voters;
|
||||
// Used for tracking the expiration of both the Fetch and FetchSnapshot requests
|
||||
private final Timer fetchTimer;
|
||||
/* Used to track if the replica has fetched successfully from the leader at least once since the transition to
|
||||
* follower in this epoch. If the replica has not yet fetched successfully, it may be able to grant PreVotes.
|
||||
*/
|
||||
private boolean hasFetchedFromLeader;
|
||||
private Optional<LogOffsetMetadata> highWatermark;
|
||||
/* Used to track the currently fetching snapshot. When fetching snapshot regular
|
||||
* Fetch request are paused
|
||||
*/
|
||||
private Optional<RawSnapshotWriter> fetchingSnapshot = Optional.empty();
|
||||
// Used to throttle update voter request and allow for Fetch/FetchSnapshot requests
|
||||
// Used to track when to send another update voter request
|
||||
private final Timer updateVoterPeriodTimer;
|
||||
|
||||
private final Logger log;
|
||||
/* Used to track if the replica has fetched successfully from the leader at least once since
|
||||
* the transition to follower in this epoch. If the replica has not yet fetched successfully,
|
||||
* it may be able to grant PreVotes.
|
||||
*/
|
||||
private boolean hasFetchedFromLeader = false;
|
||||
private Optional<LogOffsetMetadata> highWatermark;
|
||||
/* For kraft.version 0, track if the leader has received updated voter information from this
|
||||
* follower.
|
||||
*/
|
||||
private boolean hasUpdatedLeader = false;
|
||||
/* Used to track the currently fetching snapshot. When fetching snapshot regular Fetch request
|
||||
* are paused
|
||||
*/
|
||||
private Optional<RawSnapshotWriter> fetchingSnapshot = Optional.empty();
|
||||
|
||||
public FollowerState(
|
||||
Time time,
|
||||
|
@ -73,7 +79,6 @@ public class FollowerState implements EpochState {
|
|||
this.updateVoterPeriodTimer = time.timer(updateVoterPeriodMs());
|
||||
this.highWatermark = highWatermark;
|
||||
this.log = logContext.logger(FollowerState.class);
|
||||
this.hasFetchedFromLeader = false;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -140,7 +145,7 @@ public class FollowerState implements EpochState {
|
|||
private long updateVoterPeriodMs() {
|
||||
// Allow for a few rounds of fetch request before attempting to update
|
||||
// the voter state
|
||||
return fetchTimeoutMs * 3L;
|
||||
return fetchTimeoutMs;
|
||||
}
|
||||
|
||||
public boolean hasUpdateVoterPeriodExpired(long currentTimeMs) {
|
||||
|
@ -158,6 +163,14 @@ public class FollowerState implements EpochState {
|
|||
updateVoterPeriodTimer.reset(updateVoterPeriodMs());
|
||||
}
|
||||
|
||||
public boolean hasUpdatedLeader() {
|
||||
return hasUpdatedLeader;
|
||||
}
|
||||
|
||||
public void setHasUpdatedLeader() {
|
||||
this.hasUpdatedLeader = true;
|
||||
}
|
||||
|
||||
public boolean updateHighWatermark(OptionalLong newHighWatermark) {
|
||||
if (newHighWatermark.isEmpty() && highWatermark.isPresent()) {
|
||||
throw new IllegalArgumentException(
|
||||
|
|
|
@ -77,10 +77,12 @@ import org.apache.kafka.raft.internals.CloseListener;
|
|||
import org.apache.kafka.raft.internals.DefaultRequestSender;
|
||||
import org.apache.kafka.raft.internals.FuturePurgatory;
|
||||
import org.apache.kafka.raft.internals.KRaftControlRecordStateMachine;
|
||||
import org.apache.kafka.raft.internals.KRaftVersionUpgrade;
|
||||
import org.apache.kafka.raft.internals.KafkaRaftMetrics;
|
||||
import org.apache.kafka.raft.internals.MemoryBatchReader;
|
||||
import org.apache.kafka.raft.internals.RecordsBatchReader;
|
||||
import org.apache.kafka.raft.internals.RemoveVoterHandler;
|
||||
import org.apache.kafka.raft.internals.RequestSendResult;
|
||||
import org.apache.kafka.raft.internals.ThresholdPurgatory;
|
||||
import org.apache.kafka.raft.internals.UpdateVoterHandler;
|
||||
import org.apache.kafka.server.common.KRaftVersion;
|
||||
|
@ -163,7 +165,7 @@ import static org.apache.kafka.snapshot.Snapshots.BOOTSTRAP_SNAPSHOT_ID;
|
|||
* as FileRecords, but we use {@link UnalignedRecords} in FetchSnapshotResponse because the records
|
||||
* are not necessarily offset-aligned.
|
||||
*/
|
||||
@SuppressWarnings({ "ClassDataAbstractionCoupling", "ClassFanOutComplexity", "ParameterNumber", "NPathComplexity" })
|
||||
@SuppressWarnings({ "ClassDataAbstractionCoupling", "ClassFanOutComplexity", "ParameterNumber", "NPathComplexity", "JavaNCSS" })
|
||||
public final class KafkaRaftClient<T> implements RaftClient<T> {
|
||||
private static final int RETRY_BACKOFF_BASE_MS = 100;
|
||||
private static final int MAX_NUMBER_OF_BATCHES = 10;
|
||||
|
@ -592,7 +594,8 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
this.updateVoterHandler = new UpdateVoterHandler(
|
||||
nodeId,
|
||||
partitionState,
|
||||
channel.listenerName()
|
||||
channel.listenerName(),
|
||||
logContext
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -653,7 +656,7 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
// The high watermark can only be advanced once we have written a record
|
||||
// from the new leader's epoch. Hence, we write a control message immediately
|
||||
// to ensure there is no delay committing pending data.
|
||||
state.appendStartOfEpochControlRecords(quorum.localVoterNodeOrThrow(), currentTimeMs);
|
||||
state.appendStartOfEpochControlRecords(currentTimeMs);
|
||||
|
||||
resetConnections();
|
||||
kafkaRaftMetrics.maybeUpdateElectionLatency(currentTimeMs);
|
||||
|
@ -2101,8 +2104,7 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
}
|
||||
|
||||
FollowerState state = quorum.followerStateOrThrow();
|
||||
|
||||
if (Errors.forCode(partitionSnapshot.errorCode()) == Errors.SNAPSHOT_NOT_FOUND ||
|
||||
if (error == Errors.SNAPSHOT_NOT_FOUND ||
|
||||
partitionSnapshot.snapshotId().endOffset() < 0 ||
|
||||
partitionSnapshot.snapshotId().epoch() < 0) {
|
||||
|
||||
|
@ -2118,6 +2120,8 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
state.setFetchingSnapshot(Optional.empty());
|
||||
state.resetFetchTimeoutForSuccessfulFetch(currentTimeMs);
|
||||
return true;
|
||||
} else if (error != Errors.NONE) {
|
||||
return handleUnexpectedError(error, responseMetadata);
|
||||
}
|
||||
|
||||
OffsetAndEpoch snapshotId = new OffsetAndEpoch(
|
||||
|
@ -2125,15 +2129,11 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
partitionSnapshot.snapshotId().epoch()
|
||||
);
|
||||
|
||||
RawSnapshotWriter snapshot;
|
||||
if (state.fetchingSnapshot().isPresent()) {
|
||||
snapshot = state.fetchingSnapshot().get();
|
||||
} else {
|
||||
throw new IllegalStateException(
|
||||
RawSnapshotWriter snapshot = state.fetchingSnapshot().orElseThrow(
|
||||
() -> new IllegalStateException(
|
||||
String.format("Received unexpected fetch snapshot response: %s", partitionSnapshot)
|
||||
);
|
||||
}
|
||||
|
||||
)
|
||||
);
|
||||
if (!snapshot.snapshotId().equals(snapshotId)) {
|
||||
throw new IllegalStateException(
|
||||
String.format(
|
||||
|
@ -2142,8 +2142,7 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
snapshotId
|
||||
)
|
||||
);
|
||||
}
|
||||
if (snapshot.sizeInBytes() != partitionSnapshot.position()) {
|
||||
} else if (snapshot.sizeInBytes() != partitionSnapshot.position()) {
|
||||
throw new IllegalStateException(
|
||||
String.format(
|
||||
"Received fetch snapshot response with an invalid position. Expected %d; Received %d",
|
||||
|
@ -2413,7 +2412,7 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
int responseEpoch = data.currentLeader().leaderEpoch();
|
||||
|
||||
final Endpoints leaderEndpoints;
|
||||
if (responseLeaderId.isPresent() && data.currentLeader().host().isEmpty()) {
|
||||
if (responseLeaderId.isPresent() && !data.currentLeader().host().isEmpty()) {
|
||||
leaderEndpoints = Endpoints.fromInetSocketAddresses(
|
||||
Map.of(
|
||||
channel.listenerName(),
|
||||
|
@ -2435,8 +2434,18 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
responseMetadata.source(),
|
||||
currentTimeMs
|
||||
);
|
||||
if (handled.isPresent()) {
|
||||
return handled.get();
|
||||
} else if (error == Errors.NONE || error == Errors.UNSUPPORTED_VERSION) {
|
||||
FollowerState follower = quorum.followerStateOrThrow();
|
||||
follower.setHasUpdatedLeader();
|
||||
// Treat update voter similar to fetch and fetch snapshot, and reset the timer
|
||||
follower.resetFetchTimeoutForSuccessfulFetch(currentTimeMs);
|
||||
|
||||
return handled.orElse(true);
|
||||
return true;
|
||||
} else {
|
||||
return handleUnexpectedError(error, responseMetadata);
|
||||
}
|
||||
}
|
||||
|
||||
private boolean hasConsistentLeader(int epoch, OptionalInt leaderId) {
|
||||
|
@ -2746,17 +2755,27 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
}
|
||||
|
||||
/**
|
||||
* Attempt to send a request. Return the time to wait before the request can be retried.
|
||||
* Attempt to send a request.
|
||||
*
|
||||
* Return if the request was sent and the time to wait before the request can be retried.
|
||||
*
|
||||
* @param currentTimeMs the current time
|
||||
* @param destination the node receiving the request
|
||||
* @param requestSupplier the function that creates the request
|
||||
* @return the first element in the pair indicates if the request was sent; the second element
|
||||
* in the pair indicates the time to wait before retrying.
|
||||
*/
|
||||
private long maybeSendRequest(
|
||||
private RequestSendResult maybeSendRequest(
|
||||
long currentTimeMs,
|
||||
Node destination,
|
||||
Supplier<ApiMessage> requestSupplier
|
||||
) {
|
||||
var requestSent = false;
|
||||
|
||||
if (requestManager.isBackingOff(destination, currentTimeMs)) {
|
||||
long remainingBackoffMs = requestManager.remainingBackoffMs(destination, currentTimeMs);
|
||||
logger.debug("Connection for {} is backing off for {} ms", destination, remainingBackoffMs);
|
||||
return remainingBackoffMs;
|
||||
return RequestSendResult.of(requestSent, remainingBackoffMs);
|
||||
}
|
||||
|
||||
if (requestManager.isReady(destination, currentTimeMs)) {
|
||||
|
@ -2788,10 +2807,14 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
|
||||
requestManager.onRequestSent(destination, correlationId, currentTimeMs);
|
||||
channel.send(requestMessage);
|
||||
requestSent = true;
|
||||
logger.trace("Sent outbound request: {}", requestMessage);
|
||||
}
|
||||
|
||||
return requestManager.remainingRequestTimeMs(destination, currentTimeMs);
|
||||
return RequestSendResult.of(
|
||||
requestSent,
|
||||
requestManager.remainingRequestTimeMs(destination, currentTimeMs)
|
||||
);
|
||||
}
|
||||
|
||||
private EndQuorumEpochRequestData buildEndQuorumEpochRequest(
|
||||
|
@ -2813,7 +2836,8 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
) {
|
||||
long minBackoffMs = Long.MAX_VALUE;
|
||||
for (Node destination : destinations) {
|
||||
long backoffMs = maybeSendRequest(currentTimeMs, destination, requestSupplier);
|
||||
long backoffMs = maybeSendRequest(currentTimeMs, destination, requestSupplier)
|
||||
.timeToWaitMs();
|
||||
if (backoffMs < minBackoffMs) {
|
||||
minBackoffMs = backoffMs;
|
||||
}
|
||||
|
@ -2833,7 +2857,7 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
currentTimeMs,
|
||||
destinationSupplier.apply(voter.id()),
|
||||
() -> requestSupplier.apply(voter)
|
||||
);
|
||||
).timeToWaitMs();
|
||||
minBackoffMs = Math.min(minBackoffMs, backoffMs);
|
||||
}
|
||||
return minBackoffMs;
|
||||
|
@ -2884,11 +2908,13 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
|
||||
private long maybeSendFetchToAnyBootstrap(long currentTimeMs) {
|
||||
Optional<Node> readyNode = requestManager.findReadyBootstrapServer(currentTimeMs);
|
||||
return readyNode.map(node -> maybeSendRequest(
|
||||
currentTimeMs,
|
||||
node,
|
||||
this::buildFetchRequest
|
||||
)).orElseGet(() -> requestManager.backoffBeforeAvailableBootstrapServer(currentTimeMs));
|
||||
return readyNode.map(
|
||||
node -> maybeSendRequest(
|
||||
currentTimeMs,
|
||||
node,
|
||||
this::buildFetchRequest
|
||||
).timeToWaitMs()
|
||||
).orElseGet(() -> requestManager.backoffBeforeAvailableBootstrapServer(currentTimeMs));
|
||||
}
|
||||
|
||||
private FetchSnapshotRequestData buildFetchSnapshotRequest(OffsetAndEpoch snapshotId, long snapshotSize) {
|
||||
|
@ -2937,7 +2963,12 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
|
||||
future.whenComplete((commitTimeMs, exception) -> {
|
||||
if (exception != null) {
|
||||
logger.debug("Failed to commit {} records up to last offset {}", batch.numRecords, offsetAndEpoch, exception);
|
||||
logger.debug(
|
||||
"Failed to commit {} records up to last offset {}",
|
||||
batch.numRecords,
|
||||
offsetAndEpoch,
|
||||
exception
|
||||
);
|
||||
} else {
|
||||
long elapsedTime = Math.max(0, commitTimeMs - appendTimeMs);
|
||||
double elapsedTimePerRecord = (double) elapsedTime / batch.numRecords;
|
||||
|
@ -2974,6 +3005,7 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
iterator.next().release();
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
return state.accumulator().timeUntilDrain(currentTimeMs);
|
||||
|
@ -3180,6 +3212,25 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
}
|
||||
}
|
||||
|
||||
private boolean shouldSendUpdateVoteRequest(FollowerState state) {
|
||||
var version = partitionState.lastKraftVersion();
|
||||
/* When the cluster supports reconfiguration, send an updated voter configuration if the
|
||||
* one in the log doesn't match the local configuration.
|
||||
*/
|
||||
var sendWhenReconfigSupported = version.isReconfigSupported() &&
|
||||
partitionState.lastVoterSet().voterNodeNeedsUpdate(quorum.localVoterNodeOrThrow());
|
||||
|
||||
/* When the cluster doesn't support reconfiguration, the voter needs to send its voter
|
||||
* information to every new leader. This is because leaders don't persist voter information
|
||||
* when reconfiguration has not been enabled. The updated voter information is required to
|
||||
* be able to upgrade the cluster from kraft.version 0.
|
||||
*/
|
||||
var sendWhenReconfigNotSupported = !version.isReconfigSupported() &&
|
||||
!state.hasUpdatedLeader();
|
||||
|
||||
return sendWhenReconfigSupported || sendWhenReconfigNotSupported;
|
||||
}
|
||||
|
||||
private long pollFollowerAsVoter(FollowerState state, long currentTimeMs) {
|
||||
GracefulShutdown shutdown = this.shutdown.get();
|
||||
final long backoffMs;
|
||||
|
@ -3192,13 +3243,21 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
transitionToProspective(currentTimeMs);
|
||||
backoffMs = 0;
|
||||
} else if (state.hasUpdateVoterPeriodExpired(currentTimeMs)) {
|
||||
if (partitionState.lastKraftVersion().isReconfigSupported() &&
|
||||
partitionState.lastVoterSet().voterNodeNeedsUpdate(quorum.localVoterNodeOrThrow())) {
|
||||
backoffMs = maybeSendUpdateVoterRequest(state, currentTimeMs);
|
||||
final boolean resetUpdateVoterTimer;
|
||||
if (shouldSendUpdateVoteRequest(state)) {
|
||||
var sendResult = maybeSendUpdateVoterRequest(state, currentTimeMs);
|
||||
// Update the request timer if the request was sent
|
||||
resetUpdateVoterTimer = sendResult.requestSent();
|
||||
backoffMs = sendResult.timeToWaitMs();
|
||||
} else {
|
||||
backoffMs = maybeSendFetchOrFetchSnapshot(state, currentTimeMs);
|
||||
// Reset the update voter timer since there was no need to update the voter
|
||||
resetUpdateVoterTimer = true;
|
||||
backoffMs = maybeSendFetchToBestNode(state, currentTimeMs);
|
||||
}
|
||||
|
||||
if (resetUpdateVoterTimer) {
|
||||
state.resetUpdateVoterPeriod(currentTimeMs);
|
||||
}
|
||||
state.resetUpdateVoterPeriod(currentTimeMs);
|
||||
} else {
|
||||
backoffMs = maybeSendFetchToBestNode(state, currentTimeMs);
|
||||
}
|
||||
|
@ -3256,7 +3315,7 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
currentTimeMs,
|
||||
state.leaderNode(channel.listenerName()),
|
||||
requestSupplier
|
||||
);
|
||||
).timeToWaitMs();
|
||||
}
|
||||
|
||||
private UpdateRaftVoterRequestData buildUpdateVoterRequest() {
|
||||
|
@ -3269,7 +3328,7 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
);
|
||||
}
|
||||
|
||||
private long maybeSendUpdateVoterRequest(FollowerState state, long currentTimeMs) {
|
||||
private RequestSendResult maybeSendUpdateVoterRequest(FollowerState state, long currentTimeMs) {
|
||||
return maybeSendRequest(
|
||||
currentTimeMs,
|
||||
state.leaderNode(channel.listenerName()),
|
||||
|
@ -3568,7 +3627,7 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
return;
|
||||
}
|
||||
|
||||
LeaderState<Object> leaderState = leaderStateOpt.get();
|
||||
LeaderState<?> leaderState = leaderStateOpt.get();
|
||||
if (leaderState.epoch() != epoch) {
|
||||
logger.debug("Ignoring call to resign from epoch {} since it is smaller than the " +
|
||||
"current epoch {}", epoch, leaderState.epoch());
|
||||
|
@ -3621,7 +3680,35 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
|
||||
@Override
|
||||
public KRaftVersion kraftVersion() {
|
||||
return partitionState.lastKraftVersion();
|
||||
if (!isInitialized()) {
|
||||
throw new IllegalStateException("Cannot read the kraft version before the replica has been initialized");
|
||||
}
|
||||
|
||||
return quorum
|
||||
.maybeLeaderState()
|
||||
.flatMap(LeaderState::requestedKRaftVersion)
|
||||
.map(KRaftVersionUpgrade.Version::kraftVersion)
|
||||
.orElseGet(partitionState::lastKraftVersion);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void upgradeKRaftVersion(int epoch, KRaftVersion version, boolean validateOnly) {
|
||||
if (!isInitialized()) {
|
||||
throw new IllegalStateException("Cannot update the kraft version before the replica has been initialized");
|
||||
}
|
||||
|
||||
LeaderState<?> leaderState = quorum.maybeLeaderState().orElseThrow(
|
||||
() -> new NotLeaderException("Upgrade kraft version failed because the replica is not the current leader")
|
||||
);
|
||||
|
||||
leaderState.maybeAppendUpgradedKRaftVersion(
|
||||
epoch,
|
||||
version,
|
||||
partitionState.lastKraftVersion(),
|
||||
partitionState.lastVoterSet(),
|
||||
validateOnly,
|
||||
time.milliseconds()
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -16,6 +16,8 @@
|
|||
*/
|
||||
package org.apache.kafka.raft;
|
||||
|
||||
import org.apache.kafka.common.compress.Compression;
|
||||
import org.apache.kafka.common.errors.InvalidUpdateVersionException;
|
||||
import org.apache.kafka.common.message.KRaftVersionRecord;
|
||||
import org.apache.kafka.common.message.LeaderChangeMessage;
|
||||
import org.apache.kafka.common.message.LeaderChangeMessage.Voter;
|
||||
|
@ -27,14 +29,17 @@ import org.apache.kafka.common.record.TimestampType;
|
|||
import org.apache.kafka.common.utils.LogContext;
|
||||
import org.apache.kafka.common.utils.Time;
|
||||
import org.apache.kafka.common.utils.Timer;
|
||||
import org.apache.kafka.raft.errors.NotLeaderException;
|
||||
import org.apache.kafka.raft.internals.AddVoterHandlerState;
|
||||
import org.apache.kafka.raft.internals.BatchAccumulator;
|
||||
import org.apache.kafka.raft.internals.KRaftVersionUpgrade;
|
||||
import org.apache.kafka.raft.internals.KafkaRaftMetrics;
|
||||
import org.apache.kafka.raft.internals.RemoveVoterHandlerState;
|
||||
import org.apache.kafka.server.common.KRaftVersion;
|
||||
|
||||
import org.slf4j.Logger;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
|
@ -45,6 +50,7 @@ import java.util.Optional;
|
|||
import java.util.OptionalLong;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
|
@ -58,11 +64,10 @@ public class LeaderState<T> implements EpochState {
|
|||
static final long OBSERVER_SESSION_TIMEOUT_MS = 300_000L;
|
||||
static final double CHECK_QUORUM_TIMEOUT_FACTOR = 1.5;
|
||||
|
||||
private final ReplicaKey localReplicaKey;
|
||||
private final VoterSet.VoterNode localVoterNode;
|
||||
private final int epoch;
|
||||
private final long epochStartOffset;
|
||||
private final Set<Integer> grantingVoters;
|
||||
private final Endpoints localListeners;
|
||||
private final VoterSet voterSetAtEpochStart;
|
||||
// This field is non-empty if the voter set at epoch start came from a snapshot or log segment
|
||||
private final OptionalLong offsetOfVotersAtEpochStart;
|
||||
|
@ -87,9 +92,25 @@ public class LeaderState<T> implements EpochState {
|
|||
// This is volatile because resignation can be requested from an external thread.
|
||||
private volatile boolean resignRequested = false;
|
||||
|
||||
/* Used to coordinate the upgrade of the kraft.version from 0 to 1. The upgrade is triggered by
|
||||
* the clients to RaftClient.
|
||||
* 1. if the kraft version is 0, the starting state is the Voters type. The voter set is the voters in
|
||||
* the static voter set with the leader updated. See KRaftVersionUpgrade for details on the
|
||||
* Voters type.
|
||||
* 2. as the leader receives UpdateRaftVoter requests, it updates the associated Voters type. Only
|
||||
* after all of the voters have been updated will an upgrade successfully complete.
|
||||
* 3. a client of RaftClient triggers the upgrade and transition this state to the Version
|
||||
* type. See KRaftVersionUpgrade for details on the Version type.
|
||||
*
|
||||
* All transition are coordinated using optimistic locking by always calling AtomicReference#compareAndSet
|
||||
*/
|
||||
private final AtomicReference<KRaftVersionUpgrade> kraftVersionUpgradeState = new AtomicReference<>(
|
||||
KRaftVersionUpgrade.empty()
|
||||
);
|
||||
|
||||
protected LeaderState(
|
||||
Time time,
|
||||
ReplicaKey localReplicaKey,
|
||||
VoterSet.VoterNode localVoterNode,
|
||||
int epoch,
|
||||
long epochStartOffset,
|
||||
VoterSet voterSetAtEpochStart,
|
||||
|
@ -97,18 +118,30 @@ public class LeaderState<T> implements EpochState {
|
|||
KRaftVersion kraftVersionAtEpochStart,
|
||||
Set<Integer> grantingVoters,
|
||||
BatchAccumulator<T> accumulator,
|
||||
Endpoints localListeners,
|
||||
int fetchTimeoutMs,
|
||||
LogContext logContext,
|
||||
KafkaRaftMetrics kafkaRaftMetrics
|
||||
) {
|
||||
this.localReplicaKey = localReplicaKey;
|
||||
if (localVoterNode.voterKey().directoryId().isEmpty()) {
|
||||
throw new IllegalArgumentException(
|
||||
String.format("Unknown local replica directory id: %s", localVoterNode)
|
||||
);
|
||||
} else if (!voterSetAtEpochStart.isVoter(localVoterNode.voterKey())) {
|
||||
throw new IllegalArgumentException(
|
||||
String.format(
|
||||
"Local replica %s is not a voter in %s",
|
||||
localVoterNode,
|
||||
voterSetAtEpochStart
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
this.localVoterNode = localVoterNode;
|
||||
this.epoch = epoch;
|
||||
this.epochStartOffset = epochStartOffset;
|
||||
this.localListeners = localListeners;
|
||||
|
||||
for (VoterSet.VoterNode voterNode: voterSetAtEpochStart.voterNodes()) {
|
||||
boolean hasAcknowledgedLeader = voterNode.isVoter(localReplicaKey);
|
||||
boolean hasAcknowledgedLeader = voterNode.isVoter(localVoterNode.voterKey());
|
||||
this.voterStates.put(
|
||||
voterNode.voterKey().id(),
|
||||
new ReplicaState(voterNode.voterKey(), hasAcknowledgedLeader, voterNode.listeners())
|
||||
|
@ -128,6 +161,21 @@ public class LeaderState<T> implements EpochState {
|
|||
|
||||
kafkaRaftMetrics.addLeaderMetrics();
|
||||
this.kafkaRaftMetrics = kafkaRaftMetrics;
|
||||
|
||||
if (!kraftVersionAtEpochStart.isReconfigSupported()) {
|
||||
var updatedVoters = voterSetAtEpochStart
|
||||
.updateVoterIgnoringDirectoryId(localVoterNode)
|
||||
.orElseThrow(
|
||||
() -> new IllegalStateException(
|
||||
String.format(
|
||||
"Unable to update voter set %s with the latest leader information %s",
|
||||
voterSetAtEpochStart,
|
||||
localVoterNode
|
||||
)
|
||||
)
|
||||
);
|
||||
kraftVersionUpgradeState.set(new KRaftVersionUpgrade.Voters(updatedVoters));
|
||||
}
|
||||
}
|
||||
|
||||
public long timeUntilBeginQuorumEpochTimerExpires(long currentTimeMs) {
|
||||
|
@ -185,7 +233,7 @@ public class LeaderState<T> implements EpochState {
|
|||
// majority, but the leader will never be a member of the fetchedVoters.
|
||||
// If the leader is not in the voter set, it is not in the majority. Then, the
|
||||
// majority can only be composed of fetched voters.
|
||||
if (voterStates.containsKey(localReplicaKey.id())) {
|
||||
if (voterStates.containsKey(localVoterNode.voterKey().id())) {
|
||||
majority = majority - 1;
|
||||
}
|
||||
|
||||
|
@ -197,7 +245,7 @@ public class LeaderState<T> implements EpochState {
|
|||
}
|
||||
|
||||
private void updateFetchedVoters(ReplicaKey replicaKey) {
|
||||
if (replicaKey.id() == localReplicaKey.id()) {
|
||||
if (replicaKey.id() == localVoterNode.voterKey().id()) {
|
||||
throw new IllegalArgumentException("Received a FETCH/FETCH_SNAPSHOT request from the leader itself.");
|
||||
}
|
||||
|
||||
|
@ -293,25 +341,32 @@ public class LeaderState<T> implements EpochState {
|
|||
.collect(Collectors.toList());
|
||||
}
|
||||
|
||||
public void appendStartOfEpochControlRecords(VoterSet.VoterNode localVoterNode, long currentTimeMs) {
|
||||
if (!localReplicaKey.equals(localVoterNode.voterKey())) {
|
||||
throw new IllegalArgumentException(
|
||||
String.format(
|
||||
"Replica key %s didn't match the local key %s",
|
||||
localVoterNode.voterKey(),
|
||||
localReplicaKey
|
||||
)
|
||||
);
|
||||
} else if (!localListeners.equals(localVoterNode.listeners())) {
|
||||
throw new IllegalArgumentException(
|
||||
String.format(
|
||||
"Listeners %s didn't match the local listeners %s",
|
||||
localVoterNode.listeners(),
|
||||
localListeners
|
||||
)
|
||||
);
|
||||
}
|
||||
private static MemoryRecordsBuilder createControlRecordsBuilder(
|
||||
long baseOffset,
|
||||
int epoch,
|
||||
Compression compression,
|
||||
ByteBuffer buffer,
|
||||
long currentTimeMs
|
||||
) {
|
||||
return new MemoryRecordsBuilder(
|
||||
buffer,
|
||||
RecordBatch.CURRENT_MAGIC_VALUE,
|
||||
compression,
|
||||
TimestampType.CREATE_TIME,
|
||||
baseOffset,
|
||||
currentTimeMs,
|
||||
RecordBatch.NO_PRODUCER_ID,
|
||||
RecordBatch.NO_PRODUCER_EPOCH,
|
||||
RecordBatch.NO_SEQUENCE,
|
||||
false, // isTransactional
|
||||
true, // isControlBatch
|
||||
epoch,
|
||||
buffer.capacity()
|
||||
);
|
||||
}
|
||||
|
||||
|
||||
public void appendStartOfEpochControlRecords(long currentTimeMs) {
|
||||
List<Voter> voters = convertToVoters(voterStates.keySet());
|
||||
List<Voter> grantingVoters = convertToVoters(this.grantingVoters());
|
||||
|
||||
|
@ -322,20 +377,12 @@ public class LeaderState<T> implements EpochState {
|
|||
.setGrantingVoters(grantingVoters);
|
||||
|
||||
accumulator.appendControlMessages((baseOffset, epoch, compression, buffer) -> {
|
||||
try (MemoryRecordsBuilder builder = new MemoryRecordsBuilder(
|
||||
buffer,
|
||||
RecordBatch.CURRENT_MAGIC_VALUE,
|
||||
compression,
|
||||
TimestampType.CREATE_TIME,
|
||||
try (MemoryRecordsBuilder builder = createControlRecordsBuilder(
|
||||
baseOffset,
|
||||
currentTimeMs,
|
||||
RecordBatch.NO_PRODUCER_ID,
|
||||
RecordBatch.NO_PRODUCER_EPOCH,
|
||||
RecordBatch.NO_SEQUENCE,
|
||||
false, // isTransactional
|
||||
true, // isControlBatch
|
||||
epoch,
|
||||
buffer.capacity()
|
||||
compression,
|
||||
buffer,
|
||||
currentTimeMs
|
||||
)
|
||||
) {
|
||||
builder.appendLeaderChangeMessage(currentTimeMs, leaderChangeMessage);
|
||||
|
@ -395,6 +442,21 @@ public class LeaderState<T> implements EpochState {
|
|||
);
|
||||
}
|
||||
|
||||
public boolean compareAndSetVolatileVoters(
|
||||
KRaftVersionUpgrade.Voters oldVoters,
|
||||
KRaftVersionUpgrade.Voters newVoters
|
||||
) {
|
||||
return kraftVersionUpgradeState.compareAndSet(oldVoters, newVoters);
|
||||
}
|
||||
|
||||
public Optional<KRaftVersionUpgrade.Voters> volatileVoters() {
|
||||
return kraftVersionUpgradeState.get().toVoters();
|
||||
}
|
||||
|
||||
public Optional<KRaftVersionUpgrade.Version> requestedKRaftVersion() {
|
||||
return kraftVersionUpgradeState.get().toVersion();
|
||||
}
|
||||
|
||||
public boolean isResignRequested() {
|
||||
return resignRequested;
|
||||
}
|
||||
|
@ -416,6 +478,185 @@ public class LeaderState<T> implements EpochState {
|
|||
this.resignRequested = true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Upgrade the kraft version.
|
||||
*
|
||||
* This methods upgrades the kraft version to {@code newVersion}. If the version is already
|
||||
* {@code newVersion}, this is a noop operation.
|
||||
*
|
||||
* KRaft only supports upgrades, so {@code newVersion} must be greater than or equal to curent
|
||||
* kraft version {@code persistedVersion}.
|
||||
*
|
||||
* For the upgrade to succeed all of the voters in the voter set must support the new kraft
|
||||
* version. The upgrade from kraft version 0 to kraft version 1 generate one control batch
|
||||
* with one control record setting the kraft version to 1 and one voters record setting the
|
||||
* updated voter set.
|
||||
*
|
||||
* When {@code validateOnly} is true only the validation is perform and the control records are
|
||||
* not generated.
|
||||
*
|
||||
* @param currentEpoch the current epoch
|
||||
* @param newVersion the new kraft version
|
||||
* @param persistedVersion the kraft version persisted to disk
|
||||
* @param persistedVoters the set of voters persisted to disk
|
||||
* @param validateOnly determine if only validation should be performed
|
||||
* @param currentTimeMs the current time
|
||||
*/
|
||||
public boolean maybeAppendUpgradedKRaftVersion(
|
||||
int currentEpoch,
|
||||
KRaftVersion newVersion,
|
||||
KRaftVersion persistedVersion,
|
||||
VoterSet persistedVoters,
|
||||
boolean validateOnly,
|
||||
long currentTimeMs
|
||||
) {
|
||||
validateEpoch(currentEpoch);
|
||||
|
||||
var pendingVersion = kraftVersionUpgradeState.get().toVersion();
|
||||
if (pendingVersion.isPresent()) {
|
||||
if (pendingVersion.get().kraftVersion().equals(newVersion)) {
|
||||
// The version match; upgrade is a noop
|
||||
return false;
|
||||
} else {
|
||||
throw new InvalidUpdateVersionException(
|
||||
String.format(
|
||||
"Invalid concurrent upgrade of %s from version %s to %s",
|
||||
KRaftVersion.FEATURE_NAME,
|
||||
pendingVersion.get(),
|
||||
newVersion
|
||||
)
|
||||
);
|
||||
}
|
||||
} else if (persistedVersion.equals(newVersion)) {
|
||||
return false;
|
||||
} else if (persistedVersion.isMoreThan(newVersion)) {
|
||||
throw new InvalidUpdateVersionException(
|
||||
String.format(
|
||||
"Invalid upgrade of %s from version %s to %s because the new version is a downgrade",
|
||||
KRaftVersion.FEATURE_NAME,
|
||||
persistedVersion,
|
||||
newVersion
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
// Upgrade to kraft.verion 1 is only supported; this needs to change when kraft.version 2 is added
|
||||
var inMemoryVoters = kraftVersionUpgradeState.get().toVoters().orElseThrow(() ->
|
||||
new InvalidUpdateVersionException(
|
||||
String.format(
|
||||
"Invalid upgrade of %s from version %s to %s",
|
||||
KRaftVersion.FEATURE_NAME,
|
||||
persistedVersion,
|
||||
newVersion
|
||||
)
|
||||
)
|
||||
);
|
||||
if (!inMemoryVoters.voters().voterIds().equals(persistedVoters.voterIds())) {
|
||||
throw new IllegalStateException(
|
||||
String.format(
|
||||
"Unable to update %s to %s due to missing voters %s compared to %s",
|
||||
KRaftVersion.FEATURE_NAME,
|
||||
newVersion,
|
||||
inMemoryVoters.voters().voterIds(),
|
||||
persistedVoters.voterIds()
|
||||
)
|
||||
);
|
||||
} else if (!inMemoryVoters.voters().supportsVersion(newVersion)) {
|
||||
log.info("Not all voters support kraft version {}: {}", newVersion, inMemoryVoters.voters());
|
||||
throw new InvalidUpdateVersionException(
|
||||
String.format(
|
||||
"Invalid upgrade of %s to %s because not all of the voters support it",
|
||||
KRaftVersion.FEATURE_NAME,
|
||||
newVersion
|
||||
)
|
||||
);
|
||||
} else if (
|
||||
inMemoryVoters
|
||||
.voters()
|
||||
.voterKeys()
|
||||
.stream()
|
||||
.anyMatch(voterKey -> voterKey.directoryId().isEmpty())
|
||||
) {
|
||||
throw new IllegalStateException(
|
||||
String.format(
|
||||
"Directory id must be known for all of the voters: %s",
|
||||
inMemoryVoters.voters()
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
if (!validateOnly) {
|
||||
/* Note that this only supports upgrades from kraft.version 0 to kraft.version 1. When
|
||||
* kraft.version 2 is added, this logic needs to be revisited
|
||||
*/
|
||||
var successful = kraftVersionUpgradeState.compareAndSet(
|
||||
inMemoryVoters,
|
||||
new KRaftVersionUpgrade.Version(newVersion)
|
||||
);
|
||||
if (!successful) {
|
||||
throw new InvalidUpdateVersionException(
|
||||
String.format(
|
||||
"Unable to upgrade version for %s to %s due to changing voters",
|
||||
KRaftVersion.FEATURE_NAME,
|
||||
newVersion
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
// All of the validations succeeded; create control records for the upgrade
|
||||
accumulator.appendControlMessages((baseOffset, batchEpoch, compression, buffer) -> {
|
||||
try (MemoryRecordsBuilder builder = createControlRecordsBuilder(
|
||||
baseOffset,
|
||||
batchEpoch,
|
||||
compression,
|
||||
buffer,
|
||||
currentTimeMs
|
||||
)
|
||||
) {
|
||||
log.info("Appended kraft.version {} to the batch accumulator", newVersion);
|
||||
builder.appendKRaftVersionMessage(
|
||||
currentTimeMs,
|
||||
new KRaftVersionRecord()
|
||||
.setVersion(newVersion.kraftVersionRecordVersion())
|
||||
.setKRaftVersion(newVersion.featureLevel())
|
||||
);
|
||||
|
||||
if (!inMemoryVoters.voters().equals(persistedVoters)) {
|
||||
log.info("Appended voter set {} to the batch accumulator", inMemoryVoters.voters());
|
||||
builder.appendVotersMessage(
|
||||
currentTimeMs,
|
||||
inMemoryVoters.voters().toVotersRecord(newVersion.votersRecordVersion())
|
||||
);
|
||||
}
|
||||
|
||||
return builder.build();
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
private void validateEpoch(int currentEpoch) {
|
||||
if (currentEpoch < epoch()) {
|
||||
throw new NotLeaderException(
|
||||
String.format(
|
||||
"Upgrade kraft version failed because the given epoch %s is stale. Current leader epoch is %s",
|
||||
currentEpoch,
|
||||
epoch()
|
||||
)
|
||||
);
|
||||
} else if (currentEpoch > epoch()) {
|
||||
throw new IllegalArgumentException(
|
||||
String.format(
|
||||
"Attempt to append from epoch %s which is larger than the current epoch of %s",
|
||||
currentEpoch,
|
||||
epoch()
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<LogOffsetMetadata> highWatermark() {
|
||||
return highWatermark;
|
||||
|
@ -423,7 +664,7 @@ public class LeaderState<T> implements EpochState {
|
|||
|
||||
@Override
|
||||
public ElectionState election() {
|
||||
return ElectionState.withElectedLeader(epoch, localReplicaKey.id(), Optional.empty(), voterStates.keySet());
|
||||
return ElectionState.withElectedLeader(epoch, localVoterNode.voterKey().id(), Optional.empty(), voterStates.keySet());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -433,7 +674,7 @@ public class LeaderState<T> implements EpochState {
|
|||
|
||||
@Override
|
||||
public Endpoints leaderEndpoints() {
|
||||
return localListeners;
|
||||
return localVoterNode.listeners();
|
||||
}
|
||||
|
||||
Map<Integer, ReplicaState> voterStates() {
|
||||
|
@ -557,7 +798,7 @@ public class LeaderState<T> implements EpochState {
|
|||
LogOffsetMetadata endOffsetMetadata,
|
||||
VoterSet lastVoterSet
|
||||
) {
|
||||
ReplicaState state = getOrCreateReplicaState(localReplicaKey);
|
||||
ReplicaState state = getOrCreateReplicaState(localVoterNode.voterKey());
|
||||
state.endOffset.ifPresent(currentEndOffset -> {
|
||||
if (currentEndOffset.offset() > endOffsetMetadata.offset()) {
|
||||
throw new IllegalStateException("Detected non-monotonic update of local " +
|
||||
|
@ -588,7 +829,7 @@ public class LeaderState<T> implements EpochState {
|
|||
// the fetch is from non-replica. For example, a consumer.
|
||||
if (replicaKey.id() < 0) {
|
||||
return false;
|
||||
} else if (replicaKey.id() == localReplicaKey.id()) {
|
||||
} else if (replicaKey.id() == localVoterNode.voterKey().id()) {
|
||||
throw new IllegalStateException(
|
||||
String.format("Remote replica ID %s matches the local leader ID", replicaKey)
|
||||
);
|
||||
|
@ -603,7 +844,7 @@ public class LeaderState<T> implements EpochState {
|
|||
}
|
||||
});
|
||||
|
||||
Optional<LogOffsetMetadata> leaderEndOffsetOpt = getOrCreateReplicaState(localReplicaKey).endOffset;
|
||||
Optional<LogOffsetMetadata> leaderEndOffsetOpt = getOrCreateReplicaState(localVoterNode.voterKey()).endOffset;
|
||||
|
||||
state.updateFollowerState(
|
||||
currentTimeMs,
|
||||
|
@ -617,7 +858,7 @@ public class LeaderState<T> implements EpochState {
|
|||
|
||||
public List<ReplicaKey> nonLeaderVotersByDescendingFetchOffset() {
|
||||
return followersByDescendingFetchOffset()
|
||||
.filter(state -> !state.matchesKey(localReplicaKey))
|
||||
.filter(state -> !state.matchesKey(localVoterNode.voterKey()))
|
||||
.map(state -> state.replicaKey)
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
|
@ -671,7 +912,7 @@ public class LeaderState<T> implements EpochState {
|
|||
private void clearInactiveObservers(final long currentTimeMs) {
|
||||
observerStates.entrySet().removeIf(integerReplicaStateEntry ->
|
||||
currentTimeMs - integerReplicaStateEntry.getValue().lastFetchTimestamp >= OBSERVER_SESSION_TIMEOUT_MS &&
|
||||
!integerReplicaStateEntry.getKey().equals(localReplicaKey)
|
||||
!integerReplicaStateEntry.getKey().equals(localVoterNode.voterKey())
|
||||
);
|
||||
kafkaRaftMetrics.updateNumObservers(observerStates.size());
|
||||
}
|
||||
|
@ -864,8 +1105,8 @@ public class LeaderState<T> implements EpochState {
|
|||
@Override
|
||||
public String toString() {
|
||||
return String.format(
|
||||
"Leader(localReplicaKey=%s, epoch=%d, epochStartOffset=%d, highWatermark=%s, voterStates=%s)",
|
||||
localReplicaKey,
|
||||
"Leader(localVoterNode=%s, epoch=%d, epochStartOffset=%d, highWatermark=%s, voterStates=%s)",
|
||||
localVoterNode,
|
||||
epoch,
|
||||
epochStartOffset,
|
||||
highWatermark,
|
||||
|
|
|
@ -716,7 +716,7 @@ public class QuorumState {
|
|||
|
||||
LeaderState<T> state = new LeaderState<>(
|
||||
time,
|
||||
ReplicaKey.of(localIdOrThrow(), localDirectoryId),
|
||||
localVoterNodeOrThrow(),
|
||||
epoch(),
|
||||
epochStartOffset,
|
||||
partitionState.lastVoterSet(),
|
||||
|
@ -724,7 +724,6 @@ public class QuorumState {
|
|||
partitionState.lastKraftVersion(),
|
||||
candidateState.epochElection().grantingVoters(),
|
||||
accumulator,
|
||||
localListeners,
|
||||
fetchTimeoutMs,
|
||||
logContext,
|
||||
kafkaRaftMetrics
|
||||
|
|
|
@ -228,7 +228,7 @@ public interface RaftClient<T> extends AutoCloseable {
|
|||
* Returns the current end of the log. This method is thread-safe.
|
||||
*
|
||||
* @return the log end offset, which is one greater than the offset of the last record written,
|
||||
* or 0 if there have not been any records written.
|
||||
* or 0 if there have not been any records written.
|
||||
*/
|
||||
long logEndOffset();
|
||||
|
||||
|
@ -238,4 +238,18 @@ public interface RaftClient<T> extends AutoCloseable {
|
|||
* @return the current kraft.version.
|
||||
*/
|
||||
KRaftVersion kraftVersion();
|
||||
|
||||
/**
|
||||
* Request that the leader to upgrade the kraft version.
|
||||
*
|
||||
* @param epoch the current epoch
|
||||
* @param version the new kraft version to upgrade to
|
||||
* @param validateOnly whether to just validate the change and not persist it
|
||||
* @throws ApiException when the upgrade fails to validate
|
||||
*/
|
||||
void upgradeKRaftVersion(
|
||||
int epoch,
|
||||
KRaftVersion version,
|
||||
boolean validateOnly
|
||||
);
|
||||
}
|
||||
|
|
|
@ -22,6 +22,7 @@ import org.apache.kafka.common.feature.SupportedVersionRange;
|
|||
import org.apache.kafka.common.message.VotersRecord;
|
||||
import org.apache.kafka.common.network.ListenerName;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
import org.apache.kafka.server.common.KRaftVersion;
|
||||
|
||||
import java.net.InetSocketAddress;
|
||||
import java.util.HashMap;
|
||||
|
@ -250,6 +251,26 @@ public final class VoterSet {
|
|||
return Optional.empty();
|
||||
}
|
||||
|
||||
/**
|
||||
* Update a voter by only comparing the node id.
|
||||
*
|
||||
* This update voter operation doesn't compare the directory id. This is useful when upgrading
|
||||
* from a voter set that doesn't support directory id to one that supports directory ids.
|
||||
*
|
||||
* @param voter the updated voter
|
||||
* @return a new voter set if the voter was updated, otherwise {@code Optional.empty()}
|
||||
*/
|
||||
public Optional<VoterSet> updateVoterIgnoringDirectoryId(VoterNode voter) {
|
||||
if (voters.containsKey(voter.voterKey().id())) {
|
||||
HashMap<Integer, VoterNode> newVoters = new HashMap<>(voters);
|
||||
newVoters.put(voter.voterKey().id(), voter);
|
||||
|
||||
return Optional.of(new VoterSet(newVoters));
|
||||
}
|
||||
|
||||
return Optional.empty();
|
||||
}
|
||||
|
||||
/**
|
||||
* Converts a voter set to a voters record for a given version.
|
||||
*
|
||||
|
@ -303,6 +324,19 @@ public final class VoterSet {
|
|||
return Utils.diff(HashSet::new, thatReplicaKeys, thisReplicaKeys).size() <= 1;
|
||||
}
|
||||
|
||||
/**
|
||||
* Determines if the voter set supports a given kraft version.
|
||||
*
|
||||
* @param version the kraft version to check
|
||||
* @return true if all of the voter support the given kraft version, false otherwise
|
||||
*/
|
||||
public boolean supportsVersion(KRaftVersion version) {
|
||||
return voters
|
||||
.values()
|
||||
.stream()
|
||||
.allMatch(voter -> voter.supportsVersion(version));
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o) return true;
|
||||
|
@ -373,15 +407,20 @@ public final class VoterSet {
|
|||
return listeners;
|
||||
}
|
||||
|
||||
SupportedVersionRange supportedKRaftVersion() {
|
||||
private SupportedVersionRange supportedKRaftVersion() {
|
||||
return supportedKRaftVersion;
|
||||
}
|
||||
|
||||
|
||||
Optional<InetSocketAddress> address(ListenerName listener) {
|
||||
private Optional<InetSocketAddress> address(ListenerName listener) {
|
||||
return listeners.address(listener);
|
||||
}
|
||||
|
||||
private boolean supportsVersion(KRaftVersion version) {
|
||||
return version.featureLevel() >= supportedKRaftVersion.min() &&
|
||||
version.featureLevel() <= supportedKRaftVersion.max();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o) return true;
|
||||
|
|
|
@ -0,0 +1,65 @@
|
|||
/*
|
||||
* 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.raft.internals;
|
||||
|
||||
import org.apache.kafka.raft.VoterSet;
|
||||
import org.apache.kafka.server.common.KRaftVersion;
|
||||
|
||||
import java.util.Optional;
|
||||
|
||||
/**
|
||||
* KRaftVersionUpgrade is a sum types for coordinating the upgrade of the kraft version.
|
||||
*
|
||||
* {@code Voters} is used to stored in-memory the latest voter set. {@code Version} is used to
|
||||
* store in-memory the upgraded kraft version.
|
||||
*
|
||||
* This type makes it possible to upgrade the kraft version by only using compare-and-swap and
|
||||
* avoid blocking locks.
|
||||
*/
|
||||
public sealed interface KRaftVersionUpgrade {
|
||||
public record Empty() implements KRaftVersionUpgrade {
|
||||
}
|
||||
|
||||
public record Version(KRaftVersion kraftVersion) implements KRaftVersionUpgrade {
|
||||
}
|
||||
|
||||
public record Voters(VoterSet voters) implements KRaftVersionUpgrade {
|
||||
}
|
||||
|
||||
public default Optional<Voters> toVoters() {
|
||||
if (this instanceof Voters) {
|
||||
return Optional.of(((Voters) this));
|
||||
} else {
|
||||
return Optional.empty();
|
||||
}
|
||||
}
|
||||
|
||||
public default Optional<Version> toVersion() {
|
||||
if (this instanceof Version) {
|
||||
return Optional.of(((Version) this));
|
||||
} else {
|
||||
return Optional.empty();
|
||||
}
|
||||
}
|
||||
|
||||
static final KRaftVersionUpgrade EMPTY = new Empty();
|
||||
|
||||
public static KRaftVersionUpgrade empty() {
|
||||
return EMPTY;
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,30 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.kafka.raft.internals;
|
||||
|
||||
/**
|
||||
* Type to capture the atempt to send a request.
|
||||
*
|
||||
* @param requestSent true if the request was sent
|
||||
* @param timeToWaitMs the amount of time to wait in milliseconds before attempting to resend the
|
||||
* request.
|
||||
*/
|
||||
public record RequestSendResult(boolean requestSent, long timeToWaitMs) {
|
||||
public static RequestSendResult of(boolean requestSent, long timeToWaitMs) {
|
||||
return new RequestSendResult(requestSent, timeToWaitMs);
|
||||
}
|
||||
}
|
|
@ -21,6 +21,7 @@ import org.apache.kafka.common.message.UpdateRaftVoterRequestData;
|
|||
import org.apache.kafka.common.message.UpdateRaftVoterResponseData;
|
||||
import org.apache.kafka.common.network.ListenerName;
|
||||
import org.apache.kafka.common.protocol.Errors;
|
||||
import org.apache.kafka.common.utils.LogContext;
|
||||
import org.apache.kafka.raft.Endpoints;
|
||||
import org.apache.kafka.raft.LeaderAndEpoch;
|
||||
import org.apache.kafka.raft.LeaderState;
|
||||
|
@ -30,6 +31,8 @@ import org.apache.kafka.raft.ReplicaKey;
|
|||
import org.apache.kafka.raft.VoterSet;
|
||||
import org.apache.kafka.server.common.KRaftVersion;
|
||||
|
||||
import org.slf4j.Logger;
|
||||
|
||||
import java.util.Optional;
|
||||
import java.util.OptionalInt;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
|
@ -44,25 +47,29 @@ import java.util.concurrent.CompletableFuture;
|
|||
* 4. Check that the updated voter still supports the currently finalized kraft.version, otherwise
|
||||
* return the INVALID_REQUEST error.
|
||||
* 5. Check that the updated voter is still listening on the default listener.
|
||||
* 6. Append the updated VotersRecord to the log. The KRaft internal listener will read this
|
||||
* uncommitted record from the log and update the voter in the set of voters.
|
||||
* 6. Update voter set with new voter configuration.
|
||||
* a. If reconfiguration is supported, append the updated VotersRecord to the log. The KRaft internal listener will read this
|
||||
* uncommitted record from the log and update the voter in the set of voters.
|
||||
* b. If reconfiguration is not supported, update the in-memory information for the voter. This will get
|
||||
* appended to the log when the cluster is upgraded to a kraft version that supports reconfiguration.
|
||||
* 7. Send the UpdateVoter successful response to the voter.
|
||||
*
|
||||
* KAFKA-16538 is going to add support for handling this RPC when the kraft.version is 0.
|
||||
*/
|
||||
public final class UpdateVoterHandler {
|
||||
private final OptionalInt localId;
|
||||
private final KRaftControlRecordStateMachine partitionState;
|
||||
private final ListenerName defaultListenerName;
|
||||
private final Logger log;
|
||||
|
||||
public UpdateVoterHandler(
|
||||
OptionalInt localId,
|
||||
KRaftControlRecordStateMachine partitionState,
|
||||
ListenerName defaultListenerName
|
||||
ListenerName defaultListenerName,
|
||||
LogContext logContext
|
||||
) {
|
||||
this.localId = localId;
|
||||
this.partitionState = partitionState;
|
||||
this.defaultListenerName = defaultListenerName;
|
||||
this.log = logContext.logger(getClass());
|
||||
}
|
||||
|
||||
public CompletableFuture<UpdateRaftVoterResponseData> handleUpdateVoterRequest(
|
||||
|
@ -104,26 +111,42 @@ public final class UpdateVoterHandler {
|
|||
);
|
||||
}
|
||||
|
||||
// KAFKA-16538 will implement the case when the kraft.version is 0
|
||||
// Check that the cluster supports kraft.version >= 1
|
||||
// Read the voter set from the log or leader state
|
||||
KRaftVersion kraftVersion = partitionState.lastKraftVersion();
|
||||
if (!kraftVersion.isReconfigSupported()) {
|
||||
return CompletableFuture.completedFuture(
|
||||
RaftUtil.updateVoterResponse(
|
||||
Errors.UNSUPPORTED_VERSION,
|
||||
requestListenerName,
|
||||
new LeaderAndEpoch(
|
||||
localId,
|
||||
leaderState.epoch()
|
||||
),
|
||||
leaderState.leaderEndpoints()
|
||||
)
|
||||
);
|
||||
}
|
||||
final Optional<KRaftVersionUpgrade.Voters> inMemoryVoters;
|
||||
final Optional<VoterSet> voters;
|
||||
if (kraftVersion.isReconfigSupported()) {
|
||||
inMemoryVoters = Optional.empty();
|
||||
|
||||
// Check that there are no uncommitted VotersRecord
|
||||
Optional<LogHistory.Entry<VoterSet>> votersEntry = partitionState.lastVoterSetEntry();
|
||||
if (votersEntry.isEmpty() || votersEntry.get().offset() >= highWatermark.get()) {
|
||||
// Check that there are no uncommitted VotersRecord
|
||||
Optional<LogHistory.Entry<VoterSet>> votersEntry = partitionState.lastVoterSetEntry();
|
||||
if (votersEntry.isEmpty() || votersEntry.get().offset() >= highWatermark.get()) {
|
||||
voters = Optional.empty();
|
||||
} else {
|
||||
voters = votersEntry.map(LogHistory.Entry::value);
|
||||
}
|
||||
} else {
|
||||
inMemoryVoters = leaderState.volatileVoters();
|
||||
if (inMemoryVoters.isEmpty()) {
|
||||
/* This can happen if the remote voter sends an update voter request before the
|
||||
* updated kraft version has been written to the log
|
||||
*/
|
||||
return CompletableFuture.completedFuture(
|
||||
RaftUtil.updateVoterResponse(
|
||||
Errors.REQUEST_TIMED_OUT,
|
||||
requestListenerName,
|
||||
new LeaderAndEpoch(
|
||||
localId,
|
||||
leaderState.epoch()
|
||||
),
|
||||
leaderState.leaderEndpoints()
|
||||
)
|
||||
);
|
||||
}
|
||||
voters = inMemoryVoters.map(KRaftVersionUpgrade.Voters::voters);
|
||||
}
|
||||
if (voters.isEmpty()) {
|
||||
log.info("Unable to read the current voter set with kraft version {}", kraftVersion);
|
||||
return CompletableFuture.completedFuture(
|
||||
RaftUtil.updateVoterResponse(
|
||||
Errors.REQUEST_TIMED_OUT,
|
||||
|
@ -136,7 +159,6 @@ public final class UpdateVoterHandler {
|
|||
)
|
||||
);
|
||||
}
|
||||
|
||||
// Check that the supported version range is valid
|
||||
if (!validVersionRange(kraftVersion, supportedKraftVersions)) {
|
||||
return CompletableFuture.completedFuture(
|
||||
|
@ -168,19 +190,18 @@ public final class UpdateVoterHandler {
|
|||
}
|
||||
|
||||
// Update the voter
|
||||
Optional<VoterSet> updatedVoters = votersEntry
|
||||
.get()
|
||||
.value()
|
||||
.updateVoter(
|
||||
VoterSet.VoterNode.of(
|
||||
voterKey,
|
||||
voterEndpoints,
|
||||
new SupportedVersionRange(
|
||||
supportedKraftVersions.minSupportedVersion(),
|
||||
supportedKraftVersions.maxSupportedVersion()
|
||||
)
|
||||
Optional<VoterSet> updatedVoters = updateVoters(
|
||||
voters.get(),
|
||||
kraftVersion,
|
||||
VoterSet.VoterNode.of(
|
||||
voterKey,
|
||||
voterEndpoints,
|
||||
new SupportedVersionRange(
|
||||
supportedKraftVersions.minSupportedVersion(),
|
||||
supportedKraftVersions.maxSupportedVersion()
|
||||
)
|
||||
);
|
||||
)
|
||||
);
|
||||
if (updatedVoters.isEmpty()) {
|
||||
return CompletableFuture.completedFuture(
|
||||
RaftUtil.updateVoterResponse(
|
||||
|
@ -195,9 +216,80 @@ public final class UpdateVoterHandler {
|
|||
);
|
||||
}
|
||||
|
||||
leaderState.appendVotersRecord(updatedVoters.get(), currentTimeMs);
|
||||
return storeUpdatedVoters(
|
||||
leaderState,
|
||||
voterKey,
|
||||
inMemoryVoters,
|
||||
updatedVoters.get(),
|
||||
requestListenerName,
|
||||
currentTimeMs
|
||||
);
|
||||
}
|
||||
|
||||
private boolean validVersionRange(
|
||||
KRaftVersion finalizedVersion,
|
||||
UpdateRaftVoterRequestData.KRaftVersionFeature supportedKraftVersions
|
||||
) {
|
||||
return supportedKraftVersions.minSupportedVersion() <= finalizedVersion.featureLevel() &&
|
||||
supportedKraftVersions.maxSupportedVersion() >= finalizedVersion.featureLevel();
|
||||
}
|
||||
|
||||
private Optional<VoterSet> updateVoters(
|
||||
VoterSet voters,
|
||||
KRaftVersion kraftVersion,
|
||||
VoterSet.VoterNode updatedVoter
|
||||
) {
|
||||
return kraftVersion.isReconfigSupported() ?
|
||||
voters.updateVoter(updatedVoter) :
|
||||
voters.updateVoterIgnoringDirectoryId(updatedVoter);
|
||||
}
|
||||
|
||||
private CompletableFuture<UpdateRaftVoterResponseData> storeUpdatedVoters(
|
||||
LeaderState<?> leaderState,
|
||||
ReplicaKey voterKey,
|
||||
Optional<KRaftVersionUpgrade.Voters> inMemoryVoters,
|
||||
VoterSet newVoters,
|
||||
ListenerName requestListenerName,
|
||||
long currentTimeMs
|
||||
) {
|
||||
if (inMemoryVoters.isEmpty()) {
|
||||
// Since the partition support reconfig then just write the update voter set directly to the log
|
||||
leaderState.appendVotersRecord(newVoters, currentTimeMs);
|
||||
} else {
|
||||
// Store the new voters set in the leader state since it cannot be written to the log
|
||||
var successful = leaderState.compareAndSetVolatileVoters(
|
||||
inMemoryVoters.get(),
|
||||
new KRaftVersionUpgrade.Voters(newVoters)
|
||||
);
|
||||
if (successful) {
|
||||
log.info(
|
||||
"Updated in-memory voters from {} to {}",
|
||||
inMemoryVoters.get().voters(),
|
||||
newVoters
|
||||
);
|
||||
} else {
|
||||
log.info(
|
||||
"Unable to update in-memory voters from {} to {}",
|
||||
inMemoryVoters.get().voters(),
|
||||
newVoters
|
||||
);
|
||||
return CompletableFuture.completedFuture(
|
||||
RaftUtil.updateVoterResponse(
|
||||
Errors.REQUEST_TIMED_OUT,
|
||||
requestListenerName,
|
||||
new LeaderAndEpoch(
|
||||
localId,
|
||||
leaderState.epoch()
|
||||
),
|
||||
leaderState.leaderEndpoints()
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
// Reset the check quorum state since the leader received a successful request
|
||||
leaderState.updateCheckQuorumForFollowingVoter(voterKey, currentTimeMs);
|
||||
|
||||
// Reply immediately and don't wait for the change to commit
|
||||
return CompletableFuture.completedFuture(
|
||||
RaftUtil.updateVoterResponse(
|
||||
Errors.NONE,
|
||||
|
@ -210,12 +302,4 @@ public final class UpdateVoterHandler {
|
|||
)
|
||||
);
|
||||
}
|
||||
|
||||
private boolean validVersionRange(
|
||||
KRaftVersion finalizedVersion,
|
||||
UpdateRaftVoterRequestData.KRaftVersionFeature supportedKraftVersions
|
||||
) {
|
||||
return supportedKraftVersions.minSupportedVersion() <= finalizedVersion.featureLevel() &&
|
||||
supportedKraftVersions.maxSupportedVersion() >= finalizedVersion.featureLevel();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -140,4 +140,13 @@ public class FollowerStateTest {
|
|||
assertEquals(leaderId, state.leaderId());
|
||||
assertEquals(leaderEndpoints, state.leaderEndpoints());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testHasUpdatedLeader() {
|
||||
FollowerState state = newFollowerState(Set.of(0, 1, 2));
|
||||
|
||||
assertFalse(state.hasUpdatedLeader());
|
||||
state.setHasUpdatedLeader();
|
||||
assertTrue(state.hasUpdatedLeader());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -17,7 +17,9 @@
|
|||
package org.apache.kafka.raft;
|
||||
|
||||
import org.apache.kafka.common.Node;
|
||||
import org.apache.kafka.common.Uuid;
|
||||
import org.apache.kafka.common.compress.Compression;
|
||||
import org.apache.kafka.common.errors.InvalidUpdateVersionException;
|
||||
import org.apache.kafka.common.feature.SupportedVersionRange;
|
||||
import org.apache.kafka.common.message.ApiVersionsResponseData;
|
||||
import org.apache.kafka.common.message.EndQuorumEpochResponseData;
|
||||
|
@ -46,6 +48,8 @@ import org.apache.kafka.snapshot.SnapshotReader;
|
|||
import org.apache.kafka.snapshot.SnapshotWriterReaderTest;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
import org.junit.jupiter.params.provider.EnumSource;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
import java.net.InetSocketAddress;
|
||||
|
@ -59,17 +63,21 @@ import java.util.Optional;
|
|||
import java.util.OptionalInt;
|
||||
import java.util.OptionalLong;
|
||||
import java.util.concurrent.ThreadLocalRandom;
|
||||
import java.util.stream.IntStream;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import static org.apache.kafka.raft.KafkaRaftClientTest.replicaKey;
|
||||
import static org.apache.kafka.raft.RaftClientTestContext.RaftProtocol;
|
||||
import static org.apache.kafka.snapshot.Snapshots.BOOTSTRAP_SNAPSHOT_ID;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertNotEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertNull;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
public class KafkaRaftClientReconfigTest {
|
||||
private static final int NUMBER_FETCH_TIMEOUTS_IN_UPDATE_PERIOD = 1;
|
||||
|
||||
@Test
|
||||
public void testLeaderWritesBootstrapRecords() throws Exception {
|
||||
|
@ -1695,6 +1703,53 @@ public class KafkaRaftClientReconfigTest {
|
|||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInvalidUpdateVoter() throws Exception {
|
||||
ReplicaKey local = replicaKey(randomReplicaId(), true);
|
||||
ReplicaKey follower = replicaKey(local.id() + 1, true);
|
||||
|
||||
VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower));
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get())
|
||||
.withKip853Rpc(true)
|
||||
.withBootstrapSnapshot(Optional.of(voters))
|
||||
.withUnknownLeader(3)
|
||||
.build();
|
||||
|
||||
context.unattachedToLeader();
|
||||
int epoch = context.currentEpoch();
|
||||
|
||||
// missing directory id
|
||||
context.deliverRequest(
|
||||
context.updateVoterRequest(
|
||||
ReplicaKey.of(follower.id(), Uuid.ZERO_UUID),
|
||||
Feature.KRAFT_VERSION.supportedVersionRange(),
|
||||
Endpoints.empty()
|
||||
)
|
||||
);
|
||||
context.pollUntilResponse();
|
||||
context.assertSentUpdateVoterResponse(
|
||||
Errors.INVALID_REQUEST,
|
||||
OptionalInt.of(local.id()),
|
||||
epoch
|
||||
);
|
||||
|
||||
// missing endpoints
|
||||
context.deliverRequest(
|
||||
context.updateVoterRequest(
|
||||
follower,
|
||||
Feature.KRAFT_VERSION.supportedVersionRange(),
|
||||
Endpoints.empty()
|
||||
)
|
||||
);
|
||||
context.pollUntilResponse();
|
||||
context.assertSentUpdateVoterResponse(
|
||||
Errors.INVALID_REQUEST,
|
||||
OptionalInt.of(local.id()),
|
||||
epoch
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testUpdateVoterOldEpoch() throws Exception {
|
||||
ReplicaKey local = replicaKey(randomReplicaId(), true);
|
||||
|
@ -1834,7 +1889,6 @@ public class KafkaRaftClientReconfigTest {
|
|||
);
|
||||
}
|
||||
|
||||
// KAFKA-16538 is going to allow UpdateVoter RPC when the kraft.version is 0
|
||||
@Test
|
||||
void testUpdateVoterWithKraftVersion0() throws Exception {
|
||||
ReplicaKey local = replicaKey(randomReplicaId(), true);
|
||||
|
@ -1880,7 +1934,7 @@ public class KafkaRaftClientReconfigTest {
|
|||
);
|
||||
context.pollUntilResponse();
|
||||
context.assertSentUpdateVoterResponse(
|
||||
Errors.UNSUPPORTED_VERSION,
|
||||
Errors.NONE,
|
||||
OptionalInt.of(local.id()),
|
||||
epoch
|
||||
);
|
||||
|
@ -2085,8 +2139,8 @@ public class KafkaRaftClientReconfigTest {
|
|||
.withLocalListeners(localListeners)
|
||||
.build();
|
||||
|
||||
// waiting for 3 times the fetch timeout sends an update voter
|
||||
for (int i = 0; i < 3; i++) {
|
||||
// waiting for FETCH requests until the UpdateRaftVoter request is sent
|
||||
for (int i = 0; i < NUMBER_FETCH_TIMEOUTS_IN_UPDATE_PERIOD; i++) {
|
||||
context.time.sleep(context.fetchTimeoutMs - 1);
|
||||
context.pollUntilRequest();
|
||||
RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest();
|
||||
|
@ -2130,6 +2184,444 @@ public class KafkaRaftClientReconfigTest {
|
|||
context.assertFetchRequestData(fetchRequest, epoch, 0L, 0);
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@EnumSource(value = Errors.class, names = {"NONE", "UNSUPPORTED_VERSION"})
|
||||
void testFollowerSendsUpdateVoterWithKraftVersion0(Errors updateVoterError) throws Exception {
|
||||
ReplicaKey local = replicaKey(randomReplicaId(), true);
|
||||
ReplicaKey voter1 = replicaKey(local.id() + 1, true);
|
||||
ReplicaKey voter2 = replicaKey(local.id() + 2, true);
|
||||
|
||||
VoterSet voters = VoterSetTest.voterSet(Stream.of(local, voter1, voter2));
|
||||
int epoch = 4;
|
||||
|
||||
HashMap<ListenerName, InetSocketAddress> listenersMap = new HashMap<>(2);
|
||||
listenersMap.put(
|
||||
VoterSetTest.DEFAULT_LISTENER_NAME,
|
||||
InetSocketAddress.createUnresolved("localhost", 9990 + local.id())
|
||||
);
|
||||
listenersMap.put(
|
||||
ListenerName.normalised("ANOTHER_LISTENER"),
|
||||
InetSocketAddress.createUnresolved("localhost", 8990 + local.id())
|
||||
);
|
||||
Endpoints localListeners = Endpoints.fromInetSocketAddresses(listenersMap);
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get())
|
||||
.withKip853Rpc(true)
|
||||
.withStaticVoters(voters)
|
||||
.withElectedLeader(epoch, voter1.id())
|
||||
.withLocalListeners(localListeners)
|
||||
.build();
|
||||
|
||||
// waiting for FETCH request until the UpdateRaftVoter request is set
|
||||
for (int i = 0; i < NUMBER_FETCH_TIMEOUTS_IN_UPDATE_PERIOD; i++) {
|
||||
context.time.sleep(context.fetchTimeoutMs - 1);
|
||||
context.pollUntilRequest();
|
||||
RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest();
|
||||
context.assertFetchRequestData(fetchRequest, epoch, 0L, 0);
|
||||
|
||||
context.deliverResponse(
|
||||
fetchRequest.correlationId(),
|
||||
fetchRequest.destination(),
|
||||
context.fetchResponse(
|
||||
epoch,
|
||||
voter1.id(),
|
||||
MemoryRecords.EMPTY,
|
||||
0L,
|
||||
Errors.NONE
|
||||
)
|
||||
);
|
||||
// poll kraft to handle the fetch response
|
||||
context.client.poll();
|
||||
}
|
||||
|
||||
context.time.sleep(context.fetchTimeoutMs - 1);
|
||||
context.pollUntilRequest();
|
||||
RaftRequest.Outbound updateRequest = context.assertSentUpdateVoterRequest(
|
||||
local,
|
||||
epoch,
|
||||
Feature.KRAFT_VERSION.supportedVersionRange(),
|
||||
localListeners
|
||||
);
|
||||
context.deliverResponse(
|
||||
updateRequest.correlationId(),
|
||||
updateRequest.destination(),
|
||||
context.updateVoterResponse(
|
||||
updateVoterError,
|
||||
new LeaderAndEpoch(OptionalInt.of(voter1.id()), epoch)
|
||||
)
|
||||
);
|
||||
context.client.poll();
|
||||
|
||||
// after sending an update voter the next requests should be fetch and no update voter
|
||||
for (int i = 0; i < 10; i++) {
|
||||
context.time.sleep(context.fetchTimeoutMs - 1);
|
||||
context.pollUntilRequest();
|
||||
RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest();
|
||||
context.assertFetchRequestData(fetchRequest, epoch, 0L, 0);
|
||||
|
||||
context.deliverResponse(
|
||||
fetchRequest.correlationId(),
|
||||
fetchRequest.destination(),
|
||||
context.fetchResponse(
|
||||
epoch,
|
||||
voter1.id(),
|
||||
MemoryRecords.EMPTY,
|
||||
0L,
|
||||
Errors.NONE
|
||||
)
|
||||
);
|
||||
// poll kraft to handle the fetch response
|
||||
context.client.poll();
|
||||
}
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@EnumSource(value = Errors.class, names = {"NONE", "UNSUPPORTED_VERSION"})
|
||||
void testFollowerSendsUpdateVoterAfterElectionWithKraftVersion0(Errors updateVoterError) throws Exception {
|
||||
ReplicaKey local = replicaKey(randomReplicaId(), true);
|
||||
ReplicaKey voter1 = replicaKey(local.id() + 1, true);
|
||||
ReplicaKey voter2 = replicaKey(local.id() + 2, true);
|
||||
|
||||
VoterSet voters = VoterSetTest.voterSet(Stream.of(local, voter1, voter2));
|
||||
int epoch = 4;
|
||||
|
||||
HashMap<ListenerName, InetSocketAddress> listenersMap = new HashMap<>(2);
|
||||
listenersMap.put(
|
||||
VoterSetTest.DEFAULT_LISTENER_NAME,
|
||||
InetSocketAddress.createUnresolved("localhost", 9990 + local.id())
|
||||
);
|
||||
listenersMap.put(
|
||||
ListenerName.normalised("ANOTHER_LISTENER"),
|
||||
InetSocketAddress.createUnresolved("localhost", 8990 + local.id())
|
||||
);
|
||||
Endpoints localListeners = Endpoints.fromInetSocketAddresses(listenersMap);
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get())
|
||||
.withKip853Rpc(true)
|
||||
.withStaticVoters(voters)
|
||||
.withElectedLeader(epoch, voter1.id())
|
||||
.withLocalListeners(localListeners)
|
||||
.build();
|
||||
|
||||
// waiting for FETCH request until the UpdateRaftVoter request is set
|
||||
for (int i = 0; i < NUMBER_FETCH_TIMEOUTS_IN_UPDATE_PERIOD; i++) {
|
||||
context.time.sleep(context.fetchTimeoutMs - 1);
|
||||
context.pollUntilRequest();
|
||||
RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest();
|
||||
context.assertFetchRequestData(fetchRequest, epoch, 0L, 0);
|
||||
|
||||
context.deliverResponse(
|
||||
fetchRequest.correlationId(),
|
||||
fetchRequest.destination(),
|
||||
context.fetchResponse(
|
||||
epoch,
|
||||
voter1.id(),
|
||||
MemoryRecords.EMPTY,
|
||||
0L,
|
||||
Errors.NONE
|
||||
)
|
||||
);
|
||||
// poll kraft to handle the fetch response
|
||||
context.client.poll();
|
||||
}
|
||||
|
||||
context.time.sleep(context.fetchTimeoutMs - 1);
|
||||
context.pollUntilRequest();
|
||||
RaftRequest.Outbound updateRequest = context.assertSentUpdateVoterRequest(
|
||||
local,
|
||||
epoch,
|
||||
Feature.KRAFT_VERSION.supportedVersionRange(),
|
||||
localListeners
|
||||
);
|
||||
context.deliverResponse(
|
||||
updateRequest.correlationId(),
|
||||
updateRequest.destination(),
|
||||
context.updateVoterResponse(
|
||||
updateVoterError,
|
||||
new LeaderAndEpoch(OptionalInt.of(voter1.id()), epoch)
|
||||
)
|
||||
);
|
||||
|
||||
context.pollUntilRequest();
|
||||
RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest();
|
||||
context.assertFetchRequestData(fetchRequest, epoch, 0L, 0);
|
||||
|
||||
// Election a new leader causes the replica to resend update voter request
|
||||
int newEpoch = epoch + 1;
|
||||
context.deliverRequest(context.beginEpochRequest(newEpoch, voter1.id()));
|
||||
context.pollUntilResponse();
|
||||
|
||||
// waiting for FETCH request until the UpdateRaftVoter request is set
|
||||
for (int i = 0; i < NUMBER_FETCH_TIMEOUTS_IN_UPDATE_PERIOD; i++) {
|
||||
context.time.sleep(context.fetchTimeoutMs - 1);
|
||||
context.pollUntilRequest();
|
||||
fetchRequest = context.assertSentFetchRequest();
|
||||
context.assertFetchRequestData(fetchRequest, newEpoch, 0L, 0);
|
||||
|
||||
context.deliverResponse(
|
||||
fetchRequest.correlationId(),
|
||||
fetchRequest.destination(),
|
||||
context.fetchResponse(
|
||||
newEpoch,
|
||||
voter1.id(),
|
||||
MemoryRecords.EMPTY,
|
||||
0L,
|
||||
Errors.NONE
|
||||
)
|
||||
);
|
||||
// poll kraft to handle the fetch response
|
||||
context.client.poll();
|
||||
}
|
||||
|
||||
context.time.sleep(context.fetchTimeoutMs - 1);
|
||||
context.pollUntilRequest();
|
||||
updateRequest = context.assertSentUpdateVoterRequest(
|
||||
local,
|
||||
newEpoch,
|
||||
Feature.KRAFT_VERSION.supportedVersionRange(),
|
||||
localListeners
|
||||
);
|
||||
context.deliverResponse(
|
||||
updateRequest.correlationId(),
|
||||
updateRequest.destination(),
|
||||
context.updateVoterResponse(
|
||||
Errors.NONE,
|
||||
new LeaderAndEpoch(OptionalInt.of(voter1.id()), newEpoch)
|
||||
)
|
||||
);
|
||||
|
||||
context.pollUntilRequest();
|
||||
fetchRequest = context.assertSentFetchRequest();
|
||||
context.assertFetchRequestData(fetchRequest, newEpoch, 0L, 0);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testKRaftUpgradeVersion() throws Exception {
|
||||
var local = replicaKey(randomReplicaId(), true);
|
||||
var voter1 = replicaKey(local.id() + 1, true);
|
||||
var voter2 = replicaKey(local.id() + 2, true);
|
||||
|
||||
VoterSet startingVoters = VoterSetTest.voterSet(
|
||||
VoterSetTest.voterMap(IntStream.of(local.id(), voter1.id(), voter2.id()), false)
|
||||
);
|
||||
|
||||
var context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get())
|
||||
.withRaftProtocol(RaftProtocol.KIP_853_PROTOCOL)
|
||||
.withStartingVoters(startingVoters, KRaftVersion.KRAFT_VERSION_0)
|
||||
.build();
|
||||
|
||||
context.unattachedToLeader();
|
||||
var epoch = context.currentEpoch();
|
||||
|
||||
// Establish a HWM and fence previous leaders
|
||||
for (var voter : List.of(voter1, voter2)) {
|
||||
context.deliverRequest(
|
||||
context.fetchRequest(epoch, voter, context.log.endOffset().offset(), epoch, 0)
|
||||
);
|
||||
context.pollUntilResponse();
|
||||
context.assertSentFetchPartitionResponse(Errors.NONE, epoch, OptionalInt.of(local.id()));
|
||||
}
|
||||
|
||||
// Update voters so that they supports kraft version 1
|
||||
for (var voter : List.of(voter1, voter2)) {
|
||||
context.deliverRequest(
|
||||
context.updateVoterRequest(
|
||||
voter,
|
||||
Feature.KRAFT_VERSION.supportedVersionRange(),
|
||||
startingVoters.listeners(voter.id())
|
||||
)
|
||||
);
|
||||
context.pollUntilResponse();
|
||||
context.assertSentUpdateVoterResponse(
|
||||
Errors.NONE,
|
||||
OptionalInt.of(local.id()),
|
||||
epoch
|
||||
);
|
||||
}
|
||||
|
||||
context.client.upgradeKRaftVersion(epoch, KRaftVersion.KRAFT_VERSION_1, false);
|
||||
assertEquals(KRaftVersion.KRAFT_VERSION_1, context.client.kraftVersion());
|
||||
|
||||
var localLogEndOffset = context.log.endOffset().offset();
|
||||
context.client.poll();
|
||||
|
||||
// check if leader writes 2 control records to the log;
|
||||
// one for the kraft version and one for the voter set
|
||||
var updatedVoters = VoterSetTest.voterSet(Stream.of(local, voter1, voter2));
|
||||
var records = context.log.read(localLogEndOffset, Isolation.UNCOMMITTED).records;
|
||||
var batch = records.batches().iterator().next();
|
||||
assertTrue(batch.isControlBatch());
|
||||
var recordsIterator = batch.iterator();
|
||||
var controlRecord = recordsIterator.next();
|
||||
verifyKRaftVersionRecord(
|
||||
KRaftVersion.KRAFT_VERSION_1.featureLevel(),
|
||||
controlRecord.key(),
|
||||
controlRecord.value()
|
||||
);
|
||||
controlRecord = recordsIterator.next();
|
||||
verifyVotersRecord(updatedVoters, controlRecord.key(), controlRecord.value());
|
||||
}
|
||||
|
||||
@Test
|
||||
void testUpdateVoterAfterKRaftVersionUpgrade() throws Exception {
|
||||
var local = replicaKey(randomReplicaId(), true);
|
||||
var voter1 = replicaKey(local.id() + 1, true);
|
||||
var voter2 = replicaKey(local.id() + 2, true);
|
||||
|
||||
VoterSet startingVoters = VoterSetTest.voterSet(
|
||||
VoterSetTest.voterMap(IntStream.of(local.id(), voter1.id(), voter2.id()), false)
|
||||
);
|
||||
|
||||
var context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get())
|
||||
.withRaftProtocol(RaftProtocol.KIP_853_PROTOCOL)
|
||||
.withStartingVoters(startingVoters, KRaftVersion.KRAFT_VERSION_0)
|
||||
.build();
|
||||
|
||||
context.unattachedToLeader();
|
||||
var epoch = context.currentEpoch();
|
||||
|
||||
// Establish a HWM and fence previous leaders
|
||||
for (var voter : List.of(voter1, voter2)) {
|
||||
context.deliverRequest(
|
||||
context.fetchRequest(epoch, voter, context.log.endOffset().offset(), epoch, 0)
|
||||
);
|
||||
context.pollUntilResponse();
|
||||
context.assertSentFetchPartitionResponse(Errors.NONE, epoch, OptionalInt.of(local.id()));
|
||||
}
|
||||
|
||||
// Update voters so that they supports kraft version 1
|
||||
for (var voter : List.of(voter1, voter2)) {
|
||||
context.deliverRequest(
|
||||
context.updateVoterRequest(
|
||||
voter,
|
||||
Feature.KRAFT_VERSION.supportedVersionRange(),
|
||||
startingVoters.listeners(voter.id())
|
||||
)
|
||||
);
|
||||
context.pollUntilResponse();
|
||||
context.assertSentUpdateVoterResponse(
|
||||
Errors.NONE,
|
||||
OptionalInt.of(local.id()),
|
||||
epoch
|
||||
);
|
||||
}
|
||||
|
||||
context.client.upgradeKRaftVersion(epoch, KRaftVersion.KRAFT_VERSION_1, false);
|
||||
assertEquals(KRaftVersion.KRAFT_VERSION_1, context.client.kraftVersion());
|
||||
|
||||
// Push the control records to the log
|
||||
context.client.poll();
|
||||
// Advance the HWM to the LEO
|
||||
for (var voter : List.of(voter1, voter2)) {
|
||||
context.deliverRequest(
|
||||
context.fetchRequest(epoch, voter, context.log.endOffset().offset(), epoch, 0)
|
||||
);
|
||||
context.pollUntilResponse();
|
||||
context.assertSentFetchPartitionResponse(Errors.NONE, epoch, OptionalInt.of(local.id()));
|
||||
}
|
||||
|
||||
// Check that it can still handle update voter request after upgrade
|
||||
Endpoints newVoter1Listeners = Endpoints.fromInetSocketAddresses(
|
||||
Map.of(
|
||||
// first entry
|
||||
context.channel.listenerName(),
|
||||
InetSocketAddress.createUnresolved(
|
||||
"localhost",
|
||||
9990 + voter1.id()
|
||||
),
|
||||
// second entry
|
||||
ListenerName.normalised("ANOTHER_LISTENER"),
|
||||
InetSocketAddress.createUnresolved(
|
||||
"localhost",
|
||||
8990 + voter1.id()
|
||||
)
|
||||
)
|
||||
);
|
||||
context.deliverRequest(
|
||||
context.updateVoterRequest(
|
||||
voter1,
|
||||
Feature.KRAFT_VERSION.supportedVersionRange(),
|
||||
newVoter1Listeners
|
||||
)
|
||||
);
|
||||
context.pollUntilResponse();
|
||||
context.assertSentUpdateVoterResponse(
|
||||
Errors.NONE,
|
||||
OptionalInt.of(local.id()),
|
||||
epoch
|
||||
);
|
||||
|
||||
// Push the control records to the log
|
||||
var localLogEndOffset = context.log.endOffset().offset();
|
||||
context.client.poll();
|
||||
|
||||
// check that the leader wrote voters control record to the log;
|
||||
var records = context.log.read(localLogEndOffset, Isolation.UNCOMMITTED).records;
|
||||
var batch = records.batches().iterator().next();
|
||||
assertTrue(batch.isControlBatch());
|
||||
var recordsIterator = batch.iterator();
|
||||
var controlRecord = recordsIterator.next();
|
||||
assertEquals(ControlRecordType.KRAFT_VOTERS, ControlRecordType.parse(controlRecord.key()));
|
||||
ControlRecordUtils.deserializeVotersRecord(controlRecord.value());
|
||||
}
|
||||
|
||||
@Test
|
||||
void testInvalidKRaftUpgradeVersion() throws Exception {
|
||||
var local = replicaKey(randomReplicaId(), true);
|
||||
var voter1 = replicaKey(local.id() + 1, true);
|
||||
var voter2 = replicaKey(local.id() + 2, true);
|
||||
|
||||
VoterSet startingVoters = VoterSetTest.voterSet(
|
||||
VoterSetTest.voterMap(IntStream.of(local.id(), voter1.id(), voter2.id()), false)
|
||||
);
|
||||
|
||||
var context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get())
|
||||
.withRaftProtocol(RaftProtocol.KIP_853_PROTOCOL)
|
||||
.withStartingVoters(startingVoters, KRaftVersion.KRAFT_VERSION_0)
|
||||
.build();
|
||||
|
||||
context.unattachedToLeader();
|
||||
var epoch = context.currentEpoch();
|
||||
|
||||
// Upgrade not allowed since none of the remote voters support the new version
|
||||
assertEquals(KRaftVersion.KRAFT_VERSION_0, context.client.kraftVersion());
|
||||
assertThrows(
|
||||
InvalidUpdateVersionException.class,
|
||||
() -> context.client.upgradeKRaftVersion(epoch, KRaftVersion.KRAFT_VERSION_1, false)
|
||||
);
|
||||
|
||||
// Establish a HWM and fence previous leaders
|
||||
for (var voter : List.of(voter1, voter2)) {
|
||||
context.deliverRequest(
|
||||
context.fetchRequest(epoch, voter, context.log.endOffset().offset(), epoch, 0)
|
||||
);
|
||||
context.pollUntilResponse();
|
||||
context.assertSentFetchPartitionResponse(Errors.NONE, epoch, OptionalInt.of(local.id()));
|
||||
}
|
||||
|
||||
// Update only one of the voters so that they supports kraft version 1
|
||||
context.deliverRequest(
|
||||
context.updateVoterRequest(
|
||||
voter1,
|
||||
Feature.KRAFT_VERSION.supportedVersionRange(),
|
||||
startingVoters.listeners(voter1.id())
|
||||
)
|
||||
);
|
||||
context.pollUntilResponse();
|
||||
context.assertSentUpdateVoterResponse(
|
||||
Errors.NONE,
|
||||
OptionalInt.of(local.id()),
|
||||
epoch
|
||||
);
|
||||
|
||||
// Upgrade not allowed since one of the voters doesn't support the new version
|
||||
assertEquals(KRaftVersion.KRAFT_VERSION_0, context.client.kraftVersion());
|
||||
assertThrows(
|
||||
InvalidUpdateVersionException.class,
|
||||
() -> context.client.upgradeKRaftVersion(epoch, KRaftVersion.KRAFT_VERSION_1, false)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testFollowerSendsUpdateVoterWhenDifferent() throws Exception {
|
||||
ReplicaKey local = replicaKey(randomReplicaId(), true);
|
||||
|
@ -2145,8 +2637,8 @@ public class KafkaRaftClientReconfigTest {
|
|||
.withElectedLeader(epoch, voter1.id())
|
||||
.build();
|
||||
|
||||
// waiting for 3 times the fetch timeout sends an update voter
|
||||
for (int i = 0; i < 3; i++) {
|
||||
// waiting for FETCH request until the UpdateRaftVoter request is set
|
||||
for (int i = 0; i < NUMBER_FETCH_TIMEOUTS_IN_UPDATE_PERIOD; i++) {
|
||||
context.time.sleep(context.fetchTimeoutMs - 1);
|
||||
context.pollUntilRequest();
|
||||
RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest();
|
||||
|
@ -2179,6 +2671,86 @@ public class KafkaRaftClientReconfigTest {
|
|||
assertNotEquals(OptionalLong.of(0L), context.messageQueue.lastPollTimeoutMs());
|
||||
}
|
||||
|
||||
@Test
|
||||
void testFollowerSendsUpdateVoterIfPendingFetchDuringTimeout() throws Exception {
|
||||
ReplicaKey local = replicaKey(randomReplicaId(), true);
|
||||
ReplicaKey voter1 = replicaKey(local.id() + 1, true);
|
||||
ReplicaKey voter2 = replicaKey(local.id() + 2, true);
|
||||
|
||||
VoterSet voters = VoterSetTest.voterSet(Stream.of(local, voter1, voter2));
|
||||
int epoch = 4;
|
||||
|
||||
HashMap<ListenerName, InetSocketAddress> listenersMap = new HashMap<>(2);
|
||||
listenersMap.put(
|
||||
VoterSetTest.DEFAULT_LISTENER_NAME,
|
||||
InetSocketAddress.createUnresolved("localhost", 9990 + local.id())
|
||||
);
|
||||
listenersMap.put(
|
||||
ListenerName.normalised("ANOTHER_LISTENER"),
|
||||
InetSocketAddress.createUnresolved("localhost", 8990 + local.id())
|
||||
);
|
||||
Endpoints localListeners = Endpoints.fromInetSocketAddresses(listenersMap);
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get())
|
||||
.withKip853Rpc(true)
|
||||
.withBootstrapSnapshot(Optional.of(voters))
|
||||
.withElectedLeader(epoch, voter1.id())
|
||||
.withLocalListeners(localListeners)
|
||||
.build();
|
||||
|
||||
// waiting up to the last FETCH request before the UpdateRaftVoter request is set
|
||||
for (int i = 0; i < NUMBER_FETCH_TIMEOUTS_IN_UPDATE_PERIOD; i++) {
|
||||
context.time.sleep(context.fetchTimeoutMs - 1);
|
||||
context.pollUntilRequest();
|
||||
RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest();
|
||||
context.assertFetchRequestData(fetchRequest, epoch, 0L, 0);
|
||||
|
||||
context.deliverResponse(
|
||||
fetchRequest.correlationId(),
|
||||
fetchRequest.destination(),
|
||||
context.fetchResponse(
|
||||
epoch,
|
||||
voter1.id(),
|
||||
MemoryRecords.EMPTY,
|
||||
0L,
|
||||
Errors.NONE
|
||||
)
|
||||
);
|
||||
// poll kraft to handle the fetch response
|
||||
context.client.poll();
|
||||
}
|
||||
|
||||
// expect one last FETCH request
|
||||
context.pollUntilRequest();
|
||||
RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest();
|
||||
context.assertFetchRequestData(fetchRequest, epoch, 0L, 0);
|
||||
|
||||
// don't send a response but increase the time
|
||||
context.time.sleep(context.requestTimeoutMs() - 1);
|
||||
context.client.poll();
|
||||
assertFalse(context.channel.hasSentRequests());
|
||||
|
||||
// expect an update voter request after the FETCH rpc completes
|
||||
context.deliverResponse(
|
||||
fetchRequest.correlationId(),
|
||||
fetchRequest.destination(),
|
||||
context.fetchResponse(
|
||||
epoch,
|
||||
voter1.id(),
|
||||
MemoryRecords.EMPTY,
|
||||
0L,
|
||||
Errors.NONE
|
||||
)
|
||||
);
|
||||
context.pollUntilRequest();
|
||||
context.assertSentUpdateVoterRequest(
|
||||
local,
|
||||
epoch,
|
||||
Feature.KRAFT_VERSION.supportedVersionRange(),
|
||||
localListeners
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testUpdateVoterResponseCausesEpochChange() throws Exception {
|
||||
ReplicaKey local = replicaKey(randomReplicaId(), true);
|
||||
|
@ -2206,8 +2778,8 @@ public class KafkaRaftClientReconfigTest {
|
|||
.withLocalListeners(localListeners)
|
||||
.build();
|
||||
|
||||
// waiting for 3 times the fetch timeout sends an update voter
|
||||
for (int i = 0; i < 3; i++) {
|
||||
// waiting for FETCH request until the UpdateRaftVoter request is set
|
||||
for (int i = 0; i < NUMBER_FETCH_TIMEOUTS_IN_UPDATE_PERIOD; i++) {
|
||||
context.time.sleep(context.fetchTimeoutMs - 1);
|
||||
context.pollUntilRequest();
|
||||
RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest();
|
||||
|
|
|
@ -17,10 +17,13 @@
|
|||
package org.apache.kafka.raft;
|
||||
|
||||
import org.apache.kafka.common.Uuid;
|
||||
import org.apache.kafka.common.errors.InvalidUpdateVersionException;
|
||||
import org.apache.kafka.common.metrics.Metrics;
|
||||
import org.apache.kafka.common.utils.LogContext;
|
||||
import org.apache.kafka.common.utils.MockTime;
|
||||
import org.apache.kafka.raft.errors.NotLeaderException;
|
||||
import org.apache.kafka.raft.internals.BatchAccumulator;
|
||||
import org.apache.kafka.raft.internals.KRaftVersionUpgrade;
|
||||
import org.apache.kafka.raft.internals.KafkaRaftMetrics;
|
||||
import org.apache.kafka.server.common.KRaftVersion;
|
||||
|
||||
|
@ -46,23 +49,36 @@ import static org.junit.jupiter.api.Assertions.assertThrows;
|
|||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
public class LeaderStateTest {
|
||||
private final ReplicaKey localReplicaKey = ReplicaKey.of(0, Uuid.randomUuid());
|
||||
private final VoterSet.VoterNode localVoterNode = VoterSetTest.voterNode(ReplicaKey.of(0, Uuid.randomUuid()));
|
||||
private final int epoch = 5;
|
||||
private final LogContext logContext = new LogContext();
|
||||
private final BatchAccumulator<?> accumulator = Mockito.mock(BatchAccumulator.class);
|
||||
private final MockTime time = new MockTime();
|
||||
private final int fetchTimeoutMs = 2000;
|
||||
private final int checkQuorumTimeoutMs = (int) (fetchTimeoutMs * CHECK_QUORUM_TIMEOUT_FACTOR);
|
||||
private final int beginQuorumEpochTimeoutMs = fetchTimeoutMs / 2;
|
||||
private final KRaftVersion kraftVersion = KRaftVersion.KRAFT_VERSION_1;
|
||||
|
||||
private LeaderState<?> newLeaderState(
|
||||
VoterSet voters,
|
||||
long epochStartOffset
|
||||
long epochStartOffset,
|
||||
KRaftVersion kraftVersion
|
||||
) {
|
||||
return newLeaderState(
|
||||
voters,
|
||||
epochStartOffset,
|
||||
kraftVersion,
|
||||
Mockito.mock(BatchAccumulator.class)
|
||||
);
|
||||
}
|
||||
|
||||
private LeaderState<?> newLeaderState(
|
||||
VoterSet voters,
|
||||
long epochStartOffset,
|
||||
KRaftVersion kraftVersion,
|
||||
BatchAccumulator<?> accumulator
|
||||
) {
|
||||
return new LeaderState<>(
|
||||
time,
|
||||
localReplicaKey,
|
||||
localVoterNode,
|
||||
epoch,
|
||||
epochStartOffset,
|
||||
voters,
|
||||
|
@ -70,7 +86,6 @@ public class LeaderStateTest {
|
|||
kraftVersion,
|
||||
voters.voterIds(),
|
||||
accumulator,
|
||||
voters.listeners(localReplicaKey.id()),
|
||||
fetchTimeoutMs,
|
||||
logContext,
|
||||
new KafkaRaftMetrics(new Metrics(), "raft")
|
||||
|
@ -80,11 +95,13 @@ public class LeaderStateTest {
|
|||
private VoterSet localWithRemoteVoterSet(IntStream remoteIds, boolean withDirectoryId) {
|
||||
Map<Integer, VoterSet.VoterNode> voters = VoterSetTest.voterMap(remoteIds, withDirectoryId);
|
||||
if (withDirectoryId) {
|
||||
voters.put(localReplicaKey.id(), VoterSetTest.voterNode(localReplicaKey));
|
||||
voters.put(localVoterNode.voterKey().id(), localVoterNode);
|
||||
} else {
|
||||
voters.put(
|
||||
localReplicaKey.id(),
|
||||
VoterSetTest.voterNode(ReplicaKey.of(localReplicaKey.id(), ReplicaKey.NO_DIRECTORY_ID))
|
||||
localVoterNode.voterKey().id(),
|
||||
VoterSetTest.voterNode(
|
||||
ReplicaKey.of(localVoterNode.voterKey().id(), ReplicaKey.NO_DIRECTORY_ID)
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -93,8 +110,8 @@ public class LeaderStateTest {
|
|||
|
||||
private VoterSet localWithRemoteVoterSet(Stream<ReplicaKey> remoteReplicaKeys, boolean withDirectoryId) {
|
||||
ReplicaKey actualLocalVoter = withDirectoryId ?
|
||||
localReplicaKey :
|
||||
ReplicaKey.of(localReplicaKey.id(), ReplicaKey.NO_DIRECTORY_ID);
|
||||
localVoterNode.voterKey() :
|
||||
ReplicaKey.of(localVoterNode.voterKey().id(), ReplicaKey.NO_DIRECTORY_ID);
|
||||
|
||||
return VoterSetTest.voterSet(
|
||||
Stream.concat(Stream.of(actualLocalVoter), remoteReplicaKeys)
|
||||
|
@ -103,20 +120,23 @@ public class LeaderStateTest {
|
|||
|
||||
@Test
|
||||
public void testRequireNonNullAccumulator() {
|
||||
VoterSet voterSet = VoterSetTest.voterSet(Stream.of(localReplicaKey));
|
||||
VoterSet voterSet = VoterSetTest.voterSet(Stream.of(localVoterNode.voterKey()));
|
||||
assertThrows(
|
||||
NullPointerException.class,
|
||||
() -> new LeaderState<>(
|
||||
new MockTime(),
|
||||
localReplicaKey,
|
||||
voterSet.voterNodes()
|
||||
.stream()
|
||||
.filter(node -> node.voterKey().equals(localVoterNode.voterKey()))
|
||||
.findFirst()
|
||||
.get(),
|
||||
epoch,
|
||||
0,
|
||||
voterSet,
|
||||
OptionalLong.of(0),
|
||||
kraftVersion,
|
||||
KRaftVersion.KRAFT_VERSION_1,
|
||||
Set.of(),
|
||||
null,
|
||||
Endpoints.empty(),
|
||||
fetchTimeoutMs,
|
||||
logContext,
|
||||
new KafkaRaftMetrics(new Metrics(), "raft")
|
||||
|
@ -131,7 +151,8 @@ public class LeaderStateTest {
|
|||
ReplicaKey node2 = replicaKey(2, withDirectoryId);
|
||||
LeaderState<?> state = newLeaderState(
|
||||
localWithRemoteVoterSet(Stream.of(node1, node2), withDirectoryId),
|
||||
0L
|
||||
0L,
|
||||
KRaftVersion.KRAFT_VERSION_1
|
||||
);
|
||||
assertEquals(Set.of(node1, node2), state.nonAcknowledgingVoters());
|
||||
state.addAcknowledgementFrom(node1.id());
|
||||
|
@ -144,16 +165,21 @@ public class LeaderStateTest {
|
|||
public void testNonFollowerAcknowledgement() {
|
||||
int nonVoterId = 1;
|
||||
LeaderState<?> state = newLeaderState(
|
||||
VoterSetTest.voterSet(Stream.of(localReplicaKey)),
|
||||
0L
|
||||
VoterSetTest.voterSet(Stream.of(localVoterNode.voterKey())),
|
||||
0L,
|
||||
KRaftVersion.KRAFT_VERSION_1
|
||||
);
|
||||
assertThrows(IllegalArgumentException.class, () -> state.addAcknowledgementFrom(nonVoterId));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUpdateHighWatermarkQuorumSizeOne() {
|
||||
VoterSet voters = VoterSetTest.voterSet(Stream.of(localReplicaKey));
|
||||
LeaderState<?> state = newLeaderState(voters, 15L);
|
||||
VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterNode.voterKey()));
|
||||
LeaderState<?> state = newLeaderState(
|
||||
voters,
|
||||
15L,
|
||||
KRaftVersion.KRAFT_VERSION_1
|
||||
);
|
||||
|
||||
assertEquals(Optional.empty(), state.highWatermark());
|
||||
assertFalse(state.updateLocalState(new LogOffsetMetadata(15L), voters));
|
||||
|
@ -167,8 +193,12 @@ public class LeaderStateTest {
|
|||
|
||||
@Test
|
||||
public void testNonMonotonicLocalEndOffsetUpdate() {
|
||||
VoterSet voters = VoterSetTest.voterSet(Stream.of(localReplicaKey));
|
||||
LeaderState<?> state = newLeaderState(voters, 15L);
|
||||
VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterNode.voterKey()));
|
||||
LeaderState<?> state = newLeaderState(
|
||||
voters,
|
||||
15L,
|
||||
KRaftVersion.KRAFT_VERSION_1
|
||||
);
|
||||
|
||||
assertEquals(Optional.empty(), state.highWatermark());
|
||||
assertTrue(state.updateLocalState(new LogOffsetMetadata(16L), voters));
|
||||
|
@ -181,8 +211,12 @@ public class LeaderStateTest {
|
|||
|
||||
@Test
|
||||
public void testIdempotentEndOffsetUpdate() {
|
||||
VoterSet voters = VoterSetTest.voterSet(Stream.of(localReplicaKey));
|
||||
LeaderState<?> state = newLeaderState(voters, 15L);
|
||||
VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterNode.voterKey()));
|
||||
LeaderState<?> state = newLeaderState(
|
||||
voters,
|
||||
15L,
|
||||
KRaftVersion.KRAFT_VERSION_1
|
||||
);
|
||||
assertEquals(Optional.empty(), state.highWatermark());
|
||||
assertTrue(state.updateLocalState(new LogOffsetMetadata(16L), voters));
|
||||
assertFalse(state.updateLocalState(new LogOffsetMetadata(16L), voters));
|
||||
|
@ -191,8 +225,12 @@ public class LeaderStateTest {
|
|||
|
||||
@Test
|
||||
public void testUpdateHighWatermarkMetadata() {
|
||||
VoterSet voters = VoterSetTest.voterSet(Stream.of(localReplicaKey));
|
||||
LeaderState<?> state = newLeaderState(voters, 15L);
|
||||
VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterNode.voterKey()));
|
||||
LeaderState<?> state = newLeaderState(
|
||||
voters,
|
||||
15L,
|
||||
KRaftVersion.KRAFT_VERSION_1
|
||||
);
|
||||
assertEquals(Optional.empty(), state.highWatermark());
|
||||
|
||||
LogOffsetMetadata initialHw = new LogOffsetMetadata(16L, Optional.of(new MockOffsetMetadata("bar")));
|
||||
|
@ -210,7 +248,11 @@ public class LeaderStateTest {
|
|||
ReplicaKey otherNodeKey = replicaKey(1, withDirectoryId);
|
||||
|
||||
VoterSet voters = localWithRemoteVoterSet(Stream.of(otherNodeKey), withDirectoryId);
|
||||
LeaderState<?> state = newLeaderState(voters, 10L);
|
||||
LeaderState<?> state = newLeaderState(
|
||||
voters,
|
||||
10L,
|
||||
KRaftVersion.KRAFT_VERSION_1
|
||||
);
|
||||
|
||||
assertFalse(state.updateLocalState(new LogOffsetMetadata(13L), voters));
|
||||
assertEquals(Set.of(otherNodeKey), state.nonAcknowledgingVoters());
|
||||
|
@ -231,7 +273,11 @@ public class LeaderStateTest {
|
|||
ReplicaKey nodeKey2 = replicaKey(2, withDirectoryId);
|
||||
|
||||
VoterSet voters = localWithRemoteVoterSet(Stream.of(nodeKey1, nodeKey2), withDirectoryId);
|
||||
LeaderState<?> state = newLeaderState(voters, 10L);
|
||||
LeaderState<?> state = newLeaderState(
|
||||
voters,
|
||||
10L,
|
||||
KRaftVersion.KRAFT_VERSION_1
|
||||
);
|
||||
|
||||
assertFalse(state.updateLocalState(new LogOffsetMetadata(15L), voters));
|
||||
assertEquals(Set.of(nodeKey1, nodeKey2), state.nonAcknowledgingVoters());
|
||||
|
@ -258,7 +304,11 @@ public class LeaderStateTest {
|
|||
ReplicaKey nodeKey2 = ReplicaKey.of(2, Uuid.randomUuid());
|
||||
|
||||
VoterSet originalVoters = localWithRemoteVoterSet(Stream.of(nodeKey1), true);
|
||||
LeaderState<?> state = newLeaderState(originalVoters, 5L);
|
||||
LeaderState<?> state = newLeaderState(
|
||||
originalVoters,
|
||||
5L,
|
||||
KRaftVersion.KRAFT_VERSION_1
|
||||
);
|
||||
|
||||
assertFalse(state.updateLocalState(new LogOffsetMetadata(15L), originalVoters));
|
||||
assertTrue(state.updateReplicaState(nodeKey1, 0, new LogOffsetMetadata(10L)));
|
||||
|
@ -288,7 +338,11 @@ public class LeaderStateTest {
|
|||
|
||||
// start with three voters with HW at 15L
|
||||
VoterSet originalVoters = localWithRemoteVoterSet(Stream.of(nodeKey1, nodeKey2), true);
|
||||
LeaderState<?> state = newLeaderState(originalVoters, 5L);
|
||||
LeaderState<?> state = newLeaderState(
|
||||
originalVoters,
|
||||
5L,
|
||||
KRaftVersion.KRAFT_VERSION_1
|
||||
);
|
||||
|
||||
assertFalse(state.updateLocalState(new LogOffsetMetadata(15L), originalVoters));
|
||||
assertTrue(state.updateReplicaState(nodeKey1, 0, new LogOffsetMetadata(15L)));
|
||||
|
@ -322,7 +376,11 @@ public class LeaderStateTest {
|
|||
ReplicaKey nodeKey2 = ReplicaKey.of(2, Uuid.randomUuid());
|
||||
|
||||
VoterSet originalVoters = localWithRemoteVoterSet(Stream.of(nodeKey1, nodeKey2), true);
|
||||
LeaderState<?> state = newLeaderState(originalVoters, 10L);
|
||||
LeaderState<?> state = newLeaderState(
|
||||
originalVoters,
|
||||
10L,
|
||||
KRaftVersion.KRAFT_VERSION_1
|
||||
);
|
||||
|
||||
assertFalse(state.updateLocalState(new LogOffsetMetadata(15L), originalVoters));
|
||||
assertTrue(state.updateReplicaState(nodeKey1, 0, new LogOffsetMetadata(15L)));
|
||||
|
@ -355,7 +413,11 @@ public class LeaderStateTest {
|
|||
ReplicaKey nodeKey2 = ReplicaKey.of(2, Uuid.randomUuid());
|
||||
|
||||
VoterSet originalVoters = localWithRemoteVoterSet(Stream.of(nodeKey1, nodeKey2), true);
|
||||
LeaderState<?> state = newLeaderState(originalVoters, 10L);
|
||||
LeaderState<?> state = newLeaderState(
|
||||
originalVoters,
|
||||
10L,
|
||||
KRaftVersion.KRAFT_VERSION_1
|
||||
);
|
||||
|
||||
assertFalse(state.updateLocalState(new LogOffsetMetadata(15L), originalVoters));
|
||||
assertTrue(state.updateReplicaState(nodeKey1, 0, new LogOffsetMetadata(15L)));
|
||||
|
@ -363,7 +425,7 @@ public class LeaderStateTest {
|
|||
assertEquals(Optional.of(new LogOffsetMetadata(15L)), state.highWatermark());
|
||||
|
||||
// removing leader should not decrement HW to 10L
|
||||
VoterSet votersWithoutLeader = originalVoters.removeVoter(localReplicaKey).get();
|
||||
VoterSet votersWithoutLeader = originalVoters.removeVoter(localVoterNode.voterKey()).get();
|
||||
assertFalse(state.updateLocalState(new LogOffsetMetadata(17L), votersWithoutLeader));
|
||||
assertEquals(Optional.of(new LogOffsetMetadata(15L)), state.highWatermark());
|
||||
|
||||
|
@ -389,7 +451,11 @@ public class LeaderStateTest {
|
|||
ReplicaKey nodeKey1 = replicaKey(1, withDirectoryId);
|
||||
|
||||
VoterSet voters = localWithRemoteVoterSet(Stream.of(nodeKey1), withDirectoryId);
|
||||
LeaderState<?> state = newLeaderState(voters, 0L);
|
||||
LeaderState<?> state = newLeaderState(
|
||||
voters,
|
||||
0L,
|
||||
KRaftVersion.KRAFT_VERSION_1
|
||||
);
|
||||
|
||||
state.updateLocalState(new LogOffsetMetadata(10L), voters);
|
||||
state.updateReplicaState(nodeKey1, time.milliseconds(), new LogOffsetMetadata(10L));
|
||||
|
@ -410,7 +476,11 @@ public class LeaderStateTest {
|
|||
long leaderEndOffset = 15L;
|
||||
|
||||
VoterSet voters = localWithRemoteVoterSet(Stream.of(nodeKey1, nodeKey2), withDirectoryId);
|
||||
LeaderState<?> state = newLeaderState(voters, leaderStartOffset);
|
||||
LeaderState<?> state = newLeaderState(
|
||||
voters,
|
||||
leaderStartOffset,
|
||||
KRaftVersion.KRAFT_VERSION_1
|
||||
);
|
||||
|
||||
state.updateLocalState(new LogOffsetMetadata(leaderEndOffset), voters);
|
||||
assertEquals(Optional.empty(), state.highWatermark());
|
||||
|
@ -437,7 +507,11 @@ public class LeaderStateTest {
|
|||
Stream.of(nodeKey1, nodeKey2, nodeKey3, nodeKey4),
|
||||
withDirectoryId
|
||||
);
|
||||
LeaderState<?> state = newLeaderState(voters, 0L);
|
||||
LeaderState<?> state = newLeaderState(
|
||||
voters,
|
||||
0L,
|
||||
KRaftVersion.KRAFT_VERSION_1
|
||||
);
|
||||
|
||||
assertEquals(checkQuorumTimeoutMs, state.timeUntilCheckQuorumExpires(time.milliseconds()));
|
||||
int resignLeadershipTimeout = checkQuorumTimeoutMs;
|
||||
|
@ -473,7 +547,11 @@ public class LeaderStateTest {
|
|||
ReplicaKey nodeKey3 = ReplicaKey.of(3, Uuid.randomUuid());
|
||||
|
||||
VoterSet originalVoters = localWithRemoteVoterSet(Stream.of(nodeKey1, nodeKey2), true);
|
||||
LeaderState<?> state = newLeaderState(originalVoters, 0L);
|
||||
LeaderState<?> state = newLeaderState(
|
||||
originalVoters,
|
||||
0L,
|
||||
KRaftVersion.KRAFT_VERSION_1
|
||||
);
|
||||
assertEquals(checkQuorumTimeoutMs, state.timeUntilCheckQuorumExpires(time.milliseconds()));
|
||||
|
||||
// checkQuorum timeout not exceeded, should not expire the timer
|
||||
|
@ -498,7 +576,7 @@ public class LeaderStateTest {
|
|||
assertEquals(checkQuorumTimeoutMs, state.timeUntilCheckQuorumExpires(time.milliseconds()));
|
||||
|
||||
// removing leader from the voter set
|
||||
VoterSet votersWithoutLeader = votersWithNode3.removeVoter(localReplicaKey).get();
|
||||
VoterSet votersWithoutLeader = votersWithNode3.removeVoter(localVoterNode.voterKey()).get();
|
||||
state.updateLocalState(new LogOffsetMetadata(1L), votersWithoutLeader);
|
||||
|
||||
time.sleep(checkQuorumTimeoutMs / 2);
|
||||
|
@ -517,8 +595,9 @@ public class LeaderStateTest {
|
|||
|
||||
// Only 1 voter quorum
|
||||
LeaderState<?> state = newLeaderState(
|
||||
VoterSetTest.voterSet(Stream.of(localReplicaKey)),
|
||||
0L
|
||||
VoterSetTest.voterSet(Stream.of(localVoterNode.voterKey())),
|
||||
0L,
|
||||
KRaftVersion.KRAFT_VERSION_1
|
||||
);
|
||||
assertEquals(Long.MAX_VALUE, state.timeUntilCheckQuorumExpires(time.milliseconds()));
|
||||
|
||||
|
@ -536,11 +615,15 @@ public class LeaderStateTest {
|
|||
|
||||
@Test
|
||||
public void testLeaderEndpoints() {
|
||||
VoterSet voters = VoterSetTest.voterSet(Stream.of(localReplicaKey));
|
||||
LeaderState<?> state = newLeaderState(voters, 0L);
|
||||
VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterNode.voterKey()));
|
||||
LeaderState<?> state = newLeaderState(
|
||||
voters,
|
||||
0L,
|
||||
KRaftVersion.KRAFT_VERSION_1
|
||||
);
|
||||
|
||||
assertNotEquals(Endpoints.empty(), state.leaderEndpoints());
|
||||
assertEquals(voters.listeners(localReplicaKey.id()), state.leaderEndpoints());
|
||||
assertEquals(voters.listeners(localVoterNode.voterKey().id()), state.leaderEndpoints());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -555,7 +638,11 @@ public class LeaderStateTest {
|
|||
false
|
||||
);
|
||||
|
||||
LeaderState<?> state = newLeaderState(votersBeforeUpgrade, 0L);
|
||||
LeaderState<?> state = newLeaderState(
|
||||
votersBeforeUpgrade,
|
||||
0L,
|
||||
KRaftVersion.KRAFT_VERSION_1
|
||||
);
|
||||
|
||||
assertFalse(state.updateLocalState(new LogOffsetMetadata(10L), votersBeforeUpgrade));
|
||||
assertTrue(state.updateReplicaState(nodeKey1, 0L, new LogOffsetMetadata(10L)));
|
||||
|
@ -571,18 +658,29 @@ public class LeaderStateTest {
|
|||
@ParameterizedTest
|
||||
@ValueSource(booleans = {true, false})
|
||||
public void testGrantVote(boolean isLogUpToDate) {
|
||||
int[] remoteIds = {1, 2, 3};
|
||||
LeaderState<?> state = newLeaderState(
|
||||
VoterSetTest.voterSet(VoterSetTest.voterMap(IntStream.of(1, 2, 3), false)),
|
||||
1
|
||||
VoterSetTest.voterSet(
|
||||
VoterSetTest.voterMap(
|
||||
IntStream.concat(IntStream.of(localVoterNode.voterKey().id()), IntStream.of(remoteIds)),
|
||||
false
|
||||
)
|
||||
),
|
||||
1,
|
||||
KRaftVersion.KRAFT_VERSION_1
|
||||
);
|
||||
|
||||
assertFalse(state.canGrantVote(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true));
|
||||
assertFalse(state.canGrantVote(ReplicaKey.of(2, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true));
|
||||
assertFalse(state.canGrantVote(ReplicaKey.of(3, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true));
|
||||
|
||||
assertFalse(state.canGrantVote(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, false));
|
||||
assertFalse(state.canGrantVote(ReplicaKey.of(2, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, false));
|
||||
assertFalse(state.canGrantVote(ReplicaKey.of(3, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, false));
|
||||
IntStream.of(remoteIds).forEach(id -> {
|
||||
List.of(true, false).forEach(isPrevote -> {
|
||||
assertFalse(
|
||||
state.canGrantVote(
|
||||
ReplicaKey.of(id, ReplicaKey.NO_DIRECTORY_ID),
|
||||
isLogUpToDate,
|
||||
isPrevote
|
||||
)
|
||||
);
|
||||
});
|
||||
});
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
|
@ -594,7 +692,8 @@ public class LeaderStateTest {
|
|||
VoterSet voters = localWithRemoteVoterSet(IntStream.of(follower1), withDirectoryId);
|
||||
LeaderState<?> state = newLeaderState(
|
||||
voters,
|
||||
epochStartOffset
|
||||
epochStartOffset,
|
||||
KRaftVersion.KRAFT_VERSION_1
|
||||
);
|
||||
assertEquals(0, state.timeUntilBeginQuorumEpochTimerExpires(time.milliseconds()));
|
||||
|
||||
|
@ -609,6 +708,165 @@ public class LeaderStateTest {
|
|||
assertEquals(0, state.timeUntilBeginQuorumEpochTimerExpires(time.milliseconds()));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testVolatileVoters() {
|
||||
int follower1 = 1;
|
||||
long epochStartOffset = 10L;
|
||||
|
||||
VoterSet voters = localWithRemoteVoterSet(IntStream.of(follower1), false);
|
||||
LeaderState<?> state = newLeaderState(
|
||||
voters,
|
||||
epochStartOffset,
|
||||
KRaftVersion.KRAFT_VERSION_0
|
||||
);
|
||||
|
||||
var votersWithLeaderUpdated = state.volatileVoters().get();
|
||||
assertEquals(
|
||||
voters.updateVoterIgnoringDirectoryId(localVoterNode).get(),
|
||||
votersWithLeaderUpdated.voters()
|
||||
);
|
||||
|
||||
var updatedVoters = new KRaftVersionUpgrade.Voters(
|
||||
votersWithLeaderUpdated
|
||||
.voters()
|
||||
.updateVoterIgnoringDirectoryId(VoterSetTest.voterNode(follower1, true))
|
||||
.get()
|
||||
);
|
||||
|
||||
// Upate in-memory voter and check state
|
||||
assertTrue(
|
||||
state.compareAndSetVolatileVoters(votersWithLeaderUpdated, updatedVoters)
|
||||
);
|
||||
assertEquals(updatedVoters, state.volatileVoters().get());
|
||||
|
||||
// Unable to perform atomic update
|
||||
assertFalse(
|
||||
state.compareAndSetVolatileVoters(votersWithLeaderUpdated, updatedVoters)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInvalidMaybeAppendUpgradedKRaftVersion() {
|
||||
int follower1 = 1;
|
||||
int follower2 = 2;
|
||||
long epochStartOffset = 10L;
|
||||
|
||||
VoterSet persistedVoters = localWithRemoteVoterSet(IntStream.of(follower1, follower2), false);
|
||||
LeaderState<?> state = newLeaderState(
|
||||
persistedVoters,
|
||||
epochStartOffset,
|
||||
KRaftVersion.KRAFT_VERSION_0
|
||||
);
|
||||
|
||||
// none of the remove voters support kraft version 1 since the starting version is 0.
|
||||
assertThrows(
|
||||
InvalidUpdateVersionException.class,
|
||||
() ->
|
||||
state.maybeAppendUpgradedKRaftVersion(
|
||||
epoch,
|
||||
KRaftVersion.KRAFT_VERSION_1,
|
||||
KRaftVersion.KRAFT_VERSION_0,
|
||||
persistedVoters,
|
||||
false,
|
||||
time.milliseconds()
|
||||
)
|
||||
);
|
||||
|
||||
// epoch is less than the leader's epoch
|
||||
assertThrows(
|
||||
NotLeaderException.class,
|
||||
() ->
|
||||
state.maybeAppendUpgradedKRaftVersion(
|
||||
epoch - 1,
|
||||
KRaftVersion.KRAFT_VERSION_1,
|
||||
KRaftVersion.KRAFT_VERSION_0,
|
||||
persistedVoters,
|
||||
false,
|
||||
time.milliseconds()
|
||||
)
|
||||
);
|
||||
|
||||
// epoch is greater than the leader's epoch
|
||||
assertThrows(
|
||||
IllegalArgumentException.class,
|
||||
() ->
|
||||
state.maybeAppendUpgradedKRaftVersion(
|
||||
epoch + 1,
|
||||
KRaftVersion.KRAFT_VERSION_1,
|
||||
KRaftVersion.KRAFT_VERSION_0,
|
||||
persistedVoters,
|
||||
false,
|
||||
time.milliseconds()
|
||||
)
|
||||
);
|
||||
|
||||
// noop since the upgrade version is already 1
|
||||
assertFalse(
|
||||
state.maybeAppendUpgradedKRaftVersion(
|
||||
epoch,
|
||||
KRaftVersion.KRAFT_VERSION_1,
|
||||
KRaftVersion.KRAFT_VERSION_1,
|
||||
persistedVoters,
|
||||
false,
|
||||
time.milliseconds()
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMaybeAppendUpgradedKRaftVersion() {
|
||||
int follower1 = 1;
|
||||
int follower2 = 2;
|
||||
long epochStartOffset = 10L;
|
||||
BatchAccumulator<?> accumulator = Mockito.mock(BatchAccumulator.class);
|
||||
|
||||
VoterSet persistedVoters = localWithRemoteVoterSet(IntStream.of(follower1, follower2), false);
|
||||
LeaderState<?> state = newLeaderState(
|
||||
persistedVoters,
|
||||
epochStartOffset,
|
||||
KRaftVersion.KRAFT_VERSION_0,
|
||||
accumulator
|
||||
);
|
||||
|
||||
var updatedVoters = state.volatileVoters().get().voters();
|
||||
updatedVoters = updatedVoters
|
||||
.updateVoterIgnoringDirectoryId(VoterSetTest.voterNode(follower1, true))
|
||||
.get();
|
||||
updatedVoters = updatedVoters
|
||||
.updateVoterIgnoringDirectoryId(VoterSetTest.voterNode(follower2, true))
|
||||
.get();
|
||||
state.compareAndSetVolatileVoters(
|
||||
state.volatileVoters().get(),
|
||||
new KRaftVersionUpgrade.Voters(updatedVoters)
|
||||
);
|
||||
|
||||
assertTrue(
|
||||
state.maybeAppendUpgradedKRaftVersion(
|
||||
epoch,
|
||||
KRaftVersion.KRAFT_VERSION_1,
|
||||
KRaftVersion.KRAFT_VERSION_0,
|
||||
persistedVoters,
|
||||
false,
|
||||
time.milliseconds()
|
||||
)
|
||||
);
|
||||
|
||||
// Expect control records after upgrading the kraft version.
|
||||
Mockito.verify(accumulator).appendControlMessages(Mockito.any());
|
||||
|
||||
// maybe upgrade kraft version should be a noop after an upgrade
|
||||
assertFalse(
|
||||
state.maybeAppendUpgradedKRaftVersion(
|
||||
epoch,
|
||||
KRaftVersion.KRAFT_VERSION_1,
|
||||
KRaftVersion.KRAFT_VERSION_0,
|
||||
persistedVoters,
|
||||
false,
|
||||
time.milliseconds()
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
private static class MockOffsetMetadata implements OffsetMetadata {
|
||||
private final String value;
|
||||
|
||||
|
|
|
@ -21,6 +21,7 @@ import org.apache.kafka.common.Uuid;
|
|||
import org.apache.kafka.common.feature.SupportedVersionRange;
|
||||
import org.apache.kafka.common.network.ListenerName;
|
||||
import org.apache.kafka.server.common.Feature;
|
||||
import org.apache.kafka.server.common.KRaftVersion;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
|
@ -144,6 +145,34 @@ public final class VoterSetTest {
|
|||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testUpdateVoterIgnoringDirectoryId() {
|
||||
Map<Integer, VoterSet.VoterNode> aVoterMap = voterMap(IntStream.of(1, 2, 3), false);
|
||||
VoterSet voterSet = VoterSet.fromMap(new HashMap<>(aVoterMap));
|
||||
|
||||
// Cannot override node id not contianed in the voter set
|
||||
assertEquals(Optional.empty(), voterSet.updateVoterIgnoringDirectoryId(voterNode(4, true)));
|
||||
|
||||
// Test that it can override voter set with different directory ids
|
||||
VoterSet.VoterNode newVoter3 = voterNode(3, true);
|
||||
assertNotEquals(aVoterMap.get(3).voterKey(), newVoter3.voterKey());
|
||||
aVoterMap.put(3, newVoter3);
|
||||
|
||||
assertEquals(
|
||||
Optional.of(VoterSet.fromMap(new HashMap<>(aVoterMap))),
|
||||
voterSet.updateVoterIgnoringDirectoryId(newVoter3)
|
||||
);
|
||||
|
||||
// Test that it can continue to override voter set with different directory ids
|
||||
newVoter3 = voterNode(3, true);
|
||||
assertNotEquals(aVoterMap.get(3).voterKey(), newVoter3.voterKey());
|
||||
aVoterMap.put(3, newVoter3);
|
||||
|
||||
assertEquals(
|
||||
Optional.of(VoterSet.fromMap(new HashMap<>(aVoterMap))),
|
||||
voterSet.updateVoterIgnoringDirectoryId(newVoter3)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testCannotRemoveToEmptyVoterSet() {
|
||||
|
@ -314,6 +343,17 @@ public final class VoterSetTest {
|
|||
assertMajorities(false, startingVoterSet, replacedVoterSet);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testSupportsVersion() {
|
||||
VoterSet voterSet = voterSet(voterMap(IntStream.of(1, 2, 3), false));
|
||||
assertTrue(voterSet.supportsVersion(KRaftVersion.KRAFT_VERSION_0));
|
||||
assertFalse(voterSet.supportsVersion(KRaftVersion.KRAFT_VERSION_1));
|
||||
|
||||
voterSet = voterSet(voterMap(IntStream.of(1, 2, 3), true));
|
||||
assertTrue(voterSet.supportsVersion(KRaftVersion.KRAFT_VERSION_0));
|
||||
assertTrue(voterSet.supportsVersion(KRaftVersion.KRAFT_VERSION_1));
|
||||
}
|
||||
|
||||
private void assertMajorities(boolean overlap, VoterSet a, VoterSet b) {
|
||||
assertEquals(
|
||||
overlap,
|
||||
|
@ -373,11 +413,11 @@ public final class VoterSetTest {
|
|||
}
|
||||
|
||||
public static VoterSet.VoterNode voterNode(ReplicaKey replicaKey, Endpoints endpoints) {
|
||||
return new VoterSet.VoterNode(
|
||||
replicaKey,
|
||||
endpoints,
|
||||
Feature.KRAFT_VERSION.supportedVersionRange()
|
||||
);
|
||||
var supportedVersionRange = replicaKey.directoryId().isEmpty() ?
|
||||
new SupportedVersionRange((short) 0) :
|
||||
Feature.KRAFT_VERSION.supportedVersionRange();
|
||||
|
||||
return new VoterSet.VoterNode(replicaKey, endpoints, supportedVersionRange);
|
||||
}
|
||||
|
||||
public static VoterSet voterSet(Map<Integer, VoterSet.VoterNode> voters) {
|
||||
|
|
|
@ -52,7 +52,7 @@ public enum KRaftVersion implements FeatureVersion {
|
|||
case 1:
|
||||
return KRAFT_VERSION_1;
|
||||
default:
|
||||
throw new RuntimeException("Unknown KRaft feature level: " + (int) version);
|
||||
throw new IllegalArgumentException("Unknown KRaft feature level: " + (int) version);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -80,29 +80,40 @@ public enum KRaftVersion implements FeatureVersion {
|
|||
}
|
||||
}
|
||||
|
||||
public boolean isAtLeast(KRaftVersion otherVersion) {
|
||||
return this.compareTo(otherVersion) >= 0;
|
||||
}
|
||||
|
||||
public boolean isMoreThan(KRaftVersion otherVersion) {
|
||||
return this.compareTo(otherVersion) > 0;
|
||||
}
|
||||
|
||||
public short quorumStateVersion() {
|
||||
switch (this) {
|
||||
case KRAFT_VERSION_0:
|
||||
return (short) 0;
|
||||
case KRAFT_VERSION_1:
|
||||
return (short) 1;
|
||||
default:
|
||||
throw new IllegalStateException("Unsupported KRaft feature level: " + this);
|
||||
}
|
||||
throw new IllegalStateException("Unsupported KRaft feature level: " + this);
|
||||
}
|
||||
|
||||
public short kraftVersionRecordVersion() {
|
||||
switch (this) {
|
||||
case KRAFT_VERSION_1:
|
||||
return (short) 0;
|
||||
default:
|
||||
throw new IllegalStateException("Unsupported KRaft feature level: " + this);
|
||||
}
|
||||
throw new IllegalStateException("Unsupported KRaft feature level: " + this);
|
||||
}
|
||||
|
||||
public short votersRecordVersion() {
|
||||
switch (this) {
|
||||
case KRAFT_VERSION_1:
|
||||
return (short) 0;
|
||||
default:
|
||||
throw new IllegalStateException("Unsupported KRaft feature level: " + this);
|
||||
}
|
||||
throw new IllegalStateException("Unsupported KRaft feature level: " + this);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -22,7 +22,9 @@ import org.apache.kafka.common.record.ControlRecordUtils;
|
|||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
public final class KRaftVersionTest {
|
||||
@Test
|
||||
|
@ -110,4 +112,20 @@ public final class KRaftVersionTest {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testIsAtLeast() {
|
||||
assertTrue(KRaftVersion.KRAFT_VERSION_0.isAtLeast(KRaftVersion.KRAFT_VERSION_0));
|
||||
assertFalse(KRaftVersion.KRAFT_VERSION_0.isAtLeast(KRaftVersion.KRAFT_VERSION_1));
|
||||
assertTrue(KRaftVersion.KRAFT_VERSION_1.isAtLeast(KRaftVersion.KRAFT_VERSION_0));
|
||||
assertTrue(KRaftVersion.KRAFT_VERSION_1.isAtLeast(KRaftVersion.KRAFT_VERSION_1));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testIsMoreThan() {
|
||||
assertFalse(KRaftVersion.KRAFT_VERSION_0.isMoreThan(KRaftVersion.KRAFT_VERSION_0));
|
||||
assertFalse(KRaftVersion.KRAFT_VERSION_0.isMoreThan(KRaftVersion.KRAFT_VERSION_1));
|
||||
assertTrue(KRaftVersion.KRAFT_VERSION_1.isMoreThan(KRaftVersion.KRAFT_VERSION_0));
|
||||
assertFalse(KRaftVersion.KRAFT_VERSION_1.isMoreThan(KRaftVersion.KRAFT_VERSION_1));
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue