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/06/04 12:16:09 UTC

[GitHub] [pulsar] eolivelli opened a new issue #10813: Consumer stuck while reading from partitioned topic using Shared subscription

eolivelli opened a new issue #10813:
URL: https://github.com/apache/pulsar/issues/10813


   I am investigating a case in which the consumer is stuck while there are still many messages to read from a partitioned topic.
   
   At high level the test is:
   - create a partitioned topic with 100 partitions
   - produce 1.000.000 messages
   - consume the message
   
   Desired outcome: the consumer is able to fully consume the messages 
   Actual outcome (current 2.8.0-SNAPSHOT): the consumer is stuck at approximately 80% of the topic
   
   Steps to reproduce:
   - install a brand new Pulsar cluster, 2.8.0-SNAPSHOT (I am using k8s and the helm chart)
   - 3 bookies, replication 3x3x3, 1 broker, 1 proxy
   - set retention: `bin/pulsar-admin namespaces set-retention -s 3T -t 3w public/default`
   - create topic: `bin/pulsar-admin topics create-partitioned-topic -p 100 test`
   - create the "shared" subscription: `bin/pulsar-perf consume -ss nb -st Shared test`
   - verify stats: `bin/pulsar-admin topics partitioned-stats test`
   - produce messages: `bin/pulsar-perf produce -r 10000 -m 1000000 -mk random test`
   - consume the messages using a very simple Java client program (listing below)
   - compile the program: 
   -- cd "/pulsar"
   -- write SimpleConsumeMain.java
   -- javac SimpleConsumeMain.java  -cp "lib/*"
   - run the program `java -cp ".:lib/*" SimpleConsumeMain pulsar://pulsar-broker:6650 nb test`
   
   The program should consume the subscription, with 1.000.000 messages.
   
   Additional details:
   - The problem happens even in case of 2.8.0-SNAPHOT cluster and **2.7.1 client**
   -  if you use an **Exclusive** subscription the problem **does not happen**
   - if you disable "**batching**" on the Producer the problems **does not happen**
   - Initially it seemed to me that "pulsar-perf consume" worked well, but I was lucky, in fact you can the command `bin/pulsar-perf consume -ss aa -sp Earliest -st Shared test` and stop it after some time that it reports "0 msgs/s" and you will see that the total number of consumed messages is not 1.000.000, if you run it again then you receive more messages
    
   ```
   import org.apache.pulsar.client.api.Consumer;
   import org.apache.pulsar.client.api.Message;
   import org.apache.pulsar.client.api.PulsarClient;
   import org.apache.pulsar.client.api.SubscriptionInitialPosition;
   import org.apache.pulsar.client.api.SubscriptionType;
   
   public class SimpleConsumeMain {
       public static void main(String ... args) throws Exception {
           String broker = args[0];
           String subscription = args[1];
           String topic = args[2];
           PulsarClient client = PulsarClient
                   .builder()
                   .serviceUrl(broker)
                   .build();
           Consumer<byte[]> consumer = client.newConsumer()
                 .subscriptionName(subscription)
                   .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest)
                   .subscriptionType(SubscriptionType.Shared)
                 .topic(topic)
                 .subscribe();
           long count = 0;
           while (true) {
               Message<byte[]> msg = consumer.receive();
               count++;
               consumer.acknowledge(msg);
               if (count % 1000 == 0) {
                   System.out.println("Received "+count+" messages");
               }
           }
       }
   }
   
   ```
   
   


-- 
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:
users@infra.apache.org



[GitHub] [pulsar] codelipenghui closed issue #10813: Consumer stuck while reading from partitioned topic using Shared subscription

Posted by GitBox <gi...@apache.org>.
codelipenghui closed issue #10813:
URL: https://github.com/apache/pulsar/issues/10813


   


-- 
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:
users@infra.apache.org



[GitHub] [pulsar] eolivelli commented on issue #10813: Consumer stuck while reading from partitioned topic using Shared subscription

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


   If you halt the client and restart it a couple of times then you are able to consume the whole topic.
   It looks like there is something stuck


-- 
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:
users@infra.apache.org



[GitHub] [pulsar] eolivelli commented on issue #10813: Consumer stuck while reading from partitioned topic using Shared subscription

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


   I did a few more tests with pulsar-perf and it looks like even with pulsar-perf I sometimes see problems.
   I will update the description


