You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2022/11/22 18:12:32 UTC

[GitHub] [kafka] andymg3 opened a new pull request, #12892: KAFKA-14386: Change ReplicaPlacer place method to return a class instead of list of list of integers

andymg3 opened a new pull request, #12892:
URL: https://github.com/apache/kafka/pull/12892

   ### JIRA
   https://issues.apache.org/jira/browse/KAFKA-14386
   
   ### Summary
   This changes the `ReplicaPlacer` interface to return a class instead of a list of list of integers. There are two reasons for the suggestion. First, as mentioned in the JIRA, it will make the interface, arguably, a bit more readable and understandable by explicitly modeling the idea of topic and partition. Second and more importantly, it makes the interface more extendable in the future. Right now it would be challenging to add more metadata to the response. By having classes, we can easily add fields to them without breaking/changing the interface. For example, in the CreatePartitions RPC we are adding partitions to an existing topic and we might want to add some metadata to response making it clear which partition the assignment starts at which could look something like:
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] andymg3 commented on a diff in pull request #12892: KAFKA-14386: Change ReplicaPlacer place method to return a class instead of a list of list of integers

Posted by GitBox <gi...@apache.org>.
andymg3 commented on code in PR #12892:
URL: https://github.com/apache/kafka/pull/12892#discussion_r1037585759


