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

Incorrect msgBacklog value #5202

Open
shustsud opened this issue Sep 17, 2019 · 6 comments
Open

Incorrect msgBacklog value #5202

shustsud opened this issue Sep 17, 2019 · 6 comments
Labels
area/broker lifecycle/stale type/bug The PR fixed a bug or issue reported a bug

Comments

@shustsud
Copy link
Contributor

Describe the bug
Despite there are no messages in the backlog, positive value was set to "msgBacklog" in the response of partitioned-stats(API).

I think "msgBacklog" is calculated from "ManagedLedgerImpl.ENTRIES_ADDED_COUNTER_UPDATER" and "ManagedCursorImpl.messagesConsumedCounter".

long backlog = ManagedLedgerImpl.ENTRIES_ADDED_COUNTER_UPDATER.get(ledger) - messagesConsumedCounter;

Looking at the debug log, [subscriptionName1] appears to have the wrong "ManagedCursorImpl.messagesConsumedCounter" value.
In contrast, [subscriptionName2] looks correct.

Log:

12:08:44.575 [prometheus-stats-33-1] DEBUG o.a.b.mledger.impl.ManagedCursorImpl - [<topicName1>] Consumer <subscriptionName1> cursor ml-entries: 554102 -- deleted-counter: 549461 other counters: mdPos 27058740:10 rdPos 27058740:11
12:08:44.575 [prometheus-stats-33-1] DEBUG o.a.b.mledger.impl.ManagedCursorImpl - [<topicName1>] Consumer <subscriptionName2(replication)> cursor ml-entries: 554102 -- deleted-counter: 554102 other counters: mdPos 27058740:10 rdPos 27058740:11

Stats:

{
  "msgRateIn" : 3.283531005730665,
  "msgThroughputIn" : 1657.249768369514,
  "msgRateOut" : 3.2835310293187825,
  "msgThroughputOut" : 1657.2497802748082,
  "averageMsgSize" : 504.7157360406091,
  "storageSize" : 142581,
  "publishers" : [ ],
  "subscriptions" : {
    "<subscriptionName1>" : {
      "msgRateOut" : 3.2835310293187825,
      "msgThroughputOut" : 1657.2497802748082,
      "msgRateRedeliver" : 0.0,
      "msgBacklog" : 4677,
      "blockedSubscriptionOnUnackedMsgs" : false,
      "unackedMessages" : 0,
      "type" : "Exclusive",
      "activeConsumerName" : "4efd5",
      "msgRateExpired" : 0.0,
      "consumers" : [ {
        "msgRateOut" : 3.2835310293187825,
        "msgThroughputOut" : 1657.2497802748082,
        "msgRateRedeliver" : 0.0,
        "consumerName" : "4efd5",
        "availablePermits" : 946,
        "unackedMessages" : 0,
        "blockedConsumerOnUnackedMsgs" : false,
        "metadata" : { },
        "connectedSince" : "2019-09-02T13:29:21.891+09:00",
        "clientVersion" : "2.1.1-incubating",
        "address" : "/<ip1>:55920"
      } ]
    }
  },
  "replication" : {
    "<name>" : {
      "msgRateIn" : 3.283531005730665,
      "msgThroughputIn" : 1657.249768369514,
      "msgRateOut" : 0.0,
      "msgThroughputOut" : 0.0,
      "msgRateExpired" : 0.0,
      "replicationBacklog" : 0,
      "connected" : true,
      "replicationDelayInSeconds" : 0,
      "inboundConnection" : "/<ip2>:36792",
      "inboundConnectedSince" : "2019-08-29T11:36:48.612+09:00",
      "outboundConnection" : "[id: 0xb6500383, L:/<ip3>:50306 - R:<host2>/<ip2>:6651]",
      "outboundConnectedSince" : "2019-08-29T11:36:48.64+09:00"
    }
  },
  "deduplicationStatus" : "Disabled"
}

Stats-internal:

{
  "entriesAddedCounter" : 552514,
  "numberOfEntries" : 611632,
  "totalSize" : 1007353769,
  "currentLedgerEntries" : 15460,
  "currentLedgerSize" : 57848652,
  "lastLedgerCreatedTimestamp" : "2019-09-03T08:04:37.87+09:00",
  "waitingCursorsCount" : 2,
  "pendingAddEntriesCount" : 0,
  "lastConfirmedEntry" : "27010835:15459",
  "state" : "LedgerOpened",
  "ledgers" : [ {
    "ledgerId" : 25679299,
    "entries" : 2544,
    "size" : 11803597,
    "offloaded" : false
  }, {
    "ledgerId" : 25711440,
    "entries" : 11923,
    "size" : 42865506,
    "offloaded" : false
  ...(Omitted...)
  }, {
    "ledgerId" : 26992239,
    "entries" : 143,
    "size" : 658467,
    "offloaded" : false
  }, {
    "ledgerId" : 27010835,
    "entries" : 0,
    "size" : 0,
    "offloaded" : false
  } ],
  "cursors" : {
    "<subscriptionName1>" : {
      "markDeletePosition" : "27010835:15409",
      "readPosition" : "27010835:15460",
      "waitingReadOp" : true,
      "pendingReadOps" : 0,
      "messagesConsumedCounter" : 547823,
      "cursorLedger" : 27014015,
      "cursorLedgerLastEntry" : 426,
      "individuallyDeletedMessages" : "[]",
      "lastLedgerSwitchTimestamp" : "2019-09-03T09:34:14.211+09:00",
      "state" : "Open",
      "numberOfEntriesSinceFirstNotAckedMessage" : 51,
      "totalNonContiguousDeletedMessagesRange" : 0,
      "properties" : { }
    },
    "<subscriptionName2(replication)>" : {
      "markDeletePosition" : "27010835:15459",
      "readPosition" : "27010835:15460",
      "waitingReadOp" : true,
      "pendingReadOps" : 0,
      "messagesConsumedCounter" : 552514,
      "cursorLedger" : 26992238,
      "cursorLedgerLastEntry" : 848,
      "individuallyDeletedMessages" : "[]",
      "lastLedgerSwitchTimestamp" : "2019-09-03T04:00:02.255+09:00",
      "state" : "Open",
      "numberOfEntriesSinceFirstNotAckedMessage" : 1,
      "totalNonContiguousDeletedMessagesRange" : 0,
      "properties" : { }
    }
  }
}

To Reproduce
We suspect that reset-cursor operations which were executed right before caused the problem.

11:08 "reset-cursor (API)" was called. (11 days ago)
11:11 "reset-cursor (API)" was called. (1 minute ago)

The log at the time of operations is as follows.

Log:

