提交 50311421 编写于 作者: E Ezequiel Lovelle 提交者: Matteo Merli

[issue #3975] Bugfix NPE on non durable consumer (#3988)

*Motivation*

Trying to fix #3975

When a reset of a cursor is performed with some timestamp on a non-durable
consumer the message finder will fail with null pointer exception due to
`cursor.getName()` being null.

*Modifications*

  - Add method overloading for `newNonDurableCursor()` with subscription name.
  - Fix method getNonDurableSubscription to call `newNonDurableCursor()` with
    proper subscription name
  - Add test to assert issue.
上级 6a8d8389
......@@ -172,6 +172,7 @@ public interface ManagedLedger {
* @return the new NonDurableCursor
*/
ManagedCursor newNonDurableCursor(Position startCursorPosition) throws ManagedLedgerException;
ManagedCursor newNonDurableCursor(Position startPosition, String subscriptionName) throws ManagedLedgerException;
/**
* Delete a ManagedCursor asynchronously.
......
......@@ -831,6 +831,16 @@ public class ManagedLedgerImpl implements ManagedLedger, CreateCallback {
return new NonDurableCursorImpl(bookKeeper, config, this, null, (PositionImpl) startCursorPosition);
}
@Override
public ManagedCursor newNonDurableCursor(Position startCursorPosition, String cursorName)
throws ManagedLedgerException {
checkManagedLedgerIsOpen();
checkFenced();
return new NonDurableCursorImpl(bookKeeper, config, this, cursorName,
(PositionImpl) startCursorPosition);
}
@Override
public Iterable<ManagedCursor> getCursors() {
return cursors;
......
......@@ -641,7 +641,7 @@ public class PersistentTopic implements Topic, AddEntryCallback {
Position startPosition = new PositionImpl(ledgerId, entryId);
ManagedCursor cursor = null;
try {
cursor = ledger.newNonDurableCursor(startPosition);
cursor = ledger.newNonDurableCursor(startPosition, subscriptionName);
} catch (ManagedLedgerException e) {
subscriptionFuture.completeExceptionally(e);
}
......
......@@ -34,7 +34,9 @@ import java.util.Set;
import java.util.concurrent.TimeUnit;
import org.apache.pulsar.client.impl.BatchMessageIdImpl;
import org.apache.pulsar.client.impl.MessageImpl;
import org.apache.pulsar.client.impl.ReaderImpl;
import org.apache.pulsar.common.policies.data.TopicStats;
import org.apache.pulsar.common.util.RelativeTimeUtil;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.testng.Assert;
......@@ -479,7 +481,7 @@ public class TopicReaderTest extends ProducerConsumerBase {
assertTrue(reader.hasMessageAvailable());
String readOut = new String(reader.readNext().getData());
assertTrue(readOut.equals(content));
assertEquals(content, readOut);
assertFalse(reader.hasMessageAvailable());
}
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册