BewareMyPower commented on code in PR #21417:
URL: https://github.com/apache/pulsar/pull/21417#discussion_r1519284840


##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/TableViewImpl.java:
##########
@@ -289,20 +373,27 @@ private void readAllExistingMessages(Reader<T> reader, 
CompletableFuture<Reader<
                                reader.getTopic(),
                                messagesRead,
                                durationMillis / 1000.0);
-                       future.complete(reader);
+                       future.complete(null);
                    }
                 });
     }
 
     private void readTailMessages(Reader<T> reader) {
         reader.readNextAsync()
                 .thenAccept(msg -> {
-                    handleMessage(msg);
-                    readTailMessages(reader);
+                    try {
+                        handleMessage(msg);
+                        checkAllFreshTask(msg);
+                        readTailMessages(reader);
+                    } finally {
+                        msg.release();
+                    }
                 }).exceptionally(ex -> {
                     if (ex.getCause() instanceof 
PulsarClientException.AlreadyClosedException) {
                         log.error("Reader {} was closed while reading tail 
messages.",
                                 reader.getTopic(), ex);
+                        // Fail all refresh request when no more messages can 
be read.
+                        refreshRequests.keySet().forEach(future -> 
future.completeExceptionally(ex));

Review Comment:
   You should clear `refreshRequests` here.
   
   ```java
                           refreshRequests.keySet().forEach(future -> {
                               refreshRequests.remove(future);
                               future.completeExceptionally(ex);
                           });
   ```



##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/TableViewImpl.java:
##########
@@ -180,96 +203,157 @@ public void close() throws PulsarClientException {
     }
 
     private void handleMessage(Message<T> msg) {
-        try {
-            if (msg.hasKey()) {
-                String key = msg.getKey();
-                T cur = msg.size() > 0 ? msg.getValue() : null;
-                if (log.isDebugEnabled()) {
-                    log.debug("Applying message from topic {}. key={} 
value={}",
+        readPositions.put(msg.getTopicName(), msg.getMessageId());
+        if (msg.hasKey()) {
+            String key = msg.getKey();
+            T cur = msg.size() > 0 ? msg.getValue() : null;
+            if (log.isDebugEnabled()) {
+                log.debug("Applying message from topic {}. key={} value={}",
+                        conf.getTopicName(),
+                        key,
+                        cur);
+            }
+
+            boolean update = true;
+            if (compactionStrategy != null) {
+                T prev = data.get(key);
+                update = !compactionStrategy.shouldKeepLeft(prev, cur);
+                if (!update) {
+                    log.info("Skipped the message from topic {}. key={} 
value={} prev={}",
                             conf.getTopicName(),
                             key,
-                            cur);
+                            cur,
+                            prev);
+                    compactionStrategy.handleSkippedMessage(key, cur);
                 }
+            }
 
-                boolean update = true;
-                if (compactionStrategy != null) {
-                    T prev = data.get(key);
-                    update = !compactionStrategy.shouldKeepLeft(prev, cur);
-                    if (!update) {
-                        log.info("Skipped the message from topic {}. key={} 
value={} prev={}",
-                                conf.getTopicName(),
-                                key,
-                                cur,
-                                prev);
-                        compactionStrategy.handleSkippedMessage(key, cur);
+            if (update) {
+                try {
+                    listenersMutex.lock();
+                    if (null == cur) {
+                        data.remove(key);
+                    } else {
+                        data.put(key, cur);
                     }
-                }
 
-                if (update) {
-                    try {
-                        listenersMutex.lock();
-                        if (null == cur) {
-                            data.remove(key);
-                        } else {
-                            data.put(key, cur);
-                        }
-
-                        for (BiConsumer<String, T> listener : listeners) {
-                            try {
-                                listener.accept(key, cur);
-                            } catch (Throwable t) {
-                                log.error("Table view listener raised an 
exception", t);
-                            }
+                    for (BiConsumer<String, T> listener : listeners) {
+                        try {
+                            listener.accept(key, cur);
+                        } catch (Throwable t) {
+                            log.error("Table view listener raised an 
exception", t);
                         }
-                    } finally {
-                        listenersMutex.unlock();
                     }
+                } finally {
+                    listenersMutex.unlock();
                 }
             }
-        } finally {
-            msg.release();
         }
     }
 
-    private CompletableFuture<Reader<T>> readAllExistingMessages(Reader<T> 
reader) {
+    @Override
+    public CompletableFuture<Void> refreshAsync() {
+        CompletableFuture<Void> completableFuture = new CompletableFuture<>();
+        reader.thenCompose(reader -> 
getLastMessageIds(reader).thenAccept(lastMessageIds -> {
+            // After get the response of lastMessageIds, put the future and 
result into `refreshMap`
+            // and then filter out partitions that has been read to the 
lastMessageID.
+            refreshRequests.put(completableFuture, lastMessageIds);
+            filterReceivedMessages(lastMessageIds);
+            // If there is no new messages, the refresh operation could be 
completed right now.
+            if (lastMessageIds.isEmpty()) {
+                refreshRequests.remove(completableFuture);
+                completableFuture.complete(null);
+            }
+        })).exceptionally(throwable -> {
+            completableFuture.completeExceptionally(throwable);
+            log.info("[{}] Refresh Tableview failed", 
this.conf.getTopicName(), throwable);

Review Comment:
   ```suggestion
   ```
   
   The caller side can get this exception so there is no need to print the 
error logs inside the library.



##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/TableViewImpl.java:
##########
@@ -289,20 +373,27 @@ private void readAllExistingMessages(Reader<T> reader, 
CompletableFuture<Reader<
                                reader.getTopic(),
                                messagesRead,
                                durationMillis / 1000.0);
-                       future.complete(reader);
+                       future.complete(null);
                    }
                 });
     }
 
     private void readTailMessages(Reader<T> reader) {
         reader.readNextAsync()
                 .thenAccept(msg -> {
-                    handleMessage(msg);
-                    readTailMessages(reader);
+                    try {
+                        handleMessage(msg);
+                        checkAllFreshTask(msg);
+                        readTailMessages(reader);

Review Comment:
   Move `readTailMessages(reader)` out of the try block because it does not 
depend on the `msg`.
   
   BTW, this PR involves many code changes to `handleMessage` because you moved 
the `try-finally` block out of the `handleMessage`. You can just add the logic 
of `checkAllFreshTask` at the end of `handleMessage` to avoid so many code 
changes.
   
   ```java
       private void handleMessage(Message<T> msg) {
           try {
               // ...
               checkAllFreshTask(msg);
           } finally {
               msg.release();
           }
       }
   ```



##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/TableViewImpl.java:
##########
@@ -180,96 +203,157 @@ public void close() throws PulsarClientException {
     }
 
     private void handleMessage(Message<T> msg) {
-        try {
-            if (msg.hasKey()) {
-                String key = msg.getKey();
-                T cur = msg.size() > 0 ? msg.getValue() : null;
-                if (log.isDebugEnabled()) {
-                    log.debug("Applying message from topic {}. key={} 
value={}",
+        readPositions.put(msg.getTopicName(), msg.getMessageId());
+        if (msg.hasKey()) {
+            String key = msg.getKey();
+            T cur = msg.size() > 0 ? msg.getValue() : null;
+            if (log.isDebugEnabled()) {
+                log.debug("Applying message from topic {}. key={} value={}",
+                        conf.getTopicName(),
+                        key,
+                        cur);
+            }
+
+            boolean update = true;
+            if (compactionStrategy != null) {
+                T prev = data.get(key);
+                update = !compactionStrategy.shouldKeepLeft(prev, cur);
+                if (!update) {
+                    log.info("Skipped the message from topic {}. key={} 
value={} prev={}",
                             conf.getTopicName(),
                             key,
-                            cur);
+                            cur,
+                            prev);
+                    compactionStrategy.handleSkippedMessage(key, cur);
                 }
+            }
 
-                boolean update = true;
-                if (compactionStrategy != null) {
-                    T prev = data.get(key);
-                    update = !compactionStrategy.shouldKeepLeft(prev, cur);
-                    if (!update) {
-                        log.info("Skipped the message from topic {}. key={} 
value={} prev={}",
-                                conf.getTopicName(),
-                                key,
-                                cur,
-                                prev);
-                        compactionStrategy.handleSkippedMessage(key, cur);
+            if (update) {
+                try {
+                    listenersMutex.lock();
+                    if (null == cur) {
+                        data.remove(key);
+                    } else {
+                        data.put(key, cur);
                     }
-                }
 
-                if (update) {
-                    try {
-                        listenersMutex.lock();
-                        if (null == cur) {
-                            data.remove(key);
-                        } else {
-                            data.put(key, cur);
-                        }
-
-                        for (BiConsumer<String, T> listener : listeners) {
-                            try {
-                                listener.accept(key, cur);
-                            } catch (Throwable t) {
-                                log.error("Table view listener raised an 
exception", t);
-                            }
+                    for (BiConsumer<String, T> listener : listeners) {
+                        try {
+                            listener.accept(key, cur);
+                        } catch (Throwable t) {
+                            log.error("Table view listener raised an 
exception", t);
                         }
-                    } finally {
-                        listenersMutex.unlock();
                     }
+                } finally {
+                    listenersMutex.unlock();
                 }
             }
-        } finally {
-            msg.release();
         }
     }
 
-    private CompletableFuture<Reader<T>> readAllExistingMessages(Reader<T> 
reader) {
+    @Override
+    public CompletableFuture<Void> refreshAsync() {
+        CompletableFuture<Void> completableFuture = new CompletableFuture<>();
+        reader.thenCompose(reader -> 
getLastMessageIds(reader).thenAccept(lastMessageIds -> {
+            // After get the response of lastMessageIds, put the future and 
result into `refreshMap`
+            // and then filter out partitions that has been read to the 
lastMessageID.
+            refreshRequests.put(completableFuture, lastMessageIds);
+            filterReceivedMessages(lastMessageIds);
+            // If there is no new messages, the refresh operation could be 
completed right now.
+            if (lastMessageIds.isEmpty()) {
+                refreshRequests.remove(completableFuture);
+                completableFuture.complete(null);
+            }
+        })).exceptionally(throwable -> {
+            completableFuture.completeExceptionally(throwable);

Review Comment:
   It seems you missed a remove call here.
   
   ```java
   refreshRequests.remove(completableFuture);
   ```



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