##########
metadata/src/test/java/org/apache/kafka/metadata/placement/PartitionAssignmentTest.java:
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.metadata.placement;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.List;
+
+public class PartitionAssignmentTest {
+
+    @Test
+    public void testPartitionAssignmentReplicas() {
+        List<Integer> replicas = Arrays.asList(0, 1, 2);
+        assertEquals(replicas, new PartitionAssignment(replicas).replicas());
+    }
+
+    @Test
+    public void testConsistentEqualsAndHashCode() {
+        List<PartitionAssignment> partitionAssignments = Arrays.asList(
+            new PartitionAssignment(
+                    Arrays.asList(0, 1, 2)
+            ),
+            new PartitionAssignment(
+                    Arrays.asList(1, 2, 0)
+            )
+        );
+
+        for (int i = 0; i < partitionAssignments.size(); i++) {
+            for (int j = 0; j < partitionAssignments.size(); j++) {
+                if (i == j) {
+                    assertEquals(partitionAssignments.get(i), partitionAssignments.get(j));

Review Comment:
   Thats true. If `equals` is updated to compare references we arent checking the rest of the code. I can add into another assert like:
   ```
   assertEquals(partitionAssignments.get(i), new PartitionAssignment(partitionAssignments.get(j).replicas()));
   ```
   Does that make sense to you? 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] andymg3 commented on a diff in pull request #12892: KAFKA-14386: Change ReplicaPlacer place method to return a class instead of a list of list of integers

Posted by GitBox <gi...@apache.org>.
andymg3 commented on code in PR #12892:
URL: https://github.com/apache/kafka/pull/12892#discussion_r1037475316


##########
metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java:
##########
@@ -407,13 +405,8 @@ public void testPlaceReplicas(int numUsableBrokers) throws Exception {
                 new PlacementSpec(0,
                     1,
                     (short) 3),
-                new ClusterDescriber() {
-                    @Override
-                    public Iterator<UsableBroker> usableBrokers() {
-                        return clusterControl.usableBrokers();
-                    }
-                }
-            );
+                    () -> clusterControl.usableBrokers()

Review Comment:
   ```
   +++ b/metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java
   @@ -405,7 +405,7 @@ public class ClusterControlManagerTest {
                    new PlacementSpec(0,
                        1,
                        (short) 3),
   -                    () -> clusterControl.usableBrokers()
   +                    () -> clusterControl::usableBrokers
                ).assignments();
                HashSet<Integer> seen = new HashSet<>();
                for (Integer result : results.get(0).replicas()) {
   ...@...kafka % ./gradlew jar
   
   > Configure project :
   Starting build with version 3.4.0-SNAPSHOT (commit id 4ac9307e) using Gradle 7.6, Java 1.8 and Scala 2.13.8
   Build properties: maxParallelForks=12, maxScalacThreads=8, maxTestRetries=0
   
   > Task :metadata:compileTestJava FAILED
   ... kafka/metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java:408: error: incompatible types: bad return type in lambda expression
                       () -> clusterControl::usableBrokers
                             ^
       java.util.Iterator is not a functional interface
         multiple non-overriding abstract methods found in interface java.util.Iterator
   Note: Some messages have been simplified; recompile with -Xdiags:verbose to get full output
   1 error
   ```



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jsancio commented on a diff in pull request #12892: KAFKA-14386: Change ReplicaPlacer place method to return a class instead of a list of list of integers

Posted by GitBox <gi...@apache.org>.
jsancio commented on code in PR #12892:
URL: https://github.com/apache/kafka/pull/12892#discussion_r1037582963


##########
metadata/src/main/java/org/apache/kafka/metadata/placement/TopicAssignment.java:
##########
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.metadata.placement;
+
+import java.util.List;
+
+/**
+ * The KRaft topic assignment.
+ */
+public class TopicAssignment {
+    private List<PartitionAssignment> assignments;
+
+    public TopicAssignment(List<PartitionAssignment> assignments) {
+        this.assignments = assignments;

Review Comment:
   Yes. Ideally you would also clone/copy the parameter pass in the constructor because the caller has a reference to `assignments` and can update it.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] andymg3 commented on a diff in pull request #12892: KAFKA-14386: Change ReplicaPlacer place method to return a class instead of a list of list of integers

Posted by GitBox <gi...@apache.org>.
andymg3 commented on code in PR #12892:
URL: https://github.com/apache/kafka/pull/12892#discussion_r1037473300


##########
metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java:
##########
@@ -407,13 +405,8 @@ public void testPlaceReplicas(int numUsableBrokers) throws Exception {
                 new PlacementSpec(0,
                     1,
                     (short) 3),
-                new ClusterDescriber() {
-                    @Override
-                    public Iterator<UsableBroker> usableBrokers() {
-                        return clusterControl.usableBrokers();
-                    }
-                }
-            );
+                    () -> clusterControl.usableBrokers()

Review Comment:
   `clusterControl::usableBrokers` doesnt compile. `clusterControl.usableBrokers()` is what I have and it does compile.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jsancio commented on a diff in pull request #12892: KAFKA-14386: Change ReplicaPlacer place method to return a class instead of a list of list of integers

Posted by GitBox <gi...@apache.org>.
jsancio commented on code in PR #12892:
URL: https://github.com/apache/kafka/pull/12892#discussion_r1037580165


##########
metadata/src/test/java/org/apache/kafka/metadata/placement/StripedReplicaPlacerTest.java:
##########
@@ -95,12 +94,7 @@ private List<List<Integer>> place(
         PlacementSpec placementSpec = new PlacementSpec(startPartition,
             numPartitions,
             replicationFactor);
-        ClusterDescriber cluster = new ClusterDescriber() {
-            @Override
-            public Iterator<UsableBroker> usableBrokers() {
-                return brokers.iterator();
-            }
-        };
+        ClusterDescriber cluster = () -> brokers.iterator();

Review Comment:
   You want `ClusterDescribe cluster = brokers::iterator;`



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] andymg3 commented on a diff in pull request #12892: KAFKA-14386: Return TopicAssignment from the ReplicaPlacer

Posted by GitBox <gi...@apache.org>.
andymg3 commented on code in PR #12892:
URL: https://github.com/apache/kafka/pull/12892#discussion_r1037646184


##########
metadata/src/main/java/org/apache/kafka/metadata/placement/TopicAssignment.java:
##########
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.metadata.placement;
+
+import java.util.List;
+
+/**
+ * The KRaft topic assignment.
+ */
+public class TopicAssignment {
+    private List<PartitionAssignment> assignments;
+
+    public TopicAssignment(List<PartitionAssignment> assignments) {
+        this.assignments = assignments;

Review Comment:
   Updated to:
   ```
       public TopicAssignment(final List<PartitionAssignment> assignments) {
           this.assignments = Collections.unmodifiableList(assignments);
       }
   ```
   Also commented on both this class and `PartitionAssignment` that they're immutable.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] andymg3 commented on a diff in pull request #12892: KAFKA-14386: Return TopicAssignment from the ReplicaPlacer

Posted by GitBox <gi...@apache.org>.
andymg3 commented on code in PR #12892:
URL: https://github.com/apache/kafka/pull/12892#discussion_r1037481276


##########
metadata/src/main/java/org/apache/kafka/metadata/placement/TopicAssignment.java:
##########
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.metadata.placement;
+
+import java.util.List;
+
+/**
+ * The KRaft topic assignment.
+ */
+public class TopicAssignment {
+    private List<PartitionAssignment> assignments;
+
+    public TopicAssignment(List<PartitionAssignment> assignments) {
+        this.assignments = assignments;

Review Comment:
   I can update this to:
   ```
       public TopicAssignment(final List<PartitionAssignment> assignments) {
           this.assignments = Collections.unmodifiableList(new ArrayList<>(assignments));
       }
   ```
   Does that make sense? 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] andymg3 commented on a diff in pull request #12892: KAFKA-14386: Change ReplicaPlacer place method to return a class instead of a list of list of integers

Posted by GitBox <gi...@apache.org>.
andymg3 commented on code in PR #12892:
URL: https://github.com/apache/kafka/pull/12892#discussion_r1037170517


##########
metadata/src/main/java/org/apache/kafka/metadata/placement/StripedReplicaPlacer.java:
##########
@@ -440,6 +441,6 @@ public List<List<Integer>> place(
         for (int partition = 0; partition < placement.numPartitions(); partition++) {
             placements.add(rackList.place(placement.numReplicas()));
         }
-        return placements;
+        return new TopicAssignment(placements.stream().map(x -> new PartitionAssignment(x)).collect(Collectors.toList()));

Review Comment:
   Agree. Updated.



##########
metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java:
##########
@@ -407,13 +405,8 @@ public void testPlaceReplicas(int numUsableBrokers) throws Exception {
                 new PlacementSpec(0,
                     1,
                     (short) 3),
-                new ClusterDescriber() {
-                    @Override
-                    public Iterator<UsableBroker> usableBrokers() {
-                        return clusterControl.usableBrokers();
-                    }
-                }
-            );
+                    () -> clusterControl.usableBrokers()
+            ).assignments().stream().map(x -> x.replicas()).collect(Collectors.toList());

Review Comment:
   Just changed this code snippet to:
   ```
    List<PartitionAssignment> results = clusterControl.replicaPlacer().place(
                   new PlacementSpec(0,
                       1,
                       (short) 3),
                       () -> clusterControl.usableBrokers()
               ).assignments();
               HashSet<Integer> seen = new HashSet<>();
     for (Integer result : results.get(0).replicas()) {
   
   ```



##########
metadata/src/main/java/org/apache/kafka/metadata/placement/TopicAssignment.java:
##########
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.metadata.placement;
+
+import java.util.List;
+
+/**
+ * The KRaft topic assignment.
+ */
+public class TopicAssignment {

Review Comment:
   As with `PartitionAssignment` I have added a couple. Since its a pretty simple class I only added two - let me know if there's anything you think I'm missing.



##########
metadata/src/main/java/org/apache/kafka/metadata/placement/PartitionAssignment.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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.metadata.placement;
+
+import java.util.List;
+
+/**
+ * The KRaft partition assignment.

Review Comment:
   Done.



##########
metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java:
##########
@@ -1564,16 +1567,17 @@ void createPartitions(CreatePartitionsTopic topic,
                 isrs.add(isr);
             }
         } else {
-            placements = clusterControl.replicaPlacer().place(new PlacementSpec(
+            partitionAssignments = clusterControl.replicaPlacer().place(new PlacementSpec(
                 startPartitionId,
                 additional,
                 replicationFactor
-            ), clusterDescriber);
-            isrs = placements;
+            ), clusterDescriber).assignments();
+            isrs = partitionAssignments.stream().map(x -> x.replicas()).collect(Collectors.toList());

Review Comment:
   Makes sense. Changed.



##########
metadata/src/main/java/org/apache/kafka/metadata/placement/PartitionAssignment.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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.metadata.placement;
+
+import java.util.List;
+
+/**
+ * The KRaft partition assignment.
+ *
+ * The assignment is represented as a list of integers where each integer is the replica ID.
+ */
+public class PartitionAssignment {
+    private List<Integer> replicas;
+
+    public PartitionAssignment(List<Integer> replicas) {
+        this.replicas = replicas;

Review Comment:
   Done.



##########
metadata/src/main/java/org/apache/kafka/metadata/placement/TopicAssignment.java:
##########
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.metadata.placement;
+
+import java.util.List;
+
+/**
+ * The KRaft topic assignment.

Review Comment:
   Done.



##########
metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java:
##########
@@ -1564,16 +1567,17 @@ void createPartitions(CreatePartitionsTopic topic,
                 isrs.add(isr);
             }
         } else {
-            placements = clusterControl.replicaPlacer().place(new PlacementSpec(
+            partitionAssignments = clusterControl.replicaPlacer().place(new PlacementSpec(
                 startPartitionId,
                 additional,
                 replicationFactor
-            ), clusterDescriber);
-            isrs = placements;
+            ), clusterDescriber).assignments();

Review Comment:
   Changed.



##########
metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java:
##########
@@ -407,13 +405,8 @@ public void testPlaceReplicas(int numUsableBrokers) throws Exception {
                 new PlacementSpec(0,
                     1,
                     (short) 3),
-                new ClusterDescriber() {
-                    @Override
-                    public Iterator<UsableBroker> usableBrokers() {
-                        return clusterControl.usableBrokers();
-                    }
-                }
-            );
+                    () -> clusterControl.usableBrokers()

Review Comment:
   Compiler doesnt like that: `Iterator<UsableBroker> is not a functional interface`



##########
metadata/src/main/java/org/apache/kafka/metadata/placement/TopicAssignment.java:
##########
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.metadata.placement;
+
+import java.util.List;
+
+/**
+ * The KRaft topic assignment.
+ */
+public class TopicAssignment {
+    private List<PartitionAssignment> assignments;
+
+    public TopicAssignment(List<PartitionAssignment> assignments) {
+        this.assignments = assignments;

Review Comment:
   Agreed. Done.



##########
metadata/src/test/java/org/apache/kafka/metadata/placement/StripedReplicaPlacerTest.java:
##########
@@ -101,7 +102,7 @@ public Iterator<UsableBroker> usableBrokers() {
                 return brokers.iterator();
             }
         };
-        return placer.place(placementSpec, cluster);
+        return placer.place(placementSpec, cluster).assignments().stream().map(x -> x.replicas()).collect(Collectors.toList());

Review Comment:
   I was originally trying to reduce the number of changes. But I think it makes sense to return a TopicAssignment so we dont have to keep creating another list. 



##########
metadata/src/main/java/org/apache/kafka/metadata/placement/PartitionAssignment.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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.metadata.placement;
+
+import java.util.List;
+
+/**
+ * The KRaft partition assignment.
+ *
+ * The assignment is represented as a list of integers where each integer is the replica ID.
+ */
+public class PartitionAssignment {

Review Comment:
   Added a couple. Since its a pretty simple class I only added two - let me know if there's anything you think I'm missing. 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jsancio commented on a diff in pull request #12892: KAFKA-14386: Change ReplicaPlacer place method to return a class instead of a list of list of integers

Posted by GitBox <gi...@apache.org>.
jsancio commented on code in PR #12892:
URL: https://github.com/apache/kafka/pull/12892#discussion_r1036569941


##########
metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java:
##########
@@ -1564,16 +1567,17 @@ void createPartitions(CreatePartitionsTopic topic,
                 isrs.add(isr);
             }
         } else {
-            placements = clusterControl.replicaPlacer().place(new PlacementSpec(
+            partitionAssignments = clusterControl.replicaPlacer().place(new PlacementSpec(
                 startPartitionId,
                 additional,
                 replicationFactor
-            ), clusterDescriber);
-            isrs = placements;
+            ), clusterDescriber).assignments();

Review Comment:
   I think we can improve the formatting with something like:
   ```java
   partitionAssignments = clusterControl.replicaPlacer().place(
       new PlacementSpec(...),
       clusterDescriber
   ).assignments()
   ```



##########
metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java:
##########
@@ -1564,16 +1567,17 @@ void createPartitions(CreatePartitionsTopic topic,
                 isrs.add(isr);
             }
         } else {
-            placements = clusterControl.replicaPlacer().place(new PlacementSpec(
+            partitionAssignments = clusterControl.replicaPlacer().place(new PlacementSpec(
                 startPartitionId,
                 additional,
                 replicationFactor
-            ), clusterDescriber);
-            isrs = placements;
+            ), clusterDescriber).assignments();
+            isrs = partitionAssignments.stream().map(x -> x.replicas()).collect(Collectors.toList());

Review Comment:
   How about:
   ```java
               isrs = partitionAssignments.stream().map(PartitionAssignment::replicas).collect(Collectors.toList());
   ```



##########
metadata/src/main/java/org/apache/kafka/metadata/placement/PartitionAssignment.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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.metadata.placement;
+
+import java.util.List;
+
+/**
+ * The KRaft partition assignment.

Review Comment:
   I would remove the reference to KRaft.
   > The partition placement.



##########
metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java:
##########
@@ -407,13 +405,8 @@ public void testPlaceReplicas(int numUsableBrokers) throws Exception {
                 new PlacementSpec(0,
                     1,
                     (short) 3),
-                new ClusterDescriber() {
-                    @Override
-                    public Iterator<UsableBroker> usableBrokers() {
-                        return clusterControl.usableBrokers();
-                    }
-                }
-            );
+                    () -> clusterControl.usableBrokers()

Review Comment:
   How about:
   ```java
                       clusterControl::usableBrokers
   ```



##########
metadata/src/test/java/org/apache/kafka/metadata/placement/StripedReplicaPlacerTest.java:
##########
@@ -101,7 +102,7 @@ public Iterator<UsableBroker> usableBrokers() {
                 return brokers.iterator();
             }
         };
-        return placer.place(placementSpec, cluster);
+        return placer.place(placementSpec, cluster).assignments().stream().map(x -> x.replicas()).collect(Collectors.toList());

Review Comment:
   Did you consider changing the signature of this method to return `TopicPlacement` and adjusting the test above to use that type?



##########
metadata/src/main/java/org/apache/kafka/metadata/placement/StripedReplicaPlacer.java:
##########
@@ -440,6 +441,6 @@ public List<List<Integer>> place(
         for (int partition = 0; partition < placement.numPartitions(); partition++) {
             placements.add(rackList.place(placement.numReplicas()));
         }
-        return placements;
+        return new TopicAssignment(placements.stream().map(x -> new PartitionAssignment(x)).collect(Collectors.toList()));

Review Comment:
   How about:
   ```java
           return new TopicAssignment(
               placements.stream().map(PartitionAssignment::new).collect(Collectors.toList())
           );
   ```



##########
metadata/src/main/java/org/apache/kafka/metadata/placement/PartitionAssignment.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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.metadata.placement;
+
+import java.util.List;
+
+/**
+ * The KRaft partition assignment.
+ *
+ * The assignment is represented as a list of integers where each integer is the replica ID.
+ */
+public class PartitionAssignment {
+    private List<Integer> replicas;
+
+    public PartitionAssignment(List<Integer> replicas) {
+        this.replicas = replicas;

Review Comment:
   We should make this type immutable.



##########
metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java:
##########
@@ -407,13 +405,8 @@ public void testPlaceReplicas(int numUsableBrokers) throws Exception {
                 new PlacementSpec(0,
                     1,
                     (short) 3),
-                new ClusterDescriber() {
-                    @Override
-                    public Iterator<UsableBroker> usableBrokers() {
-                        return clusterControl.usableBrokers();
-                    }
-                }
-            );
+                    () -> clusterControl.usableBrokers()
+            ).assignments().stream().map(x -> x.replicas()).collect(Collectors.toList());

Review Comment:
   Did you consider changing the type of `results` to `TopicAssignment` and fixing the `for` loop below?



##########
metadata/src/main/java/org/apache/kafka/metadata/placement/TopicAssignment.java:
##########
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.metadata.placement;
+
+import java.util.List;
+
+/**
+ * The KRaft topic assignment.
+ */
+public class TopicAssignment {
+    private List<PartitionAssignment> assignments;
+
+    public TopicAssignment(List<PartitionAssignment> assignments) {
+        this.assignments = assignments;

Review Comment:
   We should make this type immutable.



##########
metadata/src/main/java/org/apache/kafka/metadata/placement/TopicAssignment.java:
##########
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.metadata.placement;
+
+import java.util.List;
+
+/**
+ * The KRaft topic assignment.

Review Comment:
   I would remove the reference to KRaft. The package name is `org.apache.kafka.metadata.placement` so it is unrelated to KRaft.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jsancio commented on a diff in pull request #12892: KAFKA-14386: Change ReplicaPlacer place method to return a class instead of a list of list of integers

Posted by GitBox <gi...@apache.org>.
jsancio commented on code in PR #12892:
URL: https://github.com/apache/kafka/pull/12892#discussion_r1037576847


##########
metadata/src/test/java/org/apache/kafka/metadata/placement/PartitionAssignmentTest.java:
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.metadata.placement;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.List;
+
+public class PartitionAssignmentTest {
+
+    @Test
+    public void testPartitionAssignmentReplicas() {
+        List<Integer> replicas = Arrays.asList(0, 1, 2);
+        assertEquals(replicas, new PartitionAssignment(replicas).replicas());
+    }
+
+    @Test
+    public void testConsistentEqualsAndHashCode() {
+        List<PartitionAssignment> partitionAssignments = Arrays.asList(
+            new PartitionAssignment(
+                    Arrays.asList(0, 1, 2)
+            ),
+            new PartitionAssignment(
+                    Arrays.asList(1, 2, 0)
+            )
+        );
+
+        for (int i = 0; i < partitionAssignments.size(); i++) {
+            for (int j = 0; j < partitionAssignments.size(); j++) {
+                if (i == j) {
+                    assertEquals(partitionAssignments.get(i), partitionAssignments.get(j));

Review Comment:
   Right but this test is uninteresting if the `equals` implementation is optimized to compare the references first.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] andymg3 commented on a diff in pull request #12892: KAFKA-14386: Change ReplicaPlacer place method to return a class instead of a list of list of integers

Posted by GitBox <gi...@apache.org>.
andymg3 commented on code in PR #12892:
URL: https://github.com/apache/kafka/pull/12892#discussion_r1037585759


##########
metadata/src/test/java/org/apache/kafka/metadata/placement/PartitionAssignmentTest.java:
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.metadata.placement;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.List;
+
+public class PartitionAssignmentTest {
+
+    @Test
+    public void testPartitionAssignmentReplicas() {
+        List<Integer> replicas = Arrays.asList(0, 1, 2);
+        assertEquals(replicas, new PartitionAssignment(replicas).replicas());
+    }
+
+    @Test
+    public void testConsistentEqualsAndHashCode() {
+        List<PartitionAssignment> partitionAssignments = Arrays.asList(
+            new PartitionAssignment(
+                    Arrays.asList(0, 1, 2)
+            ),
+            new PartitionAssignment(
+                    Arrays.asList(1, 2, 0)
+            )
+        );
+
+        for (int i = 0; i < partitionAssignments.size(); i++) {
+            for (int j = 0; j < partitionAssignments.size(); j++) {
+                if (i == j) {
+                    assertEquals(partitionAssignments.get(i), partitionAssignments.get(j));

Review Comment:
   Thats true. If `equals` is updated to compare references we arent checking the rest of the code. I can add into another assert like:
   ```
   assertEquals(partitionAssignments.get(i), new PartitionAssignment(partitionAssignments.get(i).replicas()));
   ```
   Does that make sense to you? 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] andymg3 commented on a diff in pull request #12892: KAFKA-14386: Return TopicAssignment from the ReplicaPlacer

Posted by GitBox <gi...@apache.org>.
andymg3 commented on code in PR #12892:
URL: https://github.com/apache/kafka/pull/12892#discussion_r1037646184


##########
metadata/src/main/java/org/apache/kafka/metadata/placement/TopicAssignment.java:
##########
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.metadata.placement;
+
+import java.util.List;
+
+/**
+ * The KRaft topic assignment.
+ */
+public class TopicAssignment {
+    private List<PartitionAssignment> assignments;
+
+    public TopicAssignment(List<PartitionAssignment> assignments) {
+        this.assignments = assignments;

Review Comment:
   Updated to:
   ```
       public TopicAssignment(final List<PartitionAssignment> assignments) {
           this.assignments = Collections.unmodifiableList(new ArrayList<>(assignments));
       }
   ```
   Also commented on both this class and `PartitionAssignment` that they're immutable.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] andymg3 commented on a diff in pull request #12892: KAFKA-14386: Change ReplicaPlacer place method to return a class instead of a list of list of integers

Posted by GitBox <gi...@apache.org>.
andymg3 commented on code in PR #12892:
URL: https://github.com/apache/kafka/pull/12892#discussion_r1037477576


##########
metadata/src/test/java/org/apache/kafka/metadata/placement/StripedReplicaPlacerTest.java:
##########
@@ -95,12 +94,7 @@ private List<List<Integer>> place(
         PlacementSpec placementSpec = new PlacementSpec(startPartition,
             numPartitions,
             replicationFactor);
-        ClusterDescriber cluster = new ClusterDescriber() {
-            @Override
-            public Iterator<UsableBroker> usableBrokers() {
-                return brokers.iterator();
-            }
-        };
+        ClusterDescriber cluster = () -> brokers.iterator();

Review Comment:
   Same as above. That actually doesn't compile:
   ```
   > Task :metadata:compileTestJava FAILED
   ... /kafka/metadata/src/test/java/org/apache/kafka/metadata/placement/StripedReplicaPlacerTest.java:97: error: incompatible types: bad return type in lambda expression
           ClusterDescriber cluster = () -> brokers::iterator;
                                            ^
       java.util.Iterator is not a functional interface
         multiple non-overriding abstract methods found in interface java.util.Iterator
   1 error
   ```



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jsancio commented on a diff in pull request #12892: KAFKA-14386: Change ReplicaPlacer place method to return a class instead of a list of list of integers

Posted by GitBox <gi...@apache.org>.
jsancio commented on code in PR #12892:
URL: https://github.com/apache/kafka/pull/12892#discussion_r1036579333


##########
metadata/src/main/java/org/apache/kafka/metadata/placement/TopicAssignment.java:
##########
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.metadata.placement;
+
+import java.util.List;
+
+/**
+ * The KRaft topic assignment.
+ */
+public class TopicAssignment {

Review Comment:
   Let's add unittests for this type.



##########
metadata/src/main/java/org/apache/kafka/metadata/placement/PartitionAssignment.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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.metadata.placement;
+
+import java.util.List;
+
+/**
+ * The KRaft partition assignment.
+ *
+ * The assignment is represented as a list of integers where each integer is the replica ID.
+ */
+public class PartitionAssignment {

Review Comment:
   Let's add unittests for this type.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] andymg3 commented on pull request #12892: KAFKA-14386: Change ReplicaPlacer place method to return a class instead of a list of list of integers

Posted by GitBox <gi...@apache.org>.
andymg3 commented on PR #12892:
URL: https://github.com/apache/kafka/pull/12892#issuecomment-1333944951

   > Thanks for the changes @andymg3 .
   > 
   > Should we also improve the `PartitionRegistration` type to use a `PartitionAssignment` instead of `int[] replicas`?
   
   Thanks for the review @jsancio. As of now I dont see much benefit in changing that type. In `PartitionRegistration` we also have `[]int isr`, `int[] removingReplicas`, and `int[] addingReplicas`. I'm included to leave `replicas` as is for consistency - I think it reads a bit better when they're all the same type. They're often processed at the same time as well, so having to deal with both types may be a bit cumbersome. There are quite a few places in the codebase that use the `replicas` field and in practice if we were to use `PartitionAssignment`  all callers would mostly just call `PartitionAssignment.replicas()` anyways. So overall I don't see much benefit in introducing the type. 
   
   What do you think? 


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] andymg3 commented on a diff in pull request #12892: KAFKA-14386: Change ReplicaPlacer place method to return a class instead of a list of list of integers

Posted by GitBox <gi...@apache.org>.
andymg3 commented on code in PR #12892:
URL: https://github.com/apache/kafka/pull/12892#discussion_r1037475316


##########
metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java:
##########
@@ -407,13 +405,8 @@ public void testPlaceReplicas(int numUsableBrokers) throws Exception {
                 new PlacementSpec(0,
                     1,
                     (short) 3),
-                new ClusterDescriber() {
-                    @Override
-                    public Iterator<UsableBroker> usableBrokers() {
-                        return clusterControl.usableBrokers();
-                    }
-                }
-            );
+                    () -> clusterControl.usableBrokers()

Review Comment:
   ```
   +++ b/metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java
   @@ -405,7 +405,7 @@ public class ClusterControlManagerTest {
                    new PlacementSpec(0,
                        1,
                        (short) 3),
   -                    () -> clusterControl.usableBrokers()
   +                    () -> clusterControl::usableBrokers
                ).assignments();
                HashSet<Integer> seen = new HashSet<>();
                for (Integer result : results.get(0).replicas()) {
   agrant@C02GN0121PG2 kafka % ./gradlew jar
   
   > Configure project :
   Starting build with version 3.4.0-SNAPSHOT (commit id 4ac9307e) using Gradle 7.6, Java 1.8 and Scala 2.13.8
   Build properties: maxParallelForks=12, maxScalacThreads=8, maxTestRetries=0
   
   > Task :metadata:compileTestJava FAILED
   /Users/agrant/workspace/kafka/metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java:408: error: incompatible types: bad return type in lambda expression
                       () -> clusterControl::usableBrokers
                             ^
       java.util.Iterator is not a functional interface
         multiple non-overriding abstract methods found in interface java.util.Iterator
   Note: Some messages have been simplified; recompile with -Xdiags:verbose to get full output
   1 error
   ```



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jsancio commented on a diff in pull request #12892: KAFKA-14386: Change ReplicaPlacer place method to return a class instead of a list of list of integers

Posted by GitBox <gi...@apache.org>.
jsancio commented on code in PR #12892:
URL: https://github.com/apache/kafka/pull/12892#discussion_r1037457258


##########
metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java:
##########
@@ -407,13 +405,8 @@ public void testPlaceReplicas(int numUsableBrokers) throws Exception {
                 new PlacementSpec(0,
                     1,
                     (short) 3),
-                new ClusterDescriber() {
-                    @Override
-                    public Iterator<UsableBroker> usableBrokers() {
-                        return clusterControl.usableBrokers();
-                    }
-                }
-            );
+                    () -> clusterControl.usableBrokers()

Review Comment:
   Did you use `clusterControl::usableBrokers` or `clusterControl.usableBrokers()`?



##########
metadata/src/test/java/org/apache/kafka/metadata/placement/PartitionAssignmentTest.java:
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.metadata.placement;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.List;
+
+public class PartitionAssignmentTest {
+
+    @Test
+    public void testPartitionAssignmentReplicas() {
+        List<Integer> replicas = Arrays.asList(0, 1, 2);
+        assertEquals(replicas, new PartitionAssignment(replicas).replicas());
+    }
+
+    @Test
+    public void testConsistentEqualsAndHashCode() {
+        List<PartitionAssignment> partitionAssignments = Arrays.asList(
+            new PartitionAssignment(
+                    Arrays.asList(0, 1, 2)
+            ),
+            new PartitionAssignment(
+                    Arrays.asList(1, 2, 0)
+            )
+        );
+
+        for (int i = 0; i < partitionAssignments.size(); i++) {
+            for (int j = 0; j < partitionAssignments.size(); j++) {
+                if (i == j) {
+                    assertEquals(partitionAssignments.get(i), partitionAssignments.get(j));

Review Comment:
   This assumes that the implementation of `equals` doesn't compare the references to determine equality.
   
   This comment also applies to `TopicAssignmentTest`.



##########
metadata/src/main/java/org/apache/kafka/metadata/placement/PartitionAssignment.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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.metadata.placement;
+
+import java.util.List;
+
+/**
+ * The KRaft partition assignment.
+ *
+ * The assignment is represented as a list of integers where each integer is the replica ID.
+ */
+public class PartitionAssignment {
+    private List<Integer> replicas;
+
+    public PartitionAssignment(List<Integer> replicas) {
+        this.replicas = replicas;

Review Comment:
   This type is not immutable. The user can `this.replicas().add(100)`.



##########
metadata/src/test/java/org/apache/kafka/metadata/placement/StripedReplicaPlacerTest.java:
##########
@@ -95,12 +94,7 @@ private List<List<Integer>> place(
         PlacementSpec placementSpec = new PlacementSpec(startPartition,
             numPartitions,
             replicationFactor);
-        ClusterDescriber cluster = new ClusterDescriber() {
-            @Override
-            public Iterator<UsableBroker> usableBrokers() {
-                return brokers.iterator();
-            }
-        };
+        ClusterDescriber cluster = () -> brokers.iterator();

Review Comment:
   How about `brokers::iterator`?



##########
metadata/src/main/java/org/apache/kafka/metadata/placement/TopicAssignment.java:
##########
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.metadata.placement;
+
+import java.util.List;
+
+/**
+ * The KRaft topic assignment.
+ */
+public class TopicAssignment {
+    private List<PartitionAssignment> assignments;
+
+    public TopicAssignment(List<PartitionAssignment> assignments) {
+        this.assignments = assignments;

Review Comment:
   This type is not immutable. The user can `this.assignments().add(new PartitionAssignment(...))`.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] andymg3 commented on a diff in pull request #12892: KAFKA-14386: Return TopicAssignment from the ReplicaPlacer

Posted by GitBox <gi...@apache.org>.
andymg3 commented on code in PR #12892:
URL: https://github.com/apache/kafka/pull/12892#discussion_r1037646348


##########
metadata/src/main/java/org/apache/kafka/metadata/placement/PartitionAssignment.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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.metadata.placement;
+
+import java.util.List;
+
+/**
+ * The KRaft partition assignment.
+ *
+ * The assignment is represented as a list of integers where each integer is the replica ID.
+ */
+public class PartitionAssignment {
+    private List<Integer> replicas;
+
+    public PartitionAssignment(List<Integer> replicas) {
+        this.replicas = replicas;

Review Comment:
   Updated to:
   ```
       public PartitionAssignment(final List<Integer> replicas) {
           this.replicas = Collections.unmodifiableList(new ArrayList<>(replicas));
       }
   ```



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] andymg3 commented on a diff in pull request #12892: KAFKA-14386: Return TopicAssignment from the ReplicaPlacer

Posted by GitBox <gi...@apache.org>.
andymg3 commented on code in PR #12892:
URL: https://github.com/apache/kafka/pull/12892#discussion_r1037646348


##########
metadata/src/main/java/org/apache/kafka/metadata/placement/PartitionAssignment.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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.metadata.placement;
+
+import java.util.List;
+
+/**
+ * The KRaft partition assignment.
+ *
+ * The assignment is represented as a list of integers where each integer is the replica ID.
+ */
+public class PartitionAssignment {
+    private List<Integer> replicas;
+
+    public PartitionAssignment(List<Integer> replicas) {
+        this.replicas = replicas;

Review Comment:
   Updated to:
   ```
       public PartitionAssignment(final List<Integer> replicas) {
           this.replicas = Collections.unmodifiableList(replicas);
       }
   ```



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] andymg3 commented on a diff in pull request #12892: KAFKA-14386: Change ReplicaPlacer place method to return a class instead of a list of list of integers

Posted by GitBox <gi...@apache.org>.
andymg3 commented on code in PR #12892:
URL: https://github.com/apache/kafka/pull/12892#discussion_r1037481276


##########
metadata/src/main/java/org/apache/kafka/metadata/placement/TopicAssignment.java:
##########
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.metadata.placement;
+
+import java.util.List;
+
+/**
+ * The KRaft topic assignment.
+ */
+public class TopicAssignment {
+    private List<PartitionAssignment> assignments;
+
+    public TopicAssignment(List<PartitionAssignment> assignments) {
+        this.assignments = assignments;

Review Comment:
   I can update this to:
   ```
       public TopicAssignment(final List<PartitionAssignment> assignments) {
           this.assignments = Collections.unmodifiableList(assignments);
       }
   ```
   Does that make sense? 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jsancio commented on a diff in pull request #12892: KAFKA-14386: Change ReplicaPlacer place method to return a class instead of a list of list of integers

Posted by GitBox <gi...@apache.org>.
jsancio commented on code in PR #12892:
URL: https://github.com/apache/kafka/pull/12892#discussion_r1037576847


##########
metadata/src/test/java/org/apache/kafka/metadata/placement/PartitionAssignmentTest.java:
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.metadata.placement;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.List;
+
+public class PartitionAssignmentTest {
+
+    @Test
+    public void testPartitionAssignmentReplicas() {
+        List<Integer> replicas = Arrays.asList(0, 1, 2);
+        assertEquals(replicas, new PartitionAssignment(replicas).replicas());
+    }
+
+    @Test
+    public void testConsistentEqualsAndHashCode() {
+        List<PartitionAssignment> partitionAssignments = Arrays.asList(
+            new PartitionAssignment(
+                    Arrays.asList(0, 1, 2)
+            ),
+            new PartitionAssignment(
+                    Arrays.asList(1, 2, 0)
+            )
+        );
+
+        for (int i = 0; i < partitionAssignments.size(); i++) {
+            for (int j = 0; j < partitionAssignments.size(); j++) {
+                if (i == j) {
+                    assertEquals(partitionAssignments.get(i), partitionAssignments.get(j));

Review Comment:
   Right but this test is uninteresting if the `equals` implementation is optimized to compare the references first. E.g.
   ```java
   boolean equals(Object that) {
     if (this == that) return true;
     ...
   }



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jsancio commented on a diff in pull request #12892: KAFKA-14386: Change ReplicaPlacer place method to return a class instead of a list of list of integers

Posted by GitBox <gi...@apache.org>.
jsancio commented on code in PR #12892:
URL: https://github.com/apache/kafka/pull/12892#discussion_r1037581170


##########
metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java:
##########
@@ -407,13 +405,8 @@ public void testPlaceReplicas(int numUsableBrokers) throws Exception {
                 new PlacementSpec(0,
                     1,
                     (short) 3),
-                new ClusterDescriber() {
-                    @Override
-                    public Iterator<UsableBroker> usableBrokers() {
-                        return clusterControl.usableBrokers();
-                    }
-                }
-            );
+                    () -> clusterControl.usableBrokers()

Review Comment:
   You want
   ```
   +++ b/metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java
   @@ -405,7 +405,7 @@ public class ClusterControlManagerTest {
                    new PlacementSpec(0,
                        1,
                        (short) 3),
   -                    () -> clusterControl.usableBrokers()
   +                    clusterControl::usableBrokers
                ).assignments();
                HashSet<Integer> seen = new HashSet<>();
                for (Integer result : results.get(0).replicas()) {
   ```



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] andymg3 commented on a diff in pull request #12892: KAFKA-14386: Change ReplicaPlacer place method to return a class instead of a list of list of integers

Posted by GitBox <gi...@apache.org>.
andymg3 commented on code in PR #12892:
URL: https://github.com/apache/kafka/pull/12892#discussion_r1037485087


##########
metadata/src/test/java/org/apache/kafka/metadata/placement/PartitionAssignmentTest.java:
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.metadata.placement;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.List;
+
+public class PartitionAssignmentTest {
+
+    @Test
+    public void testPartitionAssignmentReplicas() {
+        List<Integer> replicas = Arrays.asList(0, 1, 2);
+        assertEquals(replicas, new PartitionAssignment(replicas).replicas());
+    }
+
+    @Test
+    public void testConsistentEqualsAndHashCode() {
+        List<PartitionAssignment> partitionAssignments = Arrays.asList(
+            new PartitionAssignment(
+                    Arrays.asList(0, 1, 2)
+            ),
+            new PartitionAssignment(
+                    Arrays.asList(1, 2, 0)
+            )
+        );
+
+        for (int i = 0; i < partitionAssignments.size(); i++) {
+            for (int j = 0; j < partitionAssignments.size(); j++) {
+                if (i == j) {
+                    assertEquals(partitionAssignments.get(i), partitionAssignments.get(j));

Review Comment:
   Correct. I think thats what we want? Looking at the other classes where we override `equals` I think they're the same? 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jsancio commented on a diff in pull request #12892: KAFKA-14386: Change ReplicaPlacer place method to return a class instead of a list of list of integers

Posted by GitBox <gi...@apache.org>.
jsancio commented on code in PR #12892:
URL: https://github.com/apache/kafka/pull/12892#discussion_r1037592590


##########
metadata/src/test/java/org/apache/kafka/metadata/placement/PartitionAssignmentTest.java:
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.metadata.placement;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.List;
+
+public class PartitionAssignmentTest {
+
+    @Test
+    public void testPartitionAssignmentReplicas() {
+        List<Integer> replicas = Arrays.asList(0, 1, 2);
+        assertEquals(replicas, new PartitionAssignment(replicas).replicas());
+    }
+
+    @Test
+    public void testConsistentEqualsAndHashCode() {
+        List<PartitionAssignment> partitionAssignments = Arrays.asList(
+            new PartitionAssignment(
+                    Arrays.asList(0, 1, 2)
+            ),
+            new PartitionAssignment(
+                    Arrays.asList(1, 2, 0)
+            )
+        );
+
+        for (int i = 0; i < partitionAssignments.size(); i++) {
+            for (int j = 0; j < partitionAssignments.size(); j++) {
+                if (i == j) {
+                    assertEquals(partitionAssignments.get(i), partitionAssignments.get(j));

Review Comment:
   Yes.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] andymg3 commented on a diff in pull request #12892: KAFKA-14386: Change ReplicaPlacer place method to return a class instead of a list of list of integers

Posted by GitBox <gi...@apache.org>.
andymg3 commented on code in PR #12892:
URL: https://github.com/apache/kafka/pull/12892#discussion_r1037585759


##########
metadata/src/test/java/org/apache/kafka/metadata/placement/PartitionAssignmentTest.java:
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.metadata.placement;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.List;
+
+public class PartitionAssignmentTest {
+
+    @Test
+    public void testPartitionAssignmentReplicas() {
+        List<Integer> replicas = Arrays.asList(0, 1, 2);
+        assertEquals(replicas, new PartitionAssignment(replicas).replicas());
+    }
+
+    @Test
+    public void testConsistentEqualsAndHashCode() {
+        List<PartitionAssignment> partitionAssignments = Arrays.asList(
+            new PartitionAssignment(
+                    Arrays.asList(0, 1, 2)
+            ),
+            new PartitionAssignment(
+                    Arrays.asList(1, 2, 0)
+            )
+        );
+
+        for (int i = 0; i < partitionAssignments.size(); i++) {
+            for (int j = 0; j < partitionAssignments.size(); j++) {
+                if (i == j) {
+                    assertEquals(partitionAssignments.get(i), partitionAssignments.get(j));

Review Comment:
   Thats true. If `equals` is updated to compare references we arent checking the rest of the code. I can add in another assert like:
   ```
   assertEquals(partitionAssignments.get(i), new PartitionAssignment(partitionAssignments.get(i).replicas()));
   ```
   Does that make sense to you? 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] andymg3 commented on a diff in pull request #12892: KAFKA-14386: Change ReplicaPlacer place method to return a class instead of a list of list of integers

Posted by GitBox <gi...@apache.org>.
andymg3 commented on code in PR #12892:
URL: https://github.com/apache/kafka/pull/12892#discussion_r1037588465


##########
metadata/src/test/java/org/apache/kafka/metadata/placement/StripedReplicaPlacerTest.java:
##########
@@ -95,12 +94,7 @@ private List<List<Integer>> place(
         PlacementSpec placementSpec = new PlacementSpec(startPartition,
             numPartitions,
             replicationFactor);
-        ClusterDescriber cluster = new ClusterDescriber() {
-            @Override
-            public Iterator<UsableBroker> usableBrokers() {
-                return brokers.iterator();
-            }
-        };
+        ClusterDescriber cluster = () -> brokers.iterator();

Review Comment:
   I simplified further to:
   ```
           PlacementSpec placementSpec = new PlacementSpec(startPartition,
               numPartitions,
               replicationFactor);
           return placer.place(placementSpec, brokers::iterator);
   ```



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jsancio merged pull request #12892: KAFKA-14386: Return TopicAssignment from the ReplicaPlacer

Posted by GitBox <gi...@apache.org>.
jsancio merged PR #12892:
URL: https://github.com/apache/kafka/pull/12892


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org