-- 
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:
users@infra.apache.org



[GitHub] [pulsar] lhotari commented on issue #10813: Consumer stuck while reading from partitioned topic using Shared subscription

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


   Thanks for the good issue report @eolivelli !
   
   I was able to reproduce with 30 partitions and 100000 messages in a single node cluster (setup documented in https://github.com/lhotari/pulsar-playground/tree/master/test-env).
   
   Documenting the commands I used:
   ```
   pulsar-admin namespaces set-retention -s 3T -t 3w public/default
   pulsar-admin topics create-partitioned-topic -p 30 test
   pulsar-perf consume -s nb -st Shared test
   pulsar-perf produce -r 10000 -m 100000 -mk random test
   java -cp ".:$HOME/workspace-pulsar/apache-pulsar-2.7.2/lib/*" SimpleConsumeMain pulsar://pulsar-testenv-deployment-broker.pulsar-testenv.svc.cluster.local:6650/ nb test
   ```
   
   Deleting the subscription (to re-test, could also use new subscription name):
   ```
   pulsar-admin topics unsubscribe -s nb test
   ```
   
   Script to get stats:
   ```
   { for i in $(seq 0 29);do echo $i; pulsar-admin topics stats test-partition-$i; done; } | tee stats.txt
   ```
   internal stats
   ```
   { for i in $(seq 0 29);do echo $i; pulsar-admin topics stats-internal test-partition-$i; done; } | tee stats_internal.txt
   ```
   
   The topics stats includes negative permits:
   ```
   ❯ grep availablePermits stats.txt
           "availablePermits" : 434,
           "availablePermits" : 0,
           "availablePermits" : 829,
           "availablePermits" : -2,
           "availablePermits" : 0,
           "availablePermits" : 594,
           "availablePermits" : 766,
           "availablePermits" : 0,
           "availablePermits" : 849,
           "availablePermits" : 387,
           "availablePermits" : 799,
           "availablePermits" : 421,
           "availablePermits" : 552,
           "availablePermits" : 0,
           "availablePermits" : 0,
           "availablePermits" : 0,
           "availablePermits" : -1,
           "availablePermits" : -1,
           "availablePermits" : -6,
           "availablePermits" : 824,
           "availablePermits" : 986,
           "availablePermits" : 308,
           "availablePermits" : 541,
           "availablePermits" : 715,
           "availablePermits" : 913,
           "availablePermits" : 784,
           "availablePermits" : 602,
           "availablePermits" : 766,
           "availablePermits" : 559,
           "availablePermits" : 354,
   ```
   
   Could negative permits be a problem?
   
   
   
   
   


-- 
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:
users@infra.apache.org



[GitHub] [pulsar] devinbost commented on issue #10813: Consumer stuck while reading from partitioned topic using Shared subscription

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






-- 
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:
users@infra.apache.org



[GitHub] [pulsar] hpvd edited a comment on issue #10813: Consumer stuck while reading from partitioned topic using Shared subscription

Posted by GitBox <gi...@apache.org>.
hpvd edited a comment on issue #10813:
URL: https://github.com/apache/pulsar/issues/10813#issuecomment-854130933


   negative permits also are documented in other open issues:
   https://github.com/apache/pulsar/issues/6054#issuecomment-844625376
   https://github.com/apache/pulsar/issues/6255#issuecomment-816987017
   
   edit: ooops, sorry 1min too late @devinbost already jumped in..


-- 
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:
users@infra.apache.org



[GitHub] [pulsar] hpvd commented on issue #10813: Consumer stuck while reading from partitioned topic using Shared subscription

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


   negative permits also are documented in other open issues:
   https://github.com/apache/pulsar/issues/6054#issuecomment-844625376
   https://github.com/apache/pulsar/issues/6255#issuecomment-816987017


-- 
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:
users@infra.apache.org



[GitHub] [pulsar] eolivelli commented on issue #10813: Consumer stuck while reading from partitioned topic using Shared subscription

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


   Just to be clear:
   In this case the producer is done, there is no producer.
   The topic is is idle state, and also retention is disabled.


-- 
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:
users@infra.apache.org



[GitHub] [pulsar] hpvd commented on issue #10813: Consumer stuck while reading from partitioned topic using Shared subscription

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


   negative permits also are documented in other open issues:
   https://github.com/apache/pulsar/issues/6054#issuecomment-844625376
   https://github.com/apache/pulsar/issues/6255#issuecomment-816987017


-- 
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:
users@infra.apache.org



[GitHub] [pulsar] eolivelli commented on issue #10813: Consumer stuck while reading from partitioned topic using Shared subscription

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






-- 
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:
users@infra.apache.org



[GitHub] [pulsar] eolivelli closed issue #10813: Consumer stuck while reading from partitioned topic using Shared subscription

Posted by GitBox <gi...@apache.org>.
eolivelli closed issue #10813:
URL: https://github.com/apache/pulsar/issues/10813


   


-- 
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:
users@infra.apache.org



[GitHub] [pulsar] eolivelli commented on issue #10813: Consumer stuck while reading from partitioned topic using Shared subscription

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


   @merlimat @rdhabalia @codelipenghui @lhotari  PTAL
   I believe this is another blocker for 2.8.0.
   
    


-- 
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:
users@infra.apache.org



[GitHub] [pulsar] devinbost commented on issue #10813: Consumer stuck while reading from partitioned topic using Shared subscription

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


   @hpvd In both of those issues, the problem is that the producer semaphore is blocking because the producer isn't receiving ack's from the broker. After enough pending messages accumulate without ack's, the producer blocks because `blockIfQueueFull` is true by default. See https://github.com/apache/pulsar/blob/34a04b5fa4692cb4716df61e740795467f76451b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java#L756


-- 
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:
users@infra.apache.org



[GitHub] [pulsar] lhotari commented on issue #10813: Consumer stuck while reading from partitioned topic using Shared subscription

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


   Thanks for the good issue report @eolivelli !
   
   I was able to reproduce with 30 partitions and 100000 messages in a single node cluster (setup documented in https://github.com/lhotari/pulsar-playground/tree/master/test-env).
   
   Documenting the commands I used:
   ```
   pulsar-admin namespaces set-retention -s 3T -t 3w public/default
   pulsar-admin topics create-partitioned-topic -p 30 test
   pulsar-perf consume -s nb -st Shared test
   pulsar-perf produce -r 10000 -m 100000 -mk random test
   java -cp ".:$HOME/workspace-pulsar/apache-pulsar-2.7.2/lib/*" SimpleConsumeMain pulsar://pulsar-testenv-deployment-broker.pulsar-testenv.svc.cluster.local:6650/ nb test
   ```
   
   Deleting the subscription (to re-test, could also use new subscription name):
   ```
   pulsar-admin topics unsubscribe -s nb test
   ```
   
   Script to get stats:
   ```
   { for i in $(seq 0 29);do echo $i; pulsar-admin topics stats test-partition-$i; done; } | tee stats.txt
   ```
   internal stats
   ```
   { for i in $(seq 0 29);do echo $i; pulsar-admin topics stats-internal test-partition-$i; done; } | tee stats_internal.txt
   ```
   
   The topics stats includes negative permits:
   ```
   ❯ grep availablePermits stats.txt
           "availablePermits" : 434,
           "availablePermits" : 0,
           "availablePermits" : 829,
           "availablePermits" : -2,
           "availablePermits" : 0,
           "availablePermits" : 594,
           "availablePermits" : 766,
           "availablePermits" : 0,
           "availablePermits" : 849,
           "availablePermits" : 387,
           "availablePermits" : 799,
           "availablePermits" : 421,
           "availablePermits" : 552,
           "availablePermits" : 0,
           "availablePermits" : 0,
           "availablePermits" : 0,
           "availablePermits" : -1,
           "availablePermits" : -1,
           "availablePermits" : -6,
           "availablePermits" : 824,
           "availablePermits" : 986,
           "availablePermits" : 308,
           "availablePermits" : 541,
           "availablePermits" : 715,
           "availablePermits" : 913,
           "availablePermits" : 784,
           "availablePermits" : 602,
           "availablePermits" : 766,
           "availablePermits" : 559,
           "availablePermits" : 354,
   ```
   
   Could negative permits be a problem?
   
   
   
   
   


-- 
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:
users@infra.apache.org



[GitHub] [pulsar] devinbost commented on issue #10813: Consumer stuck while reading from partitioned topic using Shared subscription

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


   @eolivelli @rdhabalia Could there be two separate issues involved? It looks like the PR that closed this issue only addresses dispatcher behavior, not the ack path.


-- 
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:
users@infra.apache.org



[GitHub] [pulsar] rdhabalia commented on issue #10813: Consumer stuck while reading from partitioned topic using Shared subscription

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


   Below metrics shows the all messages are dispatched but consumer didn't ack the message after `28:8916`. seems ack issue.
   ```
   "lastConfirmedEntry" : "28:8934",
   :
   "markDeletePosition" : "28:8916",
    "readPosition" : "28:8935",
   ```
   
   Are you still able to reproduce it with the master code and sample consumer you mentioned into the issue? and do you see any broker side exception in ack message path.


-- 
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:
users@infra.apache.org



[GitHub] [pulsar] hpvd edited a comment on issue #10813: Consumer stuck while reading from partitioned topic using Shared subscription

Posted by GitBox <gi...@apache.org>.
hpvd edited a comment on issue #10813:
URL: https://github.com/apache/pulsar/issues/10813#issuecomment-854130933


   negative permits also are documented in other open issues:
   https://github.com/apache/pulsar/issues/6054#issuecomment-844625376
   https://github.com/apache/pulsar/issues/6255#issuecomment-816987017
   
   edit: ooops, sorry 1min too late @devinbost already jumped in..


-- 
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:
users@infra.apache.org



[GitHub] [pulsar] eolivelli commented on issue #10813: Consumer stuck while reading from partitioned topic using Shared subscription

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


   Unfortunately  #10824 made the situation better and the test arrives to 99% but this we have problems.
   
   Also today I see a failure even with Exclusive subscription type.
   batchenabled=true
   subscription type = Exclusive


-- 
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:
users@infra.apache.org



[GitHub] [pulsar] eolivelli commented on issue #10813: Consumer stuck while reading from partitioned topic using Shared subscription

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


   I am rerunning the after @codelipenghui  fix.
   
   This is the dump of the stats now that the consumer is still connected and waiting for messages.
   
   the interesting point is that for partition 1 (and for other partitions it happens the same)
   we see "numberOfEntriesSinceFirstNotAckedMessage" : 19 and "msgBacklog" : 18
   
   so it looks like the messages have been delivered to the consumer from the broker point of view, but they are not received by the application.
   
   
   > bin/pulsar-admin topics partitioned-stats test
   > {
   >  "msgRateIn" : 0.0,
   >  "msgThroughputIn" : 0.0,
   >  "msgRateOut" : 0.0,
   >  "msgThroughputOut" : 0.0,
   >  "bytesInCounter" : 1099448813,
   >  "msgInCounter" : 1000000,
   >  "bytesOutCounter" : 1099094585,
   >  "msgOutCounter" : 999678,
   >  "averageMsgSize" : 0.0,
   >  "msgChunkPublished" : false,
   >  "storageSize" : 1099448813,
   >  "backlogSize" : 461972,
   >  "offloadedStorageSize" : 0,
   >  "publishers" : [ ],
   >  "waitingPublishers" : 0,
   >  "subscriptions" : {
   >   "nb" : {
   >    "msgRateOut" : 0.0,
   >    "msgThroughputOut" : 0.0,
   >    "bytesOutCounter" : 1099094585,
   >    "msgOutCounter" : 999678,
   >    "msgRateRedeliver" : 0.0,
   >    "chuckedMessageRate" : 0,
   >    "chunkedMessageRate" : 0,
   >    "msgBacklog" : 295,
   >    "backlogSize" : 0,
   >    "msgBacklogNoDelayed" : 295,
   >    "blockedSubscriptionOnUnackedMsgs" : false,
   >    "msgDelayed" : 0,
   >    "unackedMessages" : 0,
   >    "msgRateExpired" : 0.0,
   >    "totalMsgExpired" : 0,
   >    "lastExpireTimestamp" : 0,
   >    "lastConsumedFlowTimestamp" : 0,
   >    "lastConsumedTimestamp" : 0,
   >    "lastAckedTimestamp" : 0,
   >    "lastMarkDeleteAdvancedTimestamp" : 0,
   >    "consumers" : [ {
   >     "msgRateOut" : 0.0,
   >     "msgThroughputOut" : 0.0,
   >     "bytesOutCounter" : 1099094585,
   >     "msgOutCounter" : 999678,
   >     "msgRateRedeliver" : 0.0,
   >     "chuckedMessageRate" : 0.0,
   >     "chunkedMessageRate" : 0.0,
   >     "availablePermits" : 87822,
   >     "unackedMessages" : 0,
   >     "avgMessagesPerEntry" : 0,
   >     "blockedConsumerOnUnackedMsgs" : false,
   >     "lastAckedTimestamp" : 0,
   >     "lastConsumedTimestamp" : 0
   >    } ],
   >    "isDurable" : true,
   >    "isReplicated" : false,
   >    "consumersAfterMarkDeletePosition" : { },
   >    "nonContiguousDeletedMessagesRanges" : 0,
   >    "nonContiguousDeletedMessagesRangesSerializedSize" : 0
   >   }
   >  },
   >  "replication" : { },
   >  "nonContiguousDeletedMessagesRanges" : 0,
   >  "nonContiguousDeletedMessagesRangesSerializedSize" : 0,
   >  "metadata" : {
   >   "partitions" : 100
   >  },
   >  "partitions" : { }
   > }
   
   > pulsar-admin topics stats test-partition-1xt
   > {
   >  "msgRateIn" : 0.0,
   >  "msgThroughputIn" : 0.0,
   >  "msgRateOut" : 0.0,
   >  "msgThroughputOut" : 0.0,
   >  "bytesInCounter" : 10908949,
   >  "msgInCounter" : 9924,
   >  "bytesOutCounter" : 10888019,
   >  "msgOutCounter" : 9905,
   >  "averageMsgSize" : 0.0,
   >  "msgChunkPublished" : false,
   >  "storageSize" : 10908949,
   >  "backlogSize" : 30209,
   >  "offloadedStorageSize" : 0,
   >  "publishers" : [ ],
   >  "waitingPublishers" : 0,
   >  "subscriptions" : {
   >   "nb" : {
   >    "msgRateOut" : 0.0,
   >    "msgThroughputOut" : 0.0,
   >    "bytesOutCounter" : 10888019,
   >    "msgOutCounter" : 9905,
   >    "msgRateRedeliver" : 0.0,
   >    "chuckedMessageRate" : 0,
   >    "chunkedMessageRate" : 0,
   >    "msgBacklog" : 18,
   >    "backlogSize" : 0,
   >    "msgBacklogNoDelayed" : 18,
   >    "blockedSubscriptionOnUnackedMsgs" : false,
   >    "msgDelayed" : 0,
   >    "unackedMessages" : 0,
   >    "type" : "Shared",
   >    "msgRateExpired" : 0.0,
   >    "totalMsgExpired" : 0,
   >    "lastExpireTimestamp" : 0,
   >    "lastConsumedFlowTimestamp" : 1622812090363,
   >    "lastConsumedTimestamp" : 1622812090524,
   >    "lastAckedTimestamp" : 1622812090688,
   >    "lastMarkDeleteAdvancedTimestamp" : 1622812090688,
   >    "consumers" : [ {
   >     "msgRateOut" : 0.0,
   >     "msgThroughputOut" : 0.0,
   >     "bytesOutCounter" : 10888019,
   >     "msgOutCounter" : 9905,
   >     "msgRateRedeliver" : 0.0,
   >     "chuckedMessageRate" : 0.0,
   >     "chunkedMessageRate" : 0.0,
   >     "consumerName" : "a1c51",
   >     "availablePermits" : 845,
   >     "unackedMessages" : 0,
   >     "avgMessagesPerEntry" : 6,
   >     "blockedConsumerOnUnackedMsgs" : false,
   >     "lastAckedTimestamp" : 1622812090688,
   >     "lastConsumedTimestamp" : 1622812090524,
   >     "metadata" : { },
   >     "address" : "/10.244.2.72:33182",
   >     "connectedSince" : "2021-06-04T13:07:49.122104Z",
   >     "clientVersion" : "2.8.0-SNAPSHOT"
   >    } ],
   >    "isDurable" : true,
   >    "isReplicated" : false,
   >    "consumersAfterMarkDeletePosition" : { },
   >    "nonContiguousDeletedMessagesRanges" : 0,
   >    "nonContiguousDeletedMessagesRangesSerializedSize" : 0
   >   }
   >  },
   >  "replication" : { },
   >  "deduplicationStatus" : "Disabled",
   >  "nonContiguousDeletedMessagesRanges" : 0,
   >  "nonContiguousDeletedMessagesRangesSerializedSize" : 0
   > }
   
   ```
   pulsar-admin topics stats-internal test-partition-1
   {
    "entriesAddedCounter" : 8935,
    "numberOfEntries" : 8935,
    "totalSize" : 10908949,
    "currentLedgerEntries" : 8935,
    "currentLedgerSize" : 10908949,
    "lastLedgerCreatedTimestamp" : "2021-06-04T12:20:18.707Z",
    "waitingCursorsCount" : 0,
    "pendingAddEntriesCount" : 0,
    "lastConfirmedEntry" : "28:8934",
    "state" : "LedgerOpened",
    "ledgers" : [ {
     "ledgerId" : 28,
     "entries" : 0,
     "size" : 0,
     "offloaded" : false,
     "underReplicated" : false
    } ],
    "cursors" : {
     "nb" : {
      "markDeletePosition" : "28:8916",
      "readPosition" : "28:8935",
      "waitingReadOp" : false,
      "pendingReadOps" : 0,
      "messagesConsumedCounter" : 8917,
      "cursorLedger" : 62,
      "cursorLedgerLastEntry" : 24,
      "individuallyDeletedMessages" : "[]",
      "lastLedgerSwitchTimestamp" : "2021-06-04T12:20:18.74Z",
      "state" : "Open",
      "numberOfEntriesSinceFirstNotAckedMessage" : 19,
      "totalNonContiguousDeletedMessagesRange" : 0,
      "subscriptionHavePendingRead" : false,
      "subscriptionHavePendingReplayRead" : false,
      "properties" : { }
     }
    },
    "schemaLedgers" : [ ],
    "compactedLedger" : {
     "ledgerId" : -1,
     "entries" : -1,
     "size" : -1,
     "offloaded" : false,
     "underReplicated" : false
    }
   }
   ```
   
   
   


-- 
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:
users@infra.apache.org



[GitHub] [pulsar] devinbost commented on issue #10813: Consumer stuck while reading from partitioned topic using Shared subscription

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


   @eolivelli @lhotari 
   
   Negative permits just means the dispatcher is waiting for permits from the consumer, which implies it's waiting for the consumer to complete processing of the messages. I'd like to know if the consumer is waiting for ack's from the broker. 
   If you get a thread dump of the consumer when it's frozen, is it waiting for the producer semaphor? Look for something like this in the consumer thread dump:
   
   ```
   "myTenant/myNamespace/function-filter-0" prio=5 tid=32 WAITING
       at sun.misc.Unsafe.park(Native Method)
       at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
       at java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836)
       at java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedInterruptibly(AbstractQueuedSynchronizer.java:997)
          local variable: java.util.concurrent.locks.AbstractQueuedSynchronizer$Node#40
          local variable: java.util.concurrent.locks.AbstractQueuedSynchronizer$Node#41
       at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(AbstractQueuedSynchronizer.java:1304)
          local variable: java.util.concurrent.Semaphore$FairSync#1
       at java.util.concurrent.Semaphore.acquire(Semaphore.java:312)
       at org.apache.pulsar.client.impl.ProducerImpl.canEnqueueRequest(ProducerImpl.java:748)
       at org.apache.pulsar.client.impl.ProducerImpl.sendAsync(ProducerImpl.java:391)
          local variable: org.apache.pulsar.client.impl.ProducerImpl#1
          local variable: org.apache.pulsar.client.impl.ProducerImpl$1#1
          local variable: io.netty.buffer.UnpooledHeapByteBuf#17
          local variable: org.apache.pulsar.common.api.proto.MessageMetadata#83
          local variable: org.apache.pulsar.client.impl.MessageImpl#82
       at org.apache.pulsar.client.impl.ProducerImpl.internalSendAsync(ProducerImpl.java:290)
          local variable: java.util.concurrent.CompletableFuture#33
       at org.apache.pulsar.client.impl.TypedMessageBuilderImpl.sendAsync(TypedMessageBuilderImpl.java:103)
   ```
   
   Another thing to look for is in a consumer heap dump. I'd like to know if `ProducerImpl.pendingMessages` has messages in it (and how many.) If it's empty, I'd like to know if `ConsumerImpl.incomingMessages` has messages in it or not. 


-- 
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:
users@infra.apache.org