fapaul commented on code in PR #21101:
URL: https://github.com/apache/flink/pull/21101#discussion_r998408316


##########
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/sink/committables/CommittableCollectorSerializerTest.java:
##########
@@ -134,55 +190,78 @@ void 
testAlignSubtaskCommittableManagerCheckpointWithCheckpointCommittableManage
                 .isEqualTo(committableManager.getCheckpointId());
     }
 
+    /**
+     * @param assertMessageHeading prefix used for assertion fail message.
+     * @param subtaskId subtaskId to get {@link SubtaskCommittableManager} 
from {@link
+     *     CheckpointCommittableManagerImpl}
+     * @param expectedNumberOfSubtasks expected number of subtasks for {@link 
CommittableSummary}
+     * @param committableCollector collector to get {@link 
CheckpointCommittableManager}s from.
+     * @param expectedCommittableSize expected number of {@link 
CheckpointCommittableManager}.
+     * @param expectedNumberOfPendingRequestsPerCommittable expected number of 
pending request per
+     *     {@link SubtaskCommittableManager}.
+     */
     private void assertCommittableCollector(
             String assertMessageHeading,
             int subtaskId,
-            int numberOfSubtasks,
-            CommittableCollector<Integer> committableCollector) {
+            int expectedNumberOfSubtasks,
+            CommittableCollector<Integer> committableCollector,
+            int expectedCommittableSize,
+            Object[] expectedNumberOfPendingRequestsPerCommittable) {
 
         assertAll(
                 assertMessageHeading,
                 () -> {
                     final Collection<CheckpointCommittableManagerImpl<Integer>>
                             checkpointCommittables =
                                     
committableCollector.getCheckpointCommittables();
-                    assertThat(checkpointCommittables).hasSize(2);
+                    
assertThat(checkpointCommittables).hasSize(expectedCommittableSize);
 
                     final Iterator<CheckpointCommittableManagerImpl<Integer>> 
committablesIterator =
                             checkpointCommittables.iterator();
-                    final CheckpointCommittableManagerImpl<Integer> 
checkpointCommittableManager1 =
-                            committablesIterator.next();
-                    final SubtaskCommittableManager<Integer> 
subtaskCommittableManagerCheckpoint1 =
-                            
checkpointCommittableManager1.getSubtaskCommittableManager(subtaskId);
-
-                    
SinkV2Assertions.assertThat(checkpointCommittableManager1.getSummary())
-                            .hasSubtaskId(subtaskId)
-                            .hasNumberOfSubtasks(numberOfSubtasks);
-                    assertThat(
-                                    subtaskCommittableManagerCheckpoint1
-                                            .getPendingRequests()
-                                            
.map(CommitRequestImpl::getCommittable)
-                                            .collect(Collectors.toList()))
-                            .containsExactly(1);
-                    
assertThat(subtaskCommittableManagerCheckpoint1.getSubtaskId())
-                            .isEqualTo(subtaskId);
-
-                    final CheckpointCommittableManagerImpl<Integer> 
checkpointCommittableManager2 =
-                            committablesIterator.next();
-                    final SubtaskCommittableManager<Integer> 
subtaskCommittableManagerCheckpoint2 =
-                            
checkpointCommittableManager2.getSubtaskCommittableManager(subtaskId);
-
-                    
SinkV2Assertions.assertThat(checkpointCommittableManager2.getSummary())
-                            .hasSubtaskId(subtaskId)
-                            .hasNumberOfSubtasks(numberOfSubtasks);
-                    assertThat(
-                                    subtaskCommittableManagerCheckpoint2
-                                            .getPendingRequests()
-                                            
.map(CommitRequestImpl::getCommittable)
-                                            .collect(Collectors.toList()))
-                            .containsExactly(2);
-                    
assertThat(subtaskCommittableManagerCheckpoint2.getSubtaskId())
-                            .isEqualTo(subtaskId);
+
+                    int i = 0;
+                    while (committablesIterator.hasNext()) {
+                        final CheckpointCommittableManagerImpl<Integer>
+                                checkpointCommittableManager = 
committablesIterator.next();
+
+                        final SubtaskCommittableManager<Integer> 
subtaskCommittableManager =
+                                
checkpointCommittableManager.getSubtaskCommittableManager(
+                                        subtaskId);
+
+                        
SinkV2Assertions.assertThat(checkpointCommittableManager.getSummary())
+                                .hasSubtaskId(subtaskId)
+                                .hasNumberOfSubtasks(expectedNumberOfSubtasks);
+
+                        assertPendingRequests(
+                                subtaskCommittableManager,
+                                
expectedNumberOfPendingRequestsPerCommittable[i++]);
+
+                        
assertThat(subtaskCommittableManager.getSubtaskId()).isEqualTo(subtaskId);
+                    }
                 });
     }
