stankiewicz commented on code in PR #38603:
URL: https://github.com/apache/beam/pull/38603#discussion_r3310894849


##########
sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceReader.java:
##########
@@ -158,23 +163,30 @@ public boolean advance() {
 
   @Override
   public void close() {
-    finalizeReadyMessages();
     sessionServiceCache.invalidate(readerUuid);
+    ActiveReadersRegistry.unregister(readerUuid);
   }
 
-  public void finalizeReadyMessages() {
-    BytesXMLMessage msg;
-    while ((msg = safeToAckMessages.poll()) != null) {
+  void finalizeCheckpoint(long checkpointId) {
+    List<BytesXMLMessage> messagesToAck = new ArrayList<>();
+
+    synchronized (lock) {
+      SortedMap<Long, List<BytesXMLMessage>> toAck = 
pendingCheckpoints.headMap(checkpointId, true);
+      for (List<BytesXMLMessage> msgs : toAck.values()) {
+        messagesToAck.addAll(msgs);
+      }
+      toAck.clear();
+    }
+
+    for (BytesXMLMessage msg : messagesToAck) {
       try {
         msg.ackMessage();

Review Comment:
   It was raised that ackMessage may be slow, I'm curious if we could just add 
this  and make `ExecutorService ioExecutor = 
Executors.newFixedThreadPool(numberOfThreads);` configurable with some good 
default. 
   
   ```
   // 1. Map messages to asynchronous tasks
   CompletableFuture<?>[] futures = messagesToAck.stream()
       .map(msg -> CompletableFuture.runAsync(() -> {
           try {
               msg.ackMessage();
           } catch (IllegalStateException e) {
               // irrelevant
           }
       }, ioExecutor))
       .toArray(CompletableFuture[]::new);
   
   // 2. Wait for all of them to complete
   CompletableFuture.allOf(futures).join();
   ```
   



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