shustsud opened a new issue #5202: Incorrect msgBacklog value
URL: https://github.com/apache/pulsar/issues/5202
 
 
   **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".
    
https://github.com/apache/pulsar/blob/dc7d01efc6cf2df5631bc509758f2212bede35ce/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java#L736
   
   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
   

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

Reply via email to