+
+    private void assertPendingRequests(
+            SubtaskCommittableManager<Integer> 
subtaskCommittableManagerCheckpoint,
+            Object expectedPendingRequestCount) {
+
+        if (expectedPendingRequestCount instanceof Integer) {
+            assertThat(
+                            subtaskCommittableManagerCheckpoint
+                                    .getPendingRequests()
+                                    .map(CommitRequestImpl::getCommittable)
+                                    .collect(Collectors.toList()))
+                    .containsExactly((Integer) expectedPendingRequestCount);
+        } else if (expectedPendingRequestCount instanceof Integer[]) {

Review Comment:
   Can't you use `containsExactlyElementsOf` and always pass a list of 
integers? 



##########
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/sink/committables/CommittableCollectorSerializerTest.java:
##########
@@ -134,55 +190,78 @@ void 
testAlignSubtaskCommittableManagerCheckpointWithCheckpointCommittableManage
                 .isEqualTo(committableManager.getCheckpointId());
     }
 
+    /**
+     * @param assertMessageHeading prefix used for assertion fail message.
+     * @param subtaskId subtaskId to get {@link SubtaskCommittableManager} 
from {@link
+     *     CheckpointCommittableManagerImpl}
+     * @param expectedNumberOfSubtasks expected number of subtasks for {@link 
CommittableSummary}
+     * @param committableCollector collector to get {@link 
CheckpointCommittableManager}s from.
+     * @param expectedCommittableSize expected number of {@link 
CheckpointCommittableManager}.
+     * @param expectedNumberOfPendingRequestsPerCommittable expected number of 
pending request per
+     *     {@link SubtaskCommittableManager}.
+     */
     private void assertCommittableCollector(
             String assertMessageHeading,
             int subtaskId,
-            int numberOfSubtasks,
-            CommittableCollector<Integer> committableCollector) {
+            int expectedNumberOfSubtasks,
+            CommittableCollector<Integer> committableCollector,
+            int expectedCommittableSize,
+            Object[] expectedNumberOfPendingRequestsPerCommittable) {
 
         assertAll(
                 assertMessageHeading,
                 () -> {
                     final Collection<CheckpointCommittableManagerImpl<Integer>>
                             checkpointCommittables =
                                     
committableCollector.getCheckpointCommittables();
-                    assertThat(checkpointCommittables).hasSize(2);
+                    
assertThat(checkpointCommittables).hasSize(expectedCommittableSize);
 
                     final Iterator<CheckpointCommittableManagerImpl<Integer>> 
committablesIterator =
                             checkpointCommittables.iterator();
-                    final CheckpointCommittableManagerImpl<Integer> 
checkpointCommittableManager1 =
-                            committablesIterator.next();
-                    final SubtaskCommittableManager<Integer> 
subtaskCommittableManagerCheckpoint1 =
-                            
checkpointCommittableManager1.getSubtaskCommittableManager(subtaskId);
-
-                    
SinkV2Assertions.assertThat(checkpointCommittableManager1.getSummary())
-                            .hasSubtaskId(subtaskId)
-                            .hasNumberOfSubtasks(numberOfSubtasks);
-                    assertThat(
-                                    subtaskCommittableManagerCheckpoint1
-                                            .getPendingRequests()
-                                            
.map(CommitRequestImpl::getCommittable)
-                                            .collect(Collectors.toList()))
-                            .containsExactly(1);
-                    
assertThat(subtaskCommittableManagerCheckpoint1.getSubtaskId())
-                            .isEqualTo(subtaskId);
-
-                    final CheckpointCommittableManagerImpl<Integer> 
checkpointCommittableManager2 =
-                            committablesIterator.next();
-                    final SubtaskCommittableManager<Integer> 
subtaskCommittableManagerCheckpoint2 =
-                            
checkpointCommittableManager2.getSubtaskCommittableManager(subtaskId);
-
-                    
SinkV2Assertions.assertThat(checkpointCommittableManager2.getSummary())
-                            .hasSubtaskId(subtaskId)
-                            .hasNumberOfSubtasks(numberOfSubtasks);
-                    assertThat(
-                                    subtaskCommittableManagerCheckpoint2
-                                            .getPendingRequests()
-                                            
.map(CommitRequestImpl::getCommittable)
-                                            .collect(Collectors.toList()))
-                            .containsExactly(2);
-                    
assertThat(subtaskCommittableManagerCheckpoint2.getSubtaskId())
-                            .isEqualTo(subtaskId);
+
+                    int i = 0;

Review Comment:
   Since the `expectedNumberOfPendingRequestsPerCommittable`  and 
`committableIterator` always have same size you can zip (guavas `Streams.zip`) 
them it also makes the loop like nice  



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to