hachikuji commented on a change in pull request #11733: URL: https://github.com/apache/kafka/pull/11733#discussion_r813188764
########## File path: core/src/test/scala/unit/kafka/cluster/PartitionTest.scala ########## @@ -1077,6 +1078,58 @@ class PartitionTest extends AbstractPartitionTest { } + @Test + def testInvalidAlterPartitionAreNotRetried(): Unit = { + val log = logManager.getOrCreateLog(topicPartition, topicId = None) + seedLogData(log, numRecords = 10, leaderEpoch = 4) + + val controllerEpoch = 0 + val leaderEpoch = 5 + val remoteBrokerId = brokerId + 1 + val replicas = List[Integer](brokerId, remoteBrokerId).asJava + val isr = List[Integer](brokerId).asJava + + partition.createLogIfNotExists(isNew = false, isFutureReplica = false, offsetCheckpoints, None) + assertTrue(partition.makeLeader( + new LeaderAndIsrPartitionState() + .setControllerEpoch(controllerEpoch) + .setLeader(brokerId) + .setLeaderEpoch(leaderEpoch) + .setIsr(isr) + .setZkVersion(1) + .setReplicas(replicas) + .setIsNew(true), + offsetCheckpoints, None), "Expected become leader transition to succeed") + assertEquals(Set(brokerId), partition.partitionState.isr) + + val remoteReplica = partition.getReplica(remoteBrokerId).get + assertEquals(LogOffsetMetadata.UnknownOffsetMetadata.messageOffset, remoteReplica.logEndOffset) + assertEquals(UnifiedLog.UnknownOffset, remoteReplica.logStartOffset) + + partition.updateFollowerFetchState(remoteBrokerId, + followerFetchOffsetMetadata = LogOffsetMetadata(10), + followerStartOffset = 0L, + followerFetchTimeMs = time.milliseconds(), + leaderEndOffset = 10L) + + // Check that the isr didn't change and alter update is scheduled Review comment: Could we assert something about the inflight ISR? ########## File path: metadata/src/main/java/org/apache/kafka/controller/PartitionChangeBuilder.java ########## @@ -104,6 +107,14 @@ public PartitionChangeBuilder setTargetAdding(List<Integer> targetAdding) { return this; } + public PartitionChangeBuilder setTargetLeaderRecoveryState(LeaderRecoveryState targetLeaderRecoveryState) { + this.targetLeaderRecoveryState = targetLeaderRecoveryState; + return this; + } + + // TODO: We need to make sure that the LeaderRecoveryState is not lost when the partition transitions from Review comment: TODO here ########## File path: core/src/test/scala/kafka/zk/TopicPartitionStateZNodeTest.scala ########## @@ -0,0 +1,53 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +*/ +package kafka.zk + +import TopicPartitionStateZNode.decode +import TopicPartitionStateZNode.encode +import kafka.api.LeaderAndIsr +import kafka.controller.LeaderIsrAndControllerEpoch +import org.apache.kafka.metadata.LeaderRecoveryState +import org.apache.zookeeper.data.Stat +import org.junit.jupiter.api.Assertions.assertEquals +import org.junit.jupiter.api.Test +import org.mockito.Mockito.mock +import org.mockito.Mockito.when + +final class TopicPartitionStateZNodeTest { + + @Test + def testEncodeDecodeRecovering(): Unit = { + val zkVersion = 5 + val stat = mock(classOf[Stat]) + when(stat.getVersion).thenReturn(zkVersion) + + val expected = LeaderIsrAndControllerEpoch(LeaderAndIsr(1, 6, List(1), LeaderRecoveryState.RECOVERING, zkVersion), 10) + + assertEquals(Some(expected), decode(encode(expected), stat)) + } + + @Test + def testEncodeDecodeRecovered(): Unit = { Review comment: Do we have the case covered when we are decoding an old value without the leader recovery state field? ########## File path: metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java ########## @@ -760,44 +719,132 @@ BrokersToIsrs brokersToIsrs() { } if (change.leader() != request.brokerId() && change.leader() != NO_LEADER_CHANGE) { - // Normally, an alterIsr request, which is made by the partition + // Normally, an alterPartition request, which is made by the partition // leader itself, is not allowed to modify the partition leader. // However, if there is an ongoing partition reassignment and the // ISR change completes it, then the leader may change as part of // the changes made during reassignment cleanup. // // In this case, we report back FENCED_LEADER_EPOCH to the leader - // which made the alterIsr request. This lets it know that it must + // which made the alterPartition request. This lets it know that it must // fetch new metadata before trying again. This return code is // unusual because we both return an error and generate a new // metadata record. We usually only do one or the other. - log.info("AlterIsr request from node {} for {}-{} completed " + + log.info("AlterPartition request from node {} for {}-{} completed " + "the ongoing partition reassignment and triggered a " + - "leadership change. Reutrning FENCED_LEADER_EPOCH.", + "leadership change. Returning FENCED_LEADER_EPOCH.", request.brokerId(), topic.name, partitionId); - responseTopicData.partitions().add(new AlterIsrResponseData.PartitionData(). + responseTopicData.partitions().add(new AlterPartitionResponseData.PartitionData(). setPartitionIndex(partitionId). setErrorCode(FENCED_LEADER_EPOCH.code())); continue; } else if (change.removingReplicas() != null || change.addingReplicas() != null) { - log.info("AlterIsr request from node {} for {}-{} completed " + + log.info("AlterPartition request from node {} for {}-{} completed " + "the ongoing partition reassignment.", request.brokerId(), topic.name, partitionId); } } - responseTopicData.partitions().add(new AlterIsrResponseData.PartitionData(). + + responseTopicData.partitions().add(new AlterPartitionResponseData.PartitionData(). setPartitionIndex(partitionId). setErrorCode(result.code()). setLeaderId(partition.leader). + setIsr(Replicas.toList(partition.isr)). + setLeaderRecoveryState(partition.leaderRecoveryState.value()). setLeaderEpoch(partition.leaderEpoch). - setCurrentIsrVersion(partition.partitionEpoch). - setIsr(Replicas.toList(partition.isr))); + setPartitionEpoch(partition.partitionEpoch)); } } + return ControllerResult.of(records, response); } + /** + * Validate the partition information included in the alter partition request. + * + * @param brokerId id of the broker requesting the alter partition + * @param topic current topic information store by the replication manager + * @param partitionId partition id being altered + * @param partition current partition registration for the partition being altered + * @param partitionData partition data from the alter partition request + * + * @return Errors.NONE for valid alter partition data; otherwise the validation error + */ + private Errors validateAlterPartitionData( + int brokerId, + TopicControlInfo topic, + int partitionId, + PartitionRegistration partition, + AlterPartitionRequestData.PartitionData partitionData + ) { + if (partition == null) { + log.info("Rejecting alterPartition request for unknown partition {}-{}.", + topic.name, partitionId); + + return UNKNOWN_TOPIC_OR_PARTITION; + } + if (partitionData.leaderEpoch() != partition.leaderEpoch) { Review comment: Not part of this patch, but I was wondering if FENCED_LEADER_EPOCH is the right error code when the request epoch is larger than the registration epoch. ########## File path: core/src/test/scala/unit/kafka/cluster/PartitionTest.scala ########## @@ -1077,6 +1078,58 @@ class PartitionTest extends AbstractPartitionTest { } + @Test + def testInvalidAlterPartitionAreNotRetried(): Unit = { Review comment: nit: perhaps `testInvalidAlterPartitionRequestsAreNotRetried`? ########## File path: metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java ########## @@ -760,44 +719,132 @@ BrokersToIsrs brokersToIsrs() { } if (change.leader() != request.brokerId() && change.leader() != NO_LEADER_CHANGE) { - // Normally, an alterIsr request, which is made by the partition + // Normally, an alterPartition request, which is made by the partition // leader itself, is not allowed to modify the partition leader. // However, if there is an ongoing partition reassignment and the // ISR change completes it, then the leader may change as part of // the changes made during reassignment cleanup. // // In this case, we report back FENCED_LEADER_EPOCH to the leader - // which made the alterIsr request. This lets it know that it must + // which made the alterPartition request. This lets it know that it must // fetch new metadata before trying again. This return code is // unusual because we both return an error and generate a new // metadata record. We usually only do one or the other. - log.info("AlterIsr request from node {} for {}-{} completed " + + log.info("AlterPartition request from node {} for {}-{} completed " + "the ongoing partition reassignment and triggered a " + - "leadership change. Reutrning FENCED_LEADER_EPOCH.", + "leadership change. Returning FENCED_LEADER_EPOCH.", request.brokerId(), topic.name, partitionId); - responseTopicData.partitions().add(new AlterIsrResponseData.PartitionData(). + responseTopicData.partitions().add(new AlterPartitionResponseData.PartitionData(). setPartitionIndex(partitionId). setErrorCode(FENCED_LEADER_EPOCH.code())); continue; } else if (change.removingReplicas() != null || change.addingReplicas() != null) { - log.info("AlterIsr request from node {} for {}-{} completed " + + log.info("AlterPartition request from node {} for {}-{} completed " + "the ongoing partition reassignment.", request.brokerId(), topic.name, partitionId); } } - responseTopicData.partitions().add(new AlterIsrResponseData.PartitionData(). + + responseTopicData.partitions().add(new AlterPartitionResponseData.PartitionData(). setPartitionIndex(partitionId). setErrorCode(result.code()). setLeaderId(partition.leader). + setIsr(Replicas.toList(partition.isr)). + setLeaderRecoveryState(partition.leaderRecoveryState.value()). Review comment: Hmm, how do we guarantee that this field is supported? ########## File path: core/src/test/scala/unit/kafka/cluster/PartitionTest.scala ########## @@ -2015,6 +2067,8 @@ class PartitionTest extends AbstractPartitionTest { verify(spyConfigRepository, times(2)).topicConfig(topicPartition.topic()) } + // TODO: Add a test that shows that the follower rejects reads until a recovered leader and isr Review comment: Don't forget about TODO ########## File path: metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java ########## @@ -760,44 +719,132 @@ BrokersToIsrs brokersToIsrs() { } if (change.leader() != request.brokerId() && change.leader() != NO_LEADER_CHANGE) { - // Normally, an alterIsr request, which is made by the partition + // Normally, an alterPartition request, which is made by the partition // leader itself, is not allowed to modify the partition leader. // However, if there is an ongoing partition reassignment and the // ISR change completes it, then the leader may change as part of // the changes made during reassignment cleanup. // // In this case, we report back FENCED_LEADER_EPOCH to the leader - // which made the alterIsr request. This lets it know that it must + // which made the alterPartition request. This lets it know that it must // fetch new metadata before trying again. This return code is // unusual because we both return an error and generate a new // metadata record. We usually only do one or the other. - log.info("AlterIsr request from node {} for {}-{} completed " + + log.info("AlterPartition request from node {} for {}-{} completed " + "the ongoing partition reassignment and triggered a " + - "leadership change. Reutrning FENCED_LEADER_EPOCH.", + "leadership change. Returning FENCED_LEADER_EPOCH.", request.brokerId(), topic.name, partitionId); - responseTopicData.partitions().add(new AlterIsrResponseData.PartitionData(). + responseTopicData.partitions().add(new AlterPartitionResponseData.PartitionData(). setPartitionIndex(partitionId). setErrorCode(FENCED_LEADER_EPOCH.code())); continue; } else if (change.removingReplicas() != null || change.addingReplicas() != null) { - log.info("AlterIsr request from node {} for {}-{} completed " + + log.info("AlterPartition request from node {} for {}-{} completed " + "the ongoing partition reassignment.", request.brokerId(), topic.name, partitionId); } } - responseTopicData.partitions().add(new AlterIsrResponseData.PartitionData(). + + responseTopicData.partitions().add(new AlterPartitionResponseData.PartitionData(). setPartitionIndex(partitionId). setErrorCode(result.code()). setLeaderId(partition.leader). + setIsr(Replicas.toList(partition.isr)). + setLeaderRecoveryState(partition.leaderRecoveryState.value()). setLeaderEpoch(partition.leaderEpoch). - setCurrentIsrVersion(partition.partitionEpoch). - setIsr(Replicas.toList(partition.isr))); + setPartitionEpoch(partition.partitionEpoch)); } } + return ControllerResult.of(records, response); } + /** + * Validate the partition information included in the alter partition request. + * + * @param brokerId id of the broker requesting the alter partition + * @param topic current topic information store by the replication manager + * @param partitionId partition id being altered + * @param partition current partition registration for the partition being altered + * @param partitionData partition data from the alter partition request + * + * @return Errors.NONE for valid alter partition data; otherwise the validation error + */ + private Errors validateAlterPartitionData( + int brokerId, + TopicControlInfo topic, + int partitionId, + PartitionRegistration partition, + AlterPartitionRequestData.PartitionData partitionData + ) { + if (partition == null) { + log.info("Rejecting alterPartition request for unknown partition {}-{}.", Review comment: nit: I think we typically capitalize request names in logging -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org