This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch 2.4
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/2.4 by this push:
     new e6bb907  MINOR: Rename brokers to replicas in the reassignment API 
(#7570)
e6bb907 is described below

commit e6bb90783b9066c1266dadd22c7153a51ca252fa
Author: José Armando García Sancio <[email protected]>
AuthorDate: Tue Oct 22 15:36:53 2019 -0700

    MINOR: Rename brokers to replicas in the reassignment API (#7570)
    
    Reviewers: Jason Gustafson <[email protected]>, Manikumar Reddy 
<[email protected]>, Vikas Singh <[email protected]>, Colin P. McCabe 
<[email protected]>
    (cherry picked from commit c00bd38ab27ceaee174be8ce803c64000c5730ed)
---
 .../org/apache/kafka/clients/admin/KafkaAdminClient.java |  2 +-
 .../kafka/clients/admin/NewPartitionReassignment.java    | 16 +++++-----------
 .../apache/kafka/clients/admin/KafkaAdminClientTest.java | 12 ++++++++----
 3 files changed, 14 insertions(+), 16 deletions(-)

diff --git 
a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java 
b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
index 0850ced..79822a6 100644
--- a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
+++ b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
@@ -3270,7 +3270,7 @@ public class KafkaAdminClient extends AdminClient {
 
                         ReassignablePartition reassignablePartition = new 
ReassignablePartition()
                                 .setPartitionIndex(partitionIndex)
-                                
.setReplicas(reassignment.map(NewPartitionReassignment::targetBrokers).orElse(null));
+                                
.setReplicas(reassignment.map(NewPartitionReassignment::targetReplicas).orElse(null));
                         reassignablePartitions.add(reassignablePartition);
                     }
 
diff --git 
a/clients/src/main/java/org/apache/kafka/clients/admin/NewPartitionReassignment.java
 
b/clients/src/main/java/org/apache/kafka/clients/admin/NewPartitionReassignment.java
index 8856470..0f7a61c 100644
--- 
a/clients/src/main/java/org/apache/kafka/clients/admin/NewPartitionReassignment.java
+++ 
b/clients/src/main/java/org/apache/kafka/clients/admin/NewPartitionReassignment.java
@@ -17,28 +17,22 @@
 
 package org.apache.kafka.clients.admin;
 
-import java.util.Arrays;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
-import java.util.Optional;
 
 /**
  * A new partition reassignment, which can be applied via {@link 
AdminClient#alterPartitionReassignments(Map, 
AlterPartitionReassignmentsOptions)}.
  */
 public class NewPartitionReassignment {
-    private final List<Integer> targetBrokers;
+    private final List<Integer> targetReplicas;
 
-    public static Optional<NewPartitionReassignment> of(Integer... brokers) {
-        return Optional.of(new 
NewPartitionReassignment(Arrays.asList(brokers)));
+    public NewPartitionReassignment(List<Integer> targetReplicas) {
+        this.targetReplicas = Collections.unmodifiableList(new 
ArrayList<>(targetReplicas));
     }
 
-    public NewPartitionReassignment(List<Integer> targetBrokers) {
-        this.targetBrokers = Collections.unmodifiableList(new 
ArrayList<>(targetBrokers));
-    }
-
-    public List<Integer> targetBrokers() {
-        return targetBrokers;
+    public List<Integer> targetReplicas() {
+        return targetReplicas;
     }
 }
diff --git 
a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java
 
b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java
index 6d870fc..eac7063 100644
--- 
a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java
+++ 
b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java
@@ -2011,7 +2011,7 @@ public class KafkaAdminClientTest {
             TopicPartition tp2 = new TopicPartition("B", 0);
             Map<TopicPartition, Optional<NewPartitionReassignment>> 
reassignments = new HashMap<>();
             reassignments.put(tp1, Optional.empty());
-            reassignments.put(tp2, NewPartitionReassignment.of(1, 2, 3));
+            reassignments.put(tp2, newPartitionReassignment(Arrays.asList(1, 
2, 3)));
 
             // 1. server returns less responses than number of partitions we 
sent
             AlterPartitionReassignmentsResponseData responseData1 = new 
AlterPartitionReassignmentsResponseData();
@@ -2102,9 +2102,9 @@ public class KafkaAdminClientTest {
             TopicPartition invalidTopicTP = new TopicPartition("", 0);
             TopicPartition invalidPartitionTP = new TopicPartition("ABC", -1);
             Map<TopicPartition, Optional<NewPartitionReassignment>> 
invalidTopicReassignments = new HashMap<>();
-            invalidTopicReassignments.put(invalidPartitionTP, 
NewPartitionReassignment.of(1, 2, 3));
-            invalidTopicReassignments.put(invalidTopicTP, 
NewPartitionReassignment.of(1, 2, 3));
-            invalidTopicReassignments.put(tp1, NewPartitionReassignment.of(1, 
2, 3));
+            invalidTopicReassignments.put(invalidPartitionTP, 
newPartitionReassignment(Arrays.asList(1, 2, 3)));
+            invalidTopicReassignments.put(invalidTopicTP, 
newPartitionReassignment(Arrays.asList(1, 2, 3)));
+            invalidTopicReassignments.put(tp1, 
newPartitionReassignment(Arrays.asList(1, 2, 3)));
 
             AlterPartitionReassignmentsResponseData singlePartResponseData =
                     new AlterPartitionReassignmentsResponseData()
@@ -2271,4 +2271,8 @@ public class KafkaAdminClientTest {
             }
         }
     }
+
+    private static Optional<NewPartitionReassignment> 
newPartitionReassignment(List<Integer> targetReplicas) {
+        return Optional.of(new NewPartitionReassignment(targetReplicas));
+    }
 }

Reply via email to