Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Fix: handle invalid markDelete position at managed-cursor #1554

Merged
merged 2 commits into from
Apr 12, 2018

Conversation

rdhabalia
Copy link
Contributor

Motivation

If client acks with invalid message-id (message-id > ledger.lastConfirmedEntry) then broker doesn't do validation and it tries to process it which can corrupt the state of cursor.

eg: if newMarkDeletePosition is invalid then ledger.getNextValidPosition() returns null and broker stores readPosition=null

readPosition = ledger.getNextValidPosition(newMarkDeletePosition);

and it creates below exception:

19:17:28.298 [pulsar-io-72-47] WARN  o.a.b.mledger.impl.ManagedCursorImpl - [prop/global/ns/persistent/topic] [sub] Error while updating individualDeletedMessages [null]
java.lang.NullPointerException: null
        at com.google.common.base.Preconditions.checkNotNull(Preconditions.java:192) ~[guava-15.0.jar:na]
        at com.google.common.collect.TreeRangeSet.rangeContaining(TreeRangeSet.java:98) ~[guava-15.0.jar:na]
        at com.google.common.collect.AbstractRangeSet.contains(AbstractRangeSet.java:29) ~[guava-15.0.jar:na]
        at com.google.common.collect.TreeRangeSet.contains(TreeRangeSet.java:42) ~[guava-15.0.jar:na]
        at org.apache.bookkeeper.mledger.impl.ManagedCursorImpl.setAcknowledgedPosition(ManagedCursorImpl.java:1248) ~[managed-ledger-1.20.16-incubating-yahoo.jar:1.20.16-incubating-yahoo]
        at org.apache.bookkeeper.mledger.impl.ManagedCursorImpl.asyncDelete(ManagedCursorImpl.java:1541) ~[managed-ledger-1.20.16-incubating-yahoo.jar:1.20.16-incubating-yahoo]
        at org.apache.pulsar.broker.service.persistent.PersistentSubscription.acknowledgeMessage(PersistentSubscription.java:182) [pulsar-broker-1.20.16-incubating-yahoo.jar:1.20.16-incubating-yahoo]
        at org.apache.pulsar.broker.service.Consumer.messageAcked(Consumer.java:346) [pulsar-broker-1.20.16-incubating-yahoo.jar:1.20.16-incubating-yahoo]
        at org.apache.pulsar.broker.service.ServerCnx.handleAck(ServerCnx.java:859) [pulsar-broker-1.20.16-incubating-yahoo.jar:1.20.16-incubating-yahoo]

Modifications

validate mark-delete position before processing it.

Result

Prevents any state corruption at cursor.

@rdhabalia rdhabalia added the type/bug The PR fixed a bug or issue reported a bug label Apr 11, 2018
@rdhabalia rdhabalia added this to the 2.0.0-incubating milestone Apr 11, 2018
@rdhabalia rdhabalia self-assigned this Apr 11, 2018
@@ -1301,6 +1301,16 @@ public void asyncMarkDelete(final Position position, Map<String, Long> propertie
final MarkDeleteCallback callback, final Object ctx) {
checkNotNull(position);
checkArgument(position instanceof PositionImpl);

if (((PositionImpl) ledger.getLastConfirmedEntry()).compareTo((PositionImpl) position) <= 0) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Shouldn't this check for < 0 ?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

oops ..fixed it.

@merlimat merlimat merged commit e33795b into apache:master Apr 12, 2018
@merlimat merlimat added this to Proposed to backport in Pulsar 1.22.1 patch Release via automation Apr 23, 2018
@jai1 jai1 moved this from Proposed to backport to In Progress in Pulsar 1.22.1 patch Release May 2, 2018
@jai1 jai1 moved this from In Progress to Merged for 1.22.1 in Pulsar 1.22.1 patch Release May 2, 2018
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
type/bug The PR fixed a bug or issue reported a bug
Projects
No open projects
Development

Successfully merging this pull request may close these issues.

None yet

2 participants