11:08:08.171 [pulsar-web-26-32] INFO  o.a.p.b.a.impl.PersistentTopicsBase  - [<serviceName1>] [persistent://<topicName1>] Received reset cursor on subscription <subscriptionName1> to time 1566439687935
11:08:08.199 [bookkeeper-ml-workers-OrderedExecutor-22-0] INFO  o.a.p.b.s.p.PersistentSubscription   - [persistent://<topicName1>][<subscriptionName1>] Unable to find position for timestamp 1566439687935. Resetting cursor to first position 25488301:0 in ledger
11:08:08.200 [bookkeeper-ml-workers-OrderedExecutor-22-0] INFO  o.a.pulsar.broker.service.Consumer   - Disconnecting consumer: Consumer{subscription=PersistentSubscription{topic=persistent://<topicName1>, name=<subscriptionName1>}, consumerId=4, consumerName=7740a, address=/<IP>:50796}
11:08:08.200 [bookkeeper-ml-workers-OrderedExecutor-22-0] INFO  o.a.p.b.s.AbstractDispatcherSingleActiveConsumer - Removing consumer Consumer{subscription=PersistentSubscription{topic=persistent://<topicName1>, name=<subscriptionName1>}, consumerId=4, consumerName=7740a, address=/<IP>:50796}
11:08:08.200 [bookkeeper-ml-workers-OrderedExecutor-22-0] INFO  o.a.p.b.s.p.PersistentSubscription   - [persistent://<topicName1>][<subscriptionName1>] Successfully disconnected consumers from subscription, proceeding with cursor reset
11:08:08.200 [bookkeeper-ml-workers-OrderedExecutor-22-0] INFO  o.a.b.mledger.impl.ManagedCursorImpl - [<topicName1>] Initiate reset position to 25488301:0 on cursor <subscriptionName1>
11:08:08.202 [BookKeeperClientWorker-OrderedExecutor-5-0] INFO  o.a.b.mledger.impl.ManagedCursorImpl - [<topicName1>] reset position to 25488301:0 before current read position 26825736:4099 on cursor <subscriptionName1>
11:08:08.202 [pulsar-web-26-32] INFO  o.a.p.b.a.impl.PersistentTopicsBase  - [serviceName1] [persistent://<topicName1>] Reset cursor on subscription <subscriptionName1> to time 1566439687935
11:08:08.301 [ForkJoinPool.commonPool-worker-16] INFO  o.a.pulsar.broker.service.ServerCnx  - [/<IP>:50796] Subscribing on topic persistent://<topicName1> / <subscriptionName1>
11:08:08.301 [ForkJoinPool.commonPool-worker-16] INFO  o.a.b.mledger.impl.ManagedCursorImpl - [<topicName1>-<subscriptionName1>] Rewind from 25488301:0 to 25488301:0
11:08:08.301 [ForkJoinPool.commonPool-worker-16] INFO  o.a.p.b.s.persistent.PersistentTopic - [persistent://<topicName1>][<subscriptionName1>] Created new subscription for 4
11:08:08.301 [ForkJoinPool.commonPool-worker-16] INFO  o.a.pulsar.broker.service.ServerCnx  - [/<IP>:50796] Created subscription on topic persistent://<topicName1> / <subscriptionName1>

11:11:28.788 [pulsar-web-26-14] INFO  o.a.p.b.a.impl.PersistentTopicsBase  - [serviceName1] [persistent://<topicName1>] Received reset cursor on subscription <subscriptionName1> to time 1567390228536
11:11:29.261 [bookkeeper-ml-workers-OrderedExecutor-22-0] INFO  o.a.p.b.s.p.PersistentMessageFinder  - [persistent://<topicName1>][<subscriptionName1>] Found position 26825736:4100 closest to provided timestamp 1567390228536
11:11:29.261 [bookkeeper-ml-workers-OrderedExecutor-22-0] INFO  o.a.pulsar.broker.service.Consumer   - Disconnecting consumer: Consumer{subscription=PersistentSubscription{topic=persistent://<topicName1>, name=<subscriptionName1>}, consumerId=4, consumerName=7740a, address=/<IP>:50796}
11:11:29.261 [bookkeeper-ml-workers-OrderedExecutor-22-0] INFO  o.a.p.b.s.AbstractDispatcherSingleActiveConsumer - Removing consumer Consumer{subscription=PersistentSubscription{topic=persistent://<topicName1>, name=<subscriptionName1>}, consumerId=4, consumerName=7740a, address=/<IP>:50796}
11:11:29.261 [bookkeeper-ml-workers-OrderedExecutor-22-0] INFO  o.a.p.b.s.p.PersistentSubscription   - [persistent://<topicName1>][<subscriptionName1>] Successfully disconnected consumers from subscription, proceeding with cursor reset
11:11:29.261 [bookkeeper-ml-workers-OrderedExecutor-22-0] INFO  o.a.b.mledger.impl.ManagedCursorImpl - [<topicName1>] Initiate reset position to 26825736:4100 on cursor <subscriptionName1>
11:11:29.263 [BookKeeperClientWorker-OrderedExecutor-5-0] INFO  o.a.b.mledger.impl.ManagedCursorImpl - [<topicName1>] reset position to 26825736:4100 skipping from current read position 25488301:7000 on cursor <subscriptionName1>
11:11:29.263 [pulsar-web-26-14] INFO  o.a.p.b.a.impl.PersistentTopicsBase  - [serviceName1] [persistent://<topicName1>] Reset cursor on subscription <subscriptionName1> to time 1567390228536
11:11:29.264 [bookkeeper-ml-workers-OrderedExecutor-22-0] INFO  o.a.b.mledger.impl.ManagedLedgerImpl - [<topicName1>] End TrimConsumedLedgers. ledgers=46 totalSize=925996468
11:11:29.264 [bookkeeper-ml-workers-OrderedExecutor-22-0] INFO  o.a.b.mledger.impl.ManagedLedgerImpl - [<topicName1>] Removing ledger 25488301 - size: 37740040
11:11:29.363 [ForkJoinPool.commonPool-worker-32] INFO  o.a.pulsar.broker.service.ServerCnx  - [/<IP>:50796] Subscribing on topic persistent://<topicName1> / <subscriptionName1>
11:11:29.363 [ForkJoinPool.commonPool-worker-32] INFO  o.a.b.mledger.impl.ManagedCursorImpl - [<topicName1>-<subscriptionName1>] Rewind from 26825736:4100 to 26825736:4100
11:11:29.363 [ForkJoinPool.commonPool-worker-32] INFO  o.a.p.b.s.persistent.PersistentTopic - [persistent://<topicName1>][<subscriptionName1>] Created new subscription for 4
11:11:29.363 [ForkJoinPool.commonPool-worker-32] INFO  o.a.pulsar.broker.service.ServerCnx  - [/<IP>:50796] Created subscription on topic persistent://<topicName1> / <subscriptionName1>

The partitioned topic has 16 partitions, but only one partition has a problem.

Pulsar version
broker: 2.3.2

@shustsud shustsud added the type/bug The PR fixed a bug or issue reported a bug label Sep 17, 2019
@codelipenghui
Copy link
Contributor

Seems like a bug which i see different subscription with same mark delete position but different delete count.

@shustsud If you are interested in fixing it, welcome!

@shustsud
Copy link
Contributor Author

@codelipenghui
thank you for your comment.

Add comments to the ticket if there is additional information.

@rocketraman
Copy link
Member

I have this issue as well... one of my subscriptions to a partitioned topic is showing a backlog of 1 message, but that is incorrect -- the backlog is actually 0.

Partitioned stats:

{
  "msgRateIn" : 0.0,
  "msgThroughputIn" : 0.0,
  "msgRateOut" : 0.0,
  "msgThroughputOut" : 0.0,
  "averageMsgSize" : 0.0,
  "storageSize" : 2189,
  "publishers" : [ {
    "msgRateIn" : 0.0,
    "msgThroughputIn" : 0.0,
    "averageMsgSize" : 0.0,
    "producerId" : 0
  } ],
  "subscriptions" : {
    "__compaction" : {
      "msgRateOut" : 0.0,
      "msgThroughputOut" : 0.0,
      "msgRateRedeliver" : 0.0,
      "msgBacklog" : 0,
      "blockedSubscriptionOnUnackedMsgs" : false,
      "msgDelayed" : 0,
      "unackedMessages" : 0,
      "msgRateExpired" : 0.0,
      "consumers" : [ ],
      "isReplicated" : false
    },
    "test-xxx" : {
      "msgRateOut" : 0.0,
      "msgThroughputOut" : 0.0,
      "msgRateRedeliver" : 0.0,
      "msgBacklog" : 1,
      "blockedSubscriptionOnUnackedMsgs" : false,
      "msgDelayed" : 0,
      "unackedMessages" : 1,
      "msgRateExpired" : 0.0,
      "consumers" : [ {
        "msgRateOut" : 0.0,
        "msgThroughputOut" : 0.0,
        "msgRateRedeliver" : 0.0,
        "availablePermits" : 998,
        "unackedMessages" : 1,
        "blockedConsumerOnUnackedMsgs" : false
      } ],
      "isReplicated" : false
    }
  },
  "replication" : { },
  "metadata" : {
    "partitions" : 1
  },
  "partitions" : { }
}

and internal stats:

{
  "entriesAddedCounter" : 28,
  "numberOfEntries" : 8,
  "totalSize" : 19297,
  "currentLedgerEntries" : 7,
  "currentLedgerSize" : 17114,
  "lastLedgerCreatedTimestamp" : "2019-10-14T14:46:51.57Z",
  "waitingCursorsCount" : 268,
  "pendingAddEntriesCount" : 0,
  "lastConfirmedEntry" : "81:6",
  "state" : "LedgerOpened",
  "ledgers" : [ {
    "ledgerId" : 81,
    "entries" : 0,
    "size" : 0,
    "offloaded" : false
  } ],
  "cursors" : {
    "__compaction" : {
      "markDeletePosition" : "81:6",
      "readPosition" : "81:7",
      "waitingReadOp" : false,
      "pendingReadOps" : 0,
      "messagesConsumedCounter" : 28,
      "cursorLedger" : 85,
      "cursorLedgerLastEntry" : 3,
      "individuallyDeletedMessages" : "[]",
      "lastLedgerSwitchTimestamp" : "2019-10-15T05:22:53.273Z",
      "state" : "Open",
      "numberOfEntriesSinceFirstNotAckedMessage" : 1,
      "totalNonContiguousDeletedMessagesRange" : 0,
      "properties" : {
        "CompactedTopicLedger" : 86
      }
    },
    "test-xxx" : {
      "markDeletePosition" : "81:6",
      "readPosition" : "81:7",
      "waitingReadOp" : true,
      "pendingReadOps" : 0,
      "messagesConsumedCounter" : 27,
      "cursorLedger" : 84,
      "cursorLedgerLastEntry" : 5,
      "individuallyDeletedMessages" : "[]",
      "lastLedgerSwitchTimestamp" : "2019-10-15T05:19:49.576Z",
      "state" : "Open",
      "numberOfEntriesSinceFirstNotAckedMessage" : 1,
      "totalNonContiguousDeletedMessagesRange" : 0,
      "properties" : { }
    }
  }
}

@Shawyeok
Copy link
Contributor

We are suffering from same problem here, msgBacklog is 40k+ but real msgBacklog is 0. It is annoying cause it'll trigger alert but there isn't any real backlog.

Stats with preciseBacklog=false

{
  "msgRateIn": 0.0,
  "msgThroughputIn": 0.0,
  "msgRateOut": 0.0,
  "msgThroughputOut": 0.0,
  "bytesInCounter": 335230277,
  "msgInCounter": 1584209,
  "bytesOutCounter": 177108372,
  "msgOutCounter": 789829,
  "averageMsgSize": 0.0,
  "msgChunkPublished": false,
  "storageSize": 305110652,
  "backlogSize": 296556524,
  "offloadedStorageSize": 0,
  "lastOffloadLedgerId": 0,
  "lastOffloadSuccessTimeStamp": 0,
  "lastOffloadFailureTimeStamp": 0,
  "publishers": [ ],
  "waitingPublishers": 0,
  "subscriptions": {
    "sub0": {
      "msgRateOut": 0.0,
      "msgThroughputOut": 0.0,
      "bytesOutCounter": 3064627,
      "msgOutCounter": 16150,
      "msgRateRedeliver": 0.0,
      "chunkedMessageRate": 0,
      "msgBacklog": 44708,
      "backlogSize": 0,
      "msgBacklogNoDelayed": 44708,
      "blockedSubscriptionOnUnackedMsgs": false,
      "msgDelayed": 0,
      "unackedMessages": 0,
      "type": "Shared",
      "msgRateExpired": 0.0,
      "totalMsgExpired": 0,
      "lastExpireTimestamp": 0,
      "lastConsumedFlowTimestamp": 1652265631373,
      "lastConsumedTimestamp": 0,
      "lastAckedTimestamp": 0,
      "lastMarkDeleteAdvancedTimestamp": 1652265631403,
      "consumers": [],
      "isDurable": true,
      "isReplicated": false,
      "consumersAfterMarkDeletePosition": {},
      "nonContiguousDeletedMessagesRanges": 0,
      "nonContiguousDeletedMessagesRangesSerializedSize": 60,
      "durable": true,
      "replicated": false
    },
    "sub1": {
      "msgRateOut": 0.0,
      "msgThroughputOut": 0.0,
      "bytesOutCounter": 171894114,
      "msgOutCounter": 761896,
      "msgRateRedeliver": 0.0,
      "chunkedMessageRate": 0,
      "msgBacklog": 0,
      "backlogSize": 0,
      "msgBacklogNoDelayed": 0,
      "blockedSubscriptionOnUnackedMsgs": false,
      "msgDelayed": 0,
      "unackedMessages": 0,
      "type": "Shared",
      "msgRateExpired": 0.0,
      "totalMsgExpired": 0,
      "lastExpireTimestamp": 0,
      "lastConsumedFlowTimestamp": 1652336505371,
      "lastConsumedTimestamp": 1652336498413,
      "lastAckedTimestamp": 1652336512153,
      "lastMarkDeleteAdvancedTimestamp": 1652336512153,
      "consumers": [ ],
      "isDurable": true,
      "isReplicated": false,
      "consumersAfterMarkDeletePosition": {},
      "nonContiguousDeletedMessagesRanges": 0,
      "nonContiguousDeletedMessagesRangesSerializedSize": 22,
      "durable": true,
      "replicated": false
    }
  },
  "replication": {},
  "deduplicationStatus": "Disabled",
  "nonContiguousDeletedMessagesRanges": 0,
  "nonContiguousDeletedMessagesRangesSerializedSize": 82,
  "compaction": {
    "lastCompactionRemovedEventCount": 0,
    "lastCompactionSucceedTimestamp": 0,
    "lastCompactionFailedTimestamp": 0,
    "lastCompactionDurationTimeInMills": 0
  }
}

internal-stats

{
  "waitingCursorsCount": 1,
  "pendingAddEntriesCount": 0,
  "entriesAddedCounter": 1584209,
  "numberOfEntries": 1413448,
  "totalSize": 305110652,
  "currentLedgerEntries": 0,
  "currentLedgerSize": 0,
  "lastLedgerCreatedTimestamp": "2022-05-12T18:04:59.17+08:00",
  "lastLedgerCreationFailureTimestamp": "",
  "lastConfirmedEntry": "114272945:43409",
  "state": "LedgerOpened",
  "ledgers": [
    {
      "ledgerId": 113999206,
      "entries": 50000,
      "size": 8831300,
      "offloaded": false
    },
    ...
    {
      "ledgerId": 114327876,
      "entries": 0,
      "size": 0,
      "offloaded": false
    }
  ],
  "cursors": {
    "sub0": {
      "markDeletePosition": "114272945:43409",
      "readPosition": "114272945:43410",
      "waitingReadOp": false,
      "pendingReadOps": 0,
      "messagesConsumedCounter": 1539501,
      "cursorLedger": 114594694,
      "cursorLedgerLastEntry": 0,
      "individuallyDeletedMessages": "[]",
      "lastLedgerWitchTimestamp": "",
      "state": "Open",
      "numberOfEntriesSinceFirstNotAckedMessage": 1,
      "totalNonContiguousDeletedMessagesRange": 0,
      "properties": {}
    },
    "sub1": {
      "markDeletePosition": "114272945:43409",
      "readPosition": "114272945:43410",
      "waitingReadOp": true,
      "pendingReadOps": 0,
      "messagesConsumedCounter": 1584421,
      "cursorLedger": 114275529,
      "cursorLedgerLastEntry": 16661,
      "individuallyDeletedMessages": "[]",
      "lastLedgerWitchTimestamp": "",
      "state": "Open",
      "numberOfEntriesSinceFirstNotAckedMessage": 1,
      "totalNonContiguousDeletedMessagesRange": 0,
      "properties": {}
    }
  }
}

@M4zfw
Copy link

M4zfw commented Aug 1, 2023

image
image
same problem~
Is there any progress on this issue?

@Shawyeok
Copy link
Contributor

Possible fixed in #19275

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
area/broker lifecycle/stale type/bug The PR fixed a bug or issue reported a bug
Projects
None yet
Development

No branches or pull requests

6 participants