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


##########
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/sink/committables/CheckpointCommittableManagerImplTest.java:
##########
@@ -93,6 +95,32 @@ void testUpdateCommittableSummary() {
                 .hasMessageContaining("FLINK-25920");
     }
 
+    // check different values of subtaskId and numberOfSubtasks to make sure 
that no value is
+    // hardcoded.
+    @ParameterizedTest(name = "subtaskId = {0}, numberOfSubtasks = {1}, 
checkpointId = {2}")
+    @CsvSource({"1, 10, 100", "2, 20, 200", "3, 30, 300"})
+    public void testCopy(int subtaskId, int numberOfSubtasks, long 
checkpointId) {
+
+        final CheckpointCommittableManagerImpl<Integer> original =
+                new CheckpointCommittableManagerImpl<>(subtaskId, 
numberOfSubtasks, checkpointId);
+        original.upsertSummary(
+                new CommittableSummary<>(subtaskId, numberOfSubtasks, 
checkpointId, 1, 0, 0));
+
+        CheckpointCommittableManagerImpl<Integer> copy = original.copy();
+
+        assertThat(copy.getCheckpointId()).isEqualTo(checkpointId);
+        assertThat(

Review Comment:
   Nit: You can also use the `CommittableSummaryAssert` here and chain the 
assertions that is the beauty of it :)
   
   ```java
   assertThat(copy.getSummary)
      .hasSubtaskId(..)
      .hasNumberOfSubtasks(...)
      ....
   ```
   



##########
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/sink/committables/CommittableCollectorSerializerTest.java:
##########
@@ -72,38 +74,90 @@ void testCommittableCollectorV1SerDe() throws IOException {
 
     @Test
     void testCommittableCollectorV2SerDe() throws IOException {
-        final CommittableCollector<Integer> committableCollector = new 
CommittableCollector<>(2, 3);
-        committableCollector.addMessage(new CommittableSummary<>(2, 3, 1L, 1, 
1, 0));
-        committableCollector.addMessage(new CommittableSummary<>(2, 3, 2L, 1, 
1, 0));
-        committableCollector.addMessage(new CommittableWithLineage<>(1, 1L, 
2));
-        committableCollector.addMessage(new CommittableWithLineage<>(2, 2L, 
2));
+        int subtaskId = 2;
+        int numberOfSubtasks = 3;
+
+        final CommittableCollectorSerializer<Integer> ccSerializer =
+                new CommittableCollectorSerializer<>(
+                        COMMITTABLE_SERIALIZER, subtaskId, numberOfSubtasks);
+
+        final CommittableCollector<Integer> committableCollector =
+                new CommittableCollector<>(subtaskId, numberOfSubtasks);
+        committableCollector.addMessage(
+                new CommittableSummary<>(subtaskId, numberOfSubtasks, 1L, 1, 
1, 0));
+        committableCollector.addMessage(
+                new CommittableSummary<>(subtaskId, numberOfSubtasks, 2L, 1, 
1, 0));
+        committableCollector.addMessage(new CommittableWithLineage<>(1, 1L, 
subtaskId));
+        committableCollector.addMessage(new CommittableWithLineage<>(2, 2L, 
subtaskId));
+
         final CommittableCollector<Integer> copy =
-                SERIALIZER.deserialize(2, 
SERIALIZER.serialize(committableCollector));
+                ccSerializer.deserialize(2, 
SERIALIZER.serialize(committableCollector));
 
         // Expect the subtask Id equal to the origin of the collector
-        assertThat(copy.getSubtaskId()).isEqualTo(1);
+        assertThat(copy.getSubtaskId()).isEqualTo(subtaskId);
         assertThat(copy.isFinished()).isFalse();
-        assertThat(copy.getNumberOfSubtasks()).isEqualTo(1);
-        final Collection<CheckpointCommittableManagerImpl<Integer>> 
checkpointCommittables =
-                committableCollector.getCheckpointCommittables();
-        assertThat(checkpointCommittables).hasSize(2);
-        final Iterator<CheckpointCommittableManagerImpl<Integer>> 
committablesIterator =
-                checkpointCommittables.iterator();
-        final SubtaskCommittableManager<Integer> 
subtaskCommittableManagerCheckpoint1 =
-                committablesIterator.next().getSubtaskCommittableManager(2);
-        assertThat(
-                        subtaskCommittableManagerCheckpoint1
-                                .getPendingRequests()
-                                .map(CommitRequestImpl::getCommittable)
-                                .collect(Collectors.toList()))
-                .containsExactly(1);
-        final SubtaskCommittableManager<Integer> 
subtaskCommittableManagerCheckpoint2 =
-                committablesIterator.next().getSubtaskCommittableManager(2);
-        assertThat(
-                        subtaskCommittableManagerCheckpoint2
-                                .getPendingRequests()
-                                .map(CommitRequestImpl::getCommittable)
-                                .collect(Collectors.toList()))
-                .containsExactly(2);
+        assertThat(copy.getNumberOfSubtasks()).isEqualTo(numberOfSubtasks);
+
+        // assert original CommittableCollector
+        assertCommittableCollector(
+                "Original CommittableCollector", subtaskId, numberOfSubtasks, 
committableCollector);
+
+        // assert deserialized CommittableCollector
+        assertCommittableCollector(
+                "Deserialized CommittableCollector", subtaskId, 
numberOfSubtasks, copy);
+    }
+
+    private void assertCommittableCollector(
+            String assertMessageHeading,
+            int subtaskId,
+            int numberOfSubtasks,
+            CommittableCollector<Integer> committableCollector) {
+
+        assertAll(
+                assertMessageHeading,
+                () -> {
+                    final Collection<CheckpointCommittableManagerImpl<Integer>>
+                            checkpointCommittables =
+                                    
committableCollector.getCheckpointCommittables();
+                    assertThat(checkpointCommittables).hasSize(2);
+
+                    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);
+                    
SinkV2Assertions.assertThat(checkpointCommittableManager1.getSummary())
+                            .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);
+                    
SinkV2Assertions.assertThat(checkpointCommittableManager2.getSummary())
+                            .hasNumberOfSubtasks(numberOfSubtasks);

Review Comment:
   Please also chain the assertions.



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