You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by GitBox <gi...@apache.org> on 2021/08/26 10:41:00 UTC

[GitHub] [pulsar] casuallc opened a new issue #11796: throw NPE when readEntry

casuallc opened a new issue #11796:
URL: https://github.com/apache/pulsar/issues/11796


   **function call chain:**
   
   **ManagedCursorImpl**
   
   ```java
   @Override
    public void asyncReadEntries(int numberOfEntriesToRead, long maxSizeBytes, ReadEntriesCallback callback,
                                 Object ctx, PositionImpl maxPosition) {
        checkArgument(numberOfEntriesToRead > 0);
        if (isClosed()) {
            callback.readEntriesFailed(new ManagedLedgerException
                    .CursorAlreadyClosedException("Cursor was already closed"), ctx);
            return;
        }
   
        int numOfEntriesToRead = applyMaxSizeCap(numberOfEntriesToRead, maxSizeBytes);
       
        PENDING_READ_OPS_UPDATER.incrementAndGet(this);
        OpReadEntry op = OpReadEntry.create(this, readPosition, numOfEntriesToRead, callback, ctx, maxPosition);
        ledger.asyncReadEntries(op);
    }
   ```
   
   **OpReadEntry**
   
   ```java
    public static OpReadEntry create(ManagedCursorImpl cursor, PositionImpl readPositionRef, int count,
               ReadEntriesCallback callback, Object ctx, PositionImpl maxPosition) {
   
        OpReadEntry op = RECYCLER.get();
        op.readPosition = cursor.ledger.startReadOperationOnLedger(readPositionRef, op);
        op.cursor = cursor;
        op.count = count;
        op.callback = callback;
        op.entries = Lists.newArrayList();
        if (maxPosition == null) {
            maxPosition = PositionImpl.latest;
        }
        op.maxPosition = maxPosition;
        op.ctx = ctx;
        op.nextReadPosition = PositionImpl.get(op.readPosition);
        return op;
    }
   ```
   
   **ManagedLedgerImpl**
   
   ````java
   PositionImpl startReadOperationOnLedger(PositionImpl position, OpReadEntry opReadEntry) {
        Long ledgerId = ledgers.ceilingKey(position.getLedgerId());
        if (null == ledgerId) {
            opReadEntry.readEntriesFailed(new ManagedLedgerException.NoMoreEntriesToReadException("The ceilingKey(K key) method is used to return the " +
                    "least key greater than or equal to the given key, or null if there is no such key"), null);
        }
   
   ```
    if (ledgerId != position.getLedgerId()) {
        // The ledger pointed by this position does not exist anymore. It was deleted because it was empty. We need
        // to skip on the next available ledger
        position = new PositionImpl(ledgerId, 0);
    }
   
    return position;
   ```
   
    }
   ````
   
   **OpReadEntry**
   
   ```java
   @Override
   public void readEntriesFailed(ManagedLedgerException exception, Object ctx) {
   	cursor.readOperationCompleted();
   	// ...
   }
   ```
   
   
   
   **description**
   if read ledger not in ledgers, there is a exception. 
   but OpReadEntry instance is not be initialized, variable cursor is null.
   function readEntriesFailed will throw NPE, and OpReadEntry is not recycle.


-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] casuallc commented on issue #11796: throw NPE when readEntry

Posted by GitBox <gi...@apache.org>.
casuallc commented on issue #11796:
URL: https://github.com/apache/pulsar/issues/11796#issuecomment-911068781


   > Did you see this error in production ?
   
   Nope.
   
   When I add some other functions in broker, this error appeared.
   
   ```java
   11:19:30.843 [broker-topic-workers-OrderedScheduler-7-0] ERROR org.apache.bookkeeper.common.util.SafeRunnable - Unexpected throwable caught 
   java.lang.NullPointerException: null
   	at org.apache.bookkeeper.mledger.impl.OpReadEntry.readEntriesFailed(OpReadEntry.java:94) ~[managed-ledger-2.8.0.jar:2.8.0]
   	at org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.startReadOperationOnLedger(ManagedLedgerImpl.java:2334) ~[managed-ledger-2.8.0.jar:2.8.0]
   	at org.apache.bookkeeper.mledger.impl.OpReadEntry.create(OpReadEntry.java:52) ~[managed-ledger-2.8.0.jar:2.8.0]
   	at org.apache.bookkeeper.mledger.impl.ManagedCursorImpl.asyncReadEntriesOrWait(ManagedCursorImpl.java:736) ~[managed-ledger-2.8.0.jar:2.8.0]
   	at org.apache.pulsar.broker.service.persistent.PersistentDispatcherSingleActiveConsumer.readMoreEntries(PersistentDispatcherSingleActiveConsumer.java:342) ~[pulsar-broker-2.8.0.jar:2.8.0]
   	at org.apache.pulsar.broker.service.persistent.PersistentDispatcherSingleActiveConsumer.lambda$null$3(PersistentDispatcherSingleActiveConsumer.java:232) ~[pulsar-broker-2.8.0.jar:2.8.0]
   	at org.apache.bookkeeper.mledger.util.SafeRun$1.safeRun(SafeRun.java:32) ~[managed-ledger-2.8.0.jar:2.8.0]
   	at org.apache.bookkeeper.common.util.SafeRunnable.run(SafeRunnable.java:36) [org.apache.bookkeeper-bookkeeper-common-4.14.1.jar:4.14.1]
   	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) [?:1.8.0_231]
   	at java.util.concurrent.FutureTask.run(FutureTask.java:266) [?:1.8.0_231]
   	at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180) [?:1.8.0_231]
   	at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293) [?:1.8.0_231]
   	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) [?:1.8.0_231]
   	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) [?:1.8.0_231]
   	at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) [io.netty-netty-common-4.1.63.Final.jar:4.1.63.Final]
   	at java.lang.Thread.run(Thread.java:748) [?:1.8.0_231]
   ```
   


-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] lhotari commented on issue #11796: throw NPE when readEntry

Posted by GitBox <gi...@apache.org>.
lhotari commented on issue #11796:
URL: https://github.com/apache/pulsar/issues/11796#issuecomment-914973746


   this might be a duplicate of #11282 .


-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] eolivelli commented on issue #11796: throw NPE when readEntry

Posted by GitBox <gi...@apache.org>.
eolivelli commented on issue #11796:
URL: https://github.com/apache/pulsar/issues/11796#issuecomment-910257820


   Did you see this error in production ?


-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] codelipenghui commented on issue #11796: throw NPE when readEntry

Posted by GitBox <gi...@apache.org>.
codelipenghui commented on issue #11796:
URL: https://github.com/apache/pulsar/issues/11796#issuecomment-1058886778


   The issue had no activity for 30 days, mark with Stale label.


-- 
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: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org