jsancio commented on a change in pull request #11733:
URL: https://github.com/apache/kafka/pull/11733#discussion_r813988441



##########
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:
       Yeah. Maybe a INVALID_REQUEST is better. It looks like the legacy 
controller just processes the request if the leader has a higher epoch then 
what the controller knows.




-- 
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


Reply via email to