nicoloboschi commented on code in PR #16545:
URL: https://github.com/apache/pulsar/pull/16545#discussion_r919965687


##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpScan.java:
##########
@@ -0,0 +1,124 @@
+/**
+ * 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.bookkeeper.mledger.impl;
+
+import com.google.common.base.Predicate;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicLong;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.AsyncCallbacks.ReadEntryCallback;
+import org.apache.bookkeeper.mledger.AsyncCallbacks.ScanCallback;
+import org.apache.bookkeeper.mledger.Entry;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.ScanOutcome;
+import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.PositionBound;
+
+@Slf4j
+class OpScan implements ReadEntryCallback {
+    private final ManagedCursorImpl cursor;
+    private final ManagedLedgerImpl ledger;
+    private final PositionImpl startPosition;
+    private final ScanCallback callback;
+    private final Predicate<Entry> condition;
+    private final Object ctx;
+    private final AtomicLong remainingEntries = new AtomicLong();
+    private final long timeOutMs;
+    private final long startTime = System.currentTimeMillis();
+
+    PositionImpl searchPosition;
+    Position lastMatchedPosition = null;
+
+    public OpScan(ManagedCursorImpl cursor, PositionImpl startPosition, 
Predicate<Entry> condition,
+                  ScanCallback callback, Object ctx, long maxEntries, long 
timeOutMs) {
+        this.cursor = cursor;
+        this.ledger = cursor.ledger;
+        this.startPosition = startPosition;
+        this.callback = callback;
+        this.condition = condition;
+        this.ctx = ctx;
+        this.searchPosition = startPosition;
+        this.remainingEntries.set(maxEntries);
+        this.timeOutMs = timeOutMs;
+    }
+
+
+    @Override
+    public void readEntryComplete(Entry entry, Object ctx) {
+        remainingEntries.decrementAndGet();
+        final Position position = entry.getPosition();
+        lastMatchedPosition = position;
+        // filter out the entry if it has been already deleted
+        // filterReadEntries will call entry.release if the entry is filtered 
out
+        List<Entry> entries = this.cursor.filterReadEntries(List.of(entry));
+        if (!entries.isEmpty()) {
+            boolean exit = false;
+            try {
+                if (!condition.apply(entry)) {
+                  exit = true;
+                }
+                entry.release();
+            } catch (Throwable err) {
+                log.error("[{}] user exception", cursor, err);

Review Comment:
   should we call `entry.release()` as well ? 



##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpScan.java:
##########
@@ -0,0 +1,124 @@
+/**
+ * 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.bookkeeper.mledger.impl;
+
+import com.google.common.base.Predicate;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicLong;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.AsyncCallbacks.ReadEntryCallback;
+import org.apache.bookkeeper.mledger.AsyncCallbacks.ScanCallback;
+import org.apache.bookkeeper.mledger.Entry;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.ScanOutcome;
+import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.PositionBound;
+
+@Slf4j
+class OpScan implements ReadEntryCallback {
+    private final ManagedCursorImpl cursor;
+    private final ManagedLedgerImpl ledger;
+    private final PositionImpl startPosition;
+    private final ScanCallback callback;
+    private final Predicate<Entry> condition;
+    private final Object ctx;
+    private final AtomicLong remainingEntries = new AtomicLong();
+    private final long timeOutMs;
+    private final long startTime = System.currentTimeMillis();
+
+    PositionImpl searchPosition;
+    Position lastMatchedPosition = null;
+
+    public OpScan(ManagedCursorImpl cursor, PositionImpl startPosition, 
Predicate<Entry> condition,
+                  ScanCallback callback, Object ctx, long maxEntries, long 
timeOutMs) {
+        this.cursor = cursor;
+        this.ledger = cursor.ledger;
+        this.startPosition = startPosition;
+        this.callback = callback;
+        this.condition = condition;
+        this.ctx = ctx;
+        this.searchPosition = startPosition;
+        this.remainingEntries.set(maxEntries);
+        this.timeOutMs = timeOutMs;
+    }
+
+
+    @Override
+    public void readEntryComplete(Entry entry, Object ctx) {
+        remainingEntries.decrementAndGet();
+        final Position position = entry.getPosition();
+        lastMatchedPosition = position;
+        // filter out the entry if it has been already deleted
+        // filterReadEntries will call entry.release if the entry is filtered 
out
+        List<Entry> entries = this.cursor.filterReadEntries(List.of(entry));
+        if (!entries.isEmpty()) {
+            boolean exit = false;
+            try {
+                if (!condition.apply(entry)) {
+                  exit = true;
+                }
+                entry.release();
+            } catch (Throwable err) {
+                log.error("[{}] user exception", cursor, err);
+                
callback.scanFailed(ManagedLedgerException.getManagedLedgerException(err),
+                        Optional.ofNullable(searchPosition), OpScan.this.ctx);
+            }
+            if (exit) {
+                // user code requested to stop our scan
+                callback.scanComplete(lastMatchedPosition, 
ScanOutcome.USER_INTERRUPTED, OpScan.this.ctx);
+                return;
+            }
+        }
+        searchPosition = ledger.getPositionAfterN((PositionImpl) position, 1, 
PositionBound.startExcluded);
+        if (log.isDebugEnabled()) {
+            log.debug("readEntryComplete {} at {} next is {}", entry, 
position, searchPosition);
+        }
+
+        if (searchPosition.compareTo((PositionImpl) position) == 0) {
+            // we have reached the end of the ledger, as we are not doing 
progress
+            callback.scanComplete(lastMatchedPosition, ScanOutcome.COMPLETED, 
OpScan.this.ctx);
+            return;
+        }
+        find();
+    }
+
+    @Override
+    public void readEntryFailed(ManagedLedgerException exception, Object ctx) {
+        callback.scanFailed(exception, Optional.ofNullable(searchPosition), 
OpScan.this.ctx);
+    }
+
+    public void find() {
+        if (remainingEntries.get() <= 0) {
+            log.warn("[{}] Scan abort after reading too many entries", 
OpScan.this.cursor);
+            callback.scanComplete(lastMatchedPosition, ScanOutcome.ABORTED, 
OpScan.this.ctx);

Review Comment:
   from a user perspective receiving ABORTED both for time out and too many 
entries is not useful. For instance, it could be useful for the user to know 
that there're too many entries and so there are more entries than the value 
passed to the scan.. so they can say "backlog is more than 100k entries"



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java:
##########
@@ -512,6 +522,95 @@ public String getTypeString() {
         return "Null";
     }
 
+    @Override
+    public CompletableFuture<AnaliseBacklogResult> analiseBacklog() {
+
+        long start = System.currentTimeMillis();
+        if (log.isDebugEnabled()) {
+            log.debug("[{}][{}] Starting to analise backlog", topicName, 
subName);
+        }
+
+        AtomicLong entries = new AtomicLong();
+        AtomicLong accepted = new AtomicLong();
+        AtomicLong rejected = new AtomicLong();
+        AtomicLong rescheduled = new AtomicLong();
+        AtomicLong messages = new AtomicLong();
+        AtomicLong acceptedMessages = new AtomicLong();
+        AtomicLong rejectedMessages = new AtomicLong();
+        AtomicLong rescheduledMessages = new AtomicLong();
+
+        Position currentPosition = cursor.getMarkDeletedPosition();
+
+        if (log.isDebugEnabled()) {
+            log.debug("[{}][{}] currentPosition {}",
+                    topicName, subName, currentPosition);
+        }
+        final AbstractBaseDispatcher abstractBaseDispatcher = dispatcher != 
null
+                ? (AbstractBaseDispatcher) dispatcher : new 
DummyDispatcherForFilters();
+        // we put some hard limits on the scan, in order to prevent denial of 
services
+        ServiceConfiguration configuration = 
topic.getBrokerService().getPulsar().getConfiguration();
+        long maxEntries = configuration.getSubscriptionBacklogScanMaxEntries();
+        long timeOutMs = configuration.getSubscriptionBacklogScanMaxTimeMs();
+        return cursor.scan(new Predicate<Entry>() {
+            @Override
+            public boolean apply(Entry entry) {
+                if (log.isDebugEnabled()) {
+                    log.debug("found {}", entry);
+                }
+
+                ByteBuf metadataAndPayload = entry.getDataBuffer();
+                MessageMetadata messageMetadata = 
Commands.peekMessageMetadata(metadataAndPayload, "", -1);
+                int numMessages = 1;
+                if (messageMetadata.hasNumMessagesInBatch()) {
+                    numMessages = messageMetadata.getNumMessagesInBatch();
+                }
+                EntryFilter.FilterResult filterResult = abstractBaseDispatcher
+                        .runFiltersForEntry(entry, messageMetadata, null);
+
+                if (filterResult == null) {
+                    filterResult = EntryFilter.FilterResult.ACCEPT;
+                }
+                switch (filterResult) {
+                    case REJECT:
+                        rejected.incrementAndGet();
+                        rejectedMessages.addAndGet(numMessages);
+                        break;
+                    case RESCHEDULE:
+                        rescheduled.incrementAndGet();
+                        rescheduledMessages.addAndGet(numMessages);
+                        break;
+                    default:
+                        accepted.incrementAndGet();
+                        acceptedMessages.addAndGet(numMessages);
+                        break;
+                }
+                entries.incrementAndGet();
+                messages.addAndGet(numMessages);
+
+                return true;
+            }
+        }, maxEntries, timeOutMs).thenApply((ScanOutcome outcome) -> {
+            long end = System.currentTimeMillis();
+            AnaliseBacklogResult result = new AnaliseBacklogResult();
+            result.setEntries(entries.get());
+            result.setMessages(messages.get());
+            result.setFilterAcceptedEntries(accepted.get());
+            result.setFilterAcceptedMessages(acceptedMessages.get());
+            result.setFilterRejectedEntries(rejected.get());
+            result.setFilterRejectedMessages(rejectedMessages.get());
+            result.setFilterRescheduledEntries(rescheduled.get());
+            result.setFilterRescheduledMessages(rescheduledMessages.get());
+            // sometimes we abort the execution due to a timeout or
+            // when we reach a maximum number of entries
+            result.setScanOutcome(outcome);
+            log.info(
+                    "[{}][{}] scan took {} ms - {}",
+                    topicName, subName, end, result);

Review Comment:
   end - start ? 



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java:
##########
@@ -2434,6 +2496,34 @@ protected void 
internalUpdateSubscriptionProperties(AsyncResponse asyncResponse,
         });
     }
 
+    protected void internalAnaliseSubscriptionBacklog(AsyncResponse 
asyncResponse, String subName,
+                                                      boolean authoritative) {
+        CompletableFuture<Void> future;
+        if (topicName.isGlobal()) {
+            future = validateGlobalNamespaceOwnershipAsync(namespaceName);
+        } else {
+            future = CompletableFuture.completedFuture(null);
+        }
+
+        future.thenCompose(__ -> validateTopicOwnershipAsync(topicName, 
authoritative)).thenAccept(__ -> {
+            if (!topicName.isPartitioned() && 
getPartitionedTopicMetadata(topicName,
+                    authoritative, false).partitions > 0) {
+                throw new RestException(Status.METHOD_NOT_ALLOWED,
+                        "Get message ID by timestamp on a partitioned topic is 
not allowed, "

Review Comment:
   this error doesn't make sense 



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java:
##########
@@ -1570,6 +1590,48 @@ private void 
internalDeleteSubscriptionForNonPartitionedTopic(AsyncResponse asyn
             });
     }
 
+    private void 
internalAnaliseSubscriptionBacklogForNonPartitionedTopic(AsyncResponse 
asyncResponse,
+                                                                          
String subName,
+                                                                          
boolean authoritative) {
+        validateTopicOwnershipAsync(topicName, authoritative)
+                .thenRun(() -> validateTopicOperation(topicName, 
TopicOperation.CONSUME))
+                .thenCompose(__ -> getTopicReferenceAsync(topicName))
+                .thenCompose(topic -> {
+                            Subscription sub = topic.getSubscription(subName);
+                            if (sub == null) {
+                                throw new RestException(Status.NOT_FOUND,
+                                        
getSubNotFoundErrorMessage(topicName.toString(), subName));
+                            }
+                            return sub.analiseBacklog();
+                        })
+                .thenAccept((AnaliseBacklogResult rawResult) -> {
+
+                        AnaliseSubscriptionBacklogResult result = new 
AnaliseSubscriptionBacklogResult();
+                        result.setEntries(rawResult.getEntries());
+                        result.setMessages(rawResult.getMessages());
+
+                        
result.setFilterAcceptedEntries(rawResult.getFilterAcceptedEntries());
+                        
result.setFilterRejectedEntries(rawResult.getFilterRejectedEntries());
+                        
result.setFilterRescheduledEntries(rawResult.getFilterRescheduledEntries());
+
+                        
result.setFilterAcceptedMessages(rawResult.getFilterAcceptedMessages());
+                        
result.setFilterRejectedMessages(rawResult.getFilterRejectedMessages());
+                        
result.setFilterRescheduledMessages(rawResult.getFilterRescheduledMessages());
+                        result.setAborted(rawResult.getScanOutcome() != 
ScanOutcome.COMPLETED);
+                        log.info("[{}] analiseBacklog topic {} subscription {} 
result {}", clientAppId(), subName,

Review Comment:
   could we add the time spent doing the analysis ? 



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java:
##########
@@ -2434,6 +2496,34 @@ protected void 
internalUpdateSubscriptionProperties(AsyncResponse asyncResponse,
         });
     }
 
+    protected void internalAnaliseSubscriptionBacklog(AsyncResponse 
asyncResponse, String subName,
+                                                      boolean authoritative) {
+        CompletableFuture<Void> future;
+        if (topicName.isGlobal()) {
+            future = validateGlobalNamespaceOwnershipAsync(namespaceName);
+        } else {
+            future = CompletableFuture.completedFuture(null);
+        }
+
+        future.thenCompose(__ -> validateTopicOwnershipAsync(topicName, 
authoritative)).thenAccept(__ -> {
+            if (!topicName.isPartitioned() && 
getPartitionedTopicMetadata(topicName,
+                    authoritative, false).partitions > 0) {
+                throw new RestException(Status.METHOD_NOT_ALLOWED,
+                        "Get message ID by timestamp on a partitioned topic is 
not allowed, "
+                                + "please try do it on specific topic 
partition");
+            }
+            
internalAnaliseSubscriptionBacklogForNonPartitionedTopic(asyncResponse, 
subName, authoritative);
+        }).exceptionally(ex -> {
+            // If the exception is not redirect exception we need to log it.
+            if (!isRedirectException(ex)) {
+                log.error("[{}] Failed to update subscription {} from topic 
{}",

Review Comment:
   Failed to analise ? 



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java:
##########
@@ -1671,6 +1671,42 @@ public void getSubscriptionProperties(
         }
     }
 
+    @GET
+    
@Path("/{tenant}/{namespace}/{topic}/subscription/{subName}/analiseBacklog")
+    @ApiOperation(value = "Analyses a subscription, by scanning all the 
unprocessed messages")

Review Comment:
   ```suggestion
       @ApiOperation(value = "Analyze a subscription by scanning all the 
unprocessed messages")
   ```



##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpScan.java:
##########
@@ -0,0 +1,124 @@
+/**
+ * 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.bookkeeper.mledger.impl;
+
+import com.google.common.base.Predicate;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicLong;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.AsyncCallbacks.ReadEntryCallback;
+import org.apache.bookkeeper.mledger.AsyncCallbacks.ScanCallback;
+import org.apache.bookkeeper.mledger.Entry;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.ScanOutcome;
+import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.PositionBound;
+
+@Slf4j
+class OpScan implements ReadEntryCallback {
+    private final ManagedCursorImpl cursor;
+    private final ManagedLedgerImpl ledger;
+    private final PositionImpl startPosition;
+    private final ScanCallback callback;
+    private final Predicate<Entry> condition;
+    private final Object ctx;
+    private final AtomicLong remainingEntries = new AtomicLong();
+    private final long timeOutMs;
+    private final long startTime = System.currentTimeMillis();
+
+    PositionImpl searchPosition;
+    Position lastMatchedPosition = null;
+
+    public OpScan(ManagedCursorImpl cursor, PositionImpl startPosition, 
Predicate<Entry> condition,
+                  ScanCallback callback, Object ctx, long maxEntries, long 
timeOutMs) {
+        this.cursor = cursor;
+        this.ledger = cursor.ledger;
+        this.startPosition = startPosition;
+        this.callback = callback;
+        this.condition = condition;
+        this.ctx = ctx;
+        this.searchPosition = startPosition;
+        this.remainingEntries.set(maxEntries);
+        this.timeOutMs = timeOutMs;
+    }
+
+
+    @Override
+    public void readEntryComplete(Entry entry, Object ctx) {
+        remainingEntries.decrementAndGet();
+        final Position position = entry.getPosition();
+        lastMatchedPosition = position;
+        // filter out the entry if it has been already deleted
+        // filterReadEntries will call entry.release if the entry is filtered 
out
+        List<Entry> entries = this.cursor.filterReadEntries(List.of(entry));
+        if (!entries.isEmpty()) {
+            boolean exit = false;
+            try {
+                if (!condition.apply(entry)) {
+                  exit = true;
+                }
+                entry.release();
+            } catch (Throwable err) {
+                log.error("[{}] user exception", cursor, err);
+                
callback.scanFailed(ManagedLedgerException.getManagedLedgerException(err),

Review Comment:
   if an exception is raised we're going to continue the scan, is it intended? 



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