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 2022/06/01 17:31:47 UTC

[GitHub] [pulsar] truong-hua opened a new issue, #15881: Inactive topic cleaning up does not work properly

truong-hua opened a new issue, #15881:
URL: https://github.com/apache/pulsar/issues/15881

   **Describe the bug**
   I'm using Pulsar 2.10.0 with `brokerDeleteInactivePartitionedTopicMetadataEnabled=true` and `allowAutoTopicCreationType=partitioned`. We disabled the `brokerDeleteInactiveTopicsEnabled` at broker level and enabled it in namespace level with this inactive topic policy: `{"inactiveTopicDeleteMode": "delete_when_subscriptions_caught_up", "maxInactiveDurationSeconds": 3600, "deleteWhileInactive": true}`.
   
   We also have some functions keep running all the time with regex subscriptions.
   
   Full settings:
   
   ```
   allowAutoTopicCreationType=partitioned
   brokerDeleteInactivePartitionedTopicMetadataEnabled=true
   brokerDeleteInactiveTopicsEnabled=false
   clusterName=mycluster
   configurationMetadataStoreUrl=zookeeper:2181
   forceDeleteNamespaceAllowed=true
   forceDeleteTenantAllowed=true
   functionsWorkerEnabled=false
   managedLedgerDefaultAckQuorum=2
   managedLedgerDefaultEnsembleSize=3
   managedLedgerDefaultWriteQuorum=3
   metadataStoreUrl=zookeeper:2181
   numHttpServerThreads=32
   numIOThreads=8
   systemTopicEnabled=true
   topicLevelPoliciesEnabled=true
   ```
   
   **To Reproduce**
   Steps to reproduce the behavior:
   1. Deploy a pulsar 2.10.0 with these settings above:
   2. Produce some messages to random topic (pulsar will auto create these new partitioned topics)
   3. Wait for few hours later
   4. These inactive topics are still there
   
   **Expected behavior**
   The inactive topics are deleted properly
   
   **Desktop (please complete the following information):**
    - OS: Linux
   
   **Additional question**
   Is it possible to force the inactive topic cleaner to run immediately via REST api or admin cli? This feature is risky because it can cause data lost so I think it is better if we can provide enough tools to verify.
   


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] truong-hua commented on issue #15881: Inactive topic cleaning up does not work properly

Posted by GitBox <gi...@apache.org>.
truong-hua commented on issue #15881:
URL: https://github.com/apache/pulsar/issues/15881#issuecomment-1145598767

   @Technoboy- 
   
   There is no backlog and no message published for a while and the topic is still there. Can I force the inactive topic cleaner to run now?
   
   ```
   {
     "msgRateIn": 0,
     "msgThroughputIn": 0,
     "msgRateOut": 0,
     "msgThroughputOut": 0,
     "bytesInCounter": 0,
     "msgInCounter": 0,
     "bytesOutCounter": 0,
     "msgOutCounter": 0,
     "averageMsgSize": 0,
     "msgChunkPublished": false,
     "storageSize": 0,
     "backlogSize": 0,
     "publishRateLimitedTimes": 0,
     "earliestMsgPublishTimeInBacklogs": 0,
     "offloadedStorageSize": 0,
     "lastOffloadLedgerId": 0,
     "lastOffloadSuccessTimeStamp": 0,
     "lastOffloadFailureTimeStamp": 0,
     "publishers": [],
     "waitingPublishers": 0,
     "subscriptions": {
       "public/throttler/facebook_to_rabbitmq": {
         "msgRateOut": 0,
         "msgThroughputOut": 0,
         "bytesOutCounter": 0,
         "msgOutCounter": 0,
         "msgRateRedeliver": 0,
         "chunkedMessageRate": 0,
         "msgBacklog": 0,
         "backlogSize": 0,
         "earliestMsgPublishTimeInBacklog": 0,
         "msgBacklogNoDelayed": 0,
         "blockedSubscriptionOnUnackedMsgs": false,
         "msgDelayed": 0,
         "unackedMessages": 0,
         "msgRateExpired": 0,
         "totalMsgExpired": 0,
         "lastExpireTimestamp": 0,
         "lastConsumedFlowTimestamp": 0,
         "lastConsumedTimestamp": 0,
         "lastAckedTimestamp": 0,
         "lastMarkDeleteAdvancedTimestamp": 0,
         "consumers": [
           {
             "msgRateOut": 0,
             "msgThroughputOut": 0,
             "bytesOutCounter": 0,
             "msgOutCounter": 0,
             "msgRateRedeliver": 0,
             "chunkedMessageRate": 0,
             "availablePermits": 3000,
             "unackedMessages": 0,
             "avgMessagesPerEntry": 0,
             "blockedConsumerOnUnackedMsgs": false,
             "lastAckedTimestamp": 0,
             "lastConsumedTimestamp": 0
           }
         ],
         "isDurable": true,
         "isReplicated": false,
         "allowOutOfOrderDelivery": false,
         "consumersAfterMarkDeletePosition": {},
         "nonContiguousDeletedMessagesRanges": 0,
         "nonContiguousDeletedMessagesRangesSerializedSize": 0,
         "subscriptionProperties": {},
         "durable": true,
         "replicated": false
       }
     },
     "replication": {},
     "nonContiguousDeletedMessagesRanges": 0,
     "nonContiguousDeletedMessagesRangesSerializedSize": 0,
     "compaction": {
       "lastCompactionRemovedEventCount": 0,
       "lastCompactionSucceedTimestamp": 0,
       "lastCompactionFailedTimestamp": 0,
       "lastCompactionDurationTimeInMills": 0
     },
     "metadata": {
       "partitions": 3
     },
     "partitions": {
       "persistent://public/throttler/facebook_message_100246245469965-partition-0": {
         "msgRateIn": 0,
         "msgThroughputIn": 0,
         "msgRateOut": 0,
         "msgThroughputOut": 0,
         "bytesInCounter": 0,
         "msgInCounter": 0,
         "bytesOutCounter": 0,
         "msgOutCounter": 0,
         "averageMsgSize": 0,
         "msgChunkPublished": false,
         "storageSize": 0,
         "backlogSize": 0,
         "publishRateLimitedTimes": 0,
         "earliestMsgPublishTimeInBacklogs": 0,
         "offloadedStorageSize": 0,
         "lastOffloadLedgerId": 0,
         "lastOffloadSuccessTimeStamp": 0,
         "lastOffloadFailureTimeStamp": 0,
         "publishers": [],
         "waitingPublishers": 0,
         "subscriptions": {
           "public/throttler/facebook_to_rabbitmq": {
             "msgRateOut": 0,
             "msgThroughputOut": 0,
             "bytesOutCounter": 0,
             "msgOutCounter": 0,
             "msgRateRedeliver": 0,
             "chunkedMessageRate": 0,
             "msgBacklog": 0,
             "backlogSize": 0,
             "earliestMsgPublishTimeInBacklog": 0,
             "msgBacklogNoDelayed": 0,
             "blockedSubscriptionOnUnackedMsgs": false,
             "msgDelayed": 0,
             "unackedMessages": 0,
             "type": "Shared",
             "msgRateExpired": 0,
             "totalMsgExpired": 0,
             "lastExpireTimestamp": 0,
             "lastConsumedFlowTimestamp": 1654165558909,
             "lastConsumedTimestamp": 0,
             "lastAckedTimestamp": 0,
             "lastMarkDeleteAdvancedTimestamp": 0,
             "consumers": [
               {
                 "msgRateOut": 0,
                 "msgThroughputOut": 0,
                 "bytesOutCounter": 0,
                 "msgOutCounter": 0,
                 "msgRateRedeliver": 0,
                 "chunkedMessageRate": 0,
                 "consumerName": "1e480",
                 "availablePermits": 1000,
                 "unackedMessages": 0,
                 "avgMessagesPerEntry": 0,
                 "blockedConsumerOnUnackedMsgs": false,
                 "lastAckedTimestamp": 0,
                 "lastConsumedTimestamp": 0,
                 "metadata": {
                   "instance_id": "0",
                   "application": "pulsar-sink",
                   "instance_hostname": "pulsar-worker",
                   "id": "public/throttler/facebook_to_rabbitmq"
                 },
                 "address": "/10.0.4.23:45888",
                 "clientVersion": "2.10.0",
                 "connectedSince": "2022-06-02T10:25:58.708356Z"
               }
             ],
             "isDurable": true,
             "isReplicated": false,
             "allowOutOfOrderDelivery": false,
             "consumersAfterMarkDeletePosition": {},
             "nonContiguousDeletedMessagesRanges": 0,
             "nonContiguousDeletedMessagesRangesSerializedSize": 0,
             "subscriptionProperties": {},
             "durable": true,
             "replicated": false
           }
         },
         "replication": {},
         "deduplicationStatus": "Disabled",
         "nonContiguousDeletedMessagesRanges": 0,
         "nonContiguousDeletedMessagesRangesSerializedSize": 0,
         "compaction": {
           "lastCompactionRemovedEventCount": 0,
           "lastCompactionSucceedTimestamp": 0,
           "lastCompactionFailedTimestamp": 0,
           "lastCompactionDurationTimeInMills": 0
         }
       },
       "persistent://public/throttler/facebook_message_100246245469965-partition-1": {
         "msgRateIn": 0,
         "msgThroughputIn": 0,
         "msgRateOut": 0,
         "msgThroughputOut": 0,
         "bytesInCounter": 0,
         "msgInCounter": 0,
         "bytesOutCounter": 0,
         "msgOutCounter": 0,
         "averageMsgSize": 0,
         "msgChunkPublished": false,
         "storageSize": 0,
         "backlogSize": 0,
         "publishRateLimitedTimes": 0,
         "earliestMsgPublishTimeInBacklogs": 0,
         "offloadedStorageSize": 0,
         "lastOffloadLedgerId": 0,
         "lastOffloadSuccessTimeStamp": 0,
         "lastOffloadFailureTimeStamp": 0,
         "publishers": [],
         "waitingPublishers": 0,
         "subscriptions": {
           "public/throttler/facebook_to_rabbitmq": {
             "msgRateOut": 0,
             "msgThroughputOut": 0,
             "bytesOutCounter": 0,
             "msgOutCounter": 0,
             "msgRateRedeliver": 0,
             "chunkedMessageRate": 0,
             "msgBacklog": 0,
             "backlogSize": 0,
             "earliestMsgPublishTimeInBacklog": 0,
             "msgBacklogNoDelayed": 0,
             "blockedSubscriptionOnUnackedMsgs": false,
             "msgDelayed": 0,
             "unackedMessages": 0,
             "type": "Shared",
             "msgRateExpired": 0,
             "totalMsgExpired": 0,
             "lastExpireTimestamp": 0,
             "lastConsumedFlowTimestamp": 1654165558925,
             "lastConsumedTimestamp": 0,
             "lastAckedTimestamp": 0,
             "lastMarkDeleteAdvancedTimestamp": 0,
             "consumers": [
               {
                 "msgRateOut": 0,
                 "msgThroughputOut": 0,
                 "bytesOutCounter": 0,
                 "msgOutCounter": 0,
                 "msgRateRedeliver": 0,
                 "chunkedMessageRate": 0,
                 "consumerName": "1e480",
                 "availablePermits": 1000,
                 "unackedMessages": 0,
                 "avgMessagesPerEntry": 0,
                 "blockedConsumerOnUnackedMsgs": false,
                 "lastAckedTimestamp": 0,
                 "lastConsumedTimestamp": 0,
                 "metadata": {
                   "instance_id": "0",
                   "application": "pulsar-sink",
                   "instance_hostname": "pulsar-worker",
                   "id": "public/throttler/facebook_to_rabbitmq"
                 },
                 "address": "/10.0.4.23:45888",
                 "clientVersion": "2.10.0",
                 "connectedSince": "2022-06-02T10:25:58.010212Z"
               }
             ],
             "isDurable": true,
             "isReplicated": false,
             "allowOutOfOrderDelivery": false,
             "consumersAfterMarkDeletePosition": {},
             "nonContiguousDeletedMessagesRanges": 0,
             "nonContiguousDeletedMessagesRangesSerializedSize": 0,
             "subscriptionProperties": {},
             "durable": true,
             "replicated": false
           }
         },
         "replication": {},
         "deduplicationStatus": "Disabled",
         "nonContiguousDeletedMessagesRanges": 0,
         "nonContiguousDeletedMessagesRangesSerializedSize": 0,
         "compaction": {
           "lastCompactionRemovedEventCount": 0,
           "lastCompactionSucceedTimestamp": 0,
           "lastCompactionFailedTimestamp": 0,
           "lastCompactionDurationTimeInMills": 0
         }
       },
       "persistent://public/throttler/facebook_message_100246245469965-partition-2": {
         "msgRateIn": 0,
         "msgThroughputIn": 0,
         "msgRateOut": 0,
         "msgThroughputOut": 0,
         "bytesInCounter": 0,
         "msgInCounter": 0,
         "bytesOutCounter": 0,
         "msgOutCounter": 0,
         "averageMsgSize": 0,
         "msgChunkPublished": false,
         "storageSize": 0,
         "backlogSize": 0,
         "publishRateLimitedTimes": 0,
         "earliestMsgPublishTimeInBacklogs": 0,
         "offloadedStorageSize": 0,
         "lastOffloadLedgerId": 0,
         "lastOffloadSuccessTimeStamp": 0,
         "lastOffloadFailureTimeStamp": 0,
         "publishers": [],
         "waitingPublishers": 0,
         "subscriptions": {
           "public/throttler/facebook_to_rabbitmq": {
             "msgRateOut": 0,
             "msgThroughputOut": 0,
             "bytesOutCounter": 0,
             "msgOutCounter": 0,
             "msgRateRedeliver": 0,
             "chunkedMessageRate": 0,
             "msgBacklog": 0,
             "backlogSize": 0,
             "earliestMsgPublishTimeInBacklog": 0,
             "msgBacklogNoDelayed": 0,
             "blockedSubscriptionOnUnackedMsgs": false,
             "msgDelayed": 0,
             "unackedMessages": 0,
             "type": "Shared",
             "msgRateExpired": 0,
             "totalMsgExpired": 0,
             "lastExpireTimestamp": 0,
             "lastConsumedFlowTimestamp": 1654165558927,
             "lastConsumedTimestamp": 0,
             "lastAckedTimestamp": 0,
             "lastMarkDeleteAdvancedTimestamp": 0,
             "consumers": [
               {
                 "msgRateOut": 0,
                 "msgThroughputOut": 0,
                 "bytesOutCounter": 0,
                 "msgOutCounter": 0,
                 "msgRateRedeliver": 0,
                 "chunkedMessageRate": 0,
                 "consumerName": "1e480",
                 "availablePermits": 1000,
                 "unackedMessages": 0,
                 "avgMessagesPerEntry": 0,
                 "blockedConsumerOnUnackedMsgs": false,
                 "lastAckedTimestamp": 0,
                 "lastConsumedTimestamp": 0,
                 "metadata": {
                   "instance_id": "0",
                   "application": "pulsar-sink",
                   "instance_hostname": "pulsar-worker",
                   "id": "public/throttler/facebook_to_rabbitmq"
                 },
                 "address": "/10.0.4.23:45888",
                 "clientVersion": "2.10.0",
                 "connectedSince": "2022-06-02T10:25:57.972045Z"
               }
             ],
             "isDurable": true,
             "isReplicated": false,
             "allowOutOfOrderDelivery": false,
             "consumersAfterMarkDeletePosition": {},
             "nonContiguousDeletedMessagesRanges": 0,
             "nonContiguousDeletedMessagesRangesSerializedSize": 0,
             "subscriptionProperties": {},
             "durable": true,
             "replicated": false
           }
         },
         "replication": {},
         "deduplicationStatus": "Disabled",
         "nonContiguousDeletedMessagesRanges": 0,
         "nonContiguousDeletedMessagesRangesSerializedSize": 0,
         "compaction": {
           "lastCompactionRemovedEventCount": 0,
           "lastCompactionSucceedTimestamp": 0,
           "lastCompactionFailedTimestamp": 0,
           "lastCompactionDurationTimeInMills": 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.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] Technoboy- commented on issue #15881: Inactive topic cleaning up does not work properly

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

   > > Seems that there exist some subscriptions are not caught up which means these subscriptions have messages not acked.
   > 
   > Is there anyway to verify it? The backlog size is 0 and we have disable the only producer for a while.
   
   You can use topics status to verify this : https://pulsar.apache.org/docs/next/admin-api-topics
   ```
   pulsar-admin topics stats \
     persistent://test-tenant/ns1/tp1 \
   ```


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] Technoboy- commented on issue #15881: Inactive topic cleaning up does not work properly

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

   > FYI, this is the inactiveTopicPolicies configured at that namespace
   > 
   > ```
   > {"inactiveTopicDeleteMode":"delete_when_subscriptions_caught_up","maxInactiveDurationSeconds":3600,"deleteWhileInactive":true}
   > ```
   
   Ah, sorry, I mistakenly thought it was topic level. Could you use the CLI to get the topic level policy ?
   ```
   admin.topicPolicies().getInactiveTopicPolicies(topic)
   ```


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] Technoboy- commented on issue #15881: Inactive topic cleaning up does not work properly

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

   Seems that there exist some subscriptions are not caught up which means these subscriptions have messages not acked.


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] truong-hua commented on issue #15881: Inactive topic cleaning up does not work properly

Posted by GitBox <gi...@apache.org>.
truong-hua commented on issue #15881:
URL: https://github.com/apache/pulsar/issues/15881#issuecomment-1144614688

   > Seems that there exist some subscriptions are not caught up which means these subscriptions have messages not acked.
   
   Is there anyway to verify it? The backlog size is 0 and we have disable the only producer for a while.


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] truong-hua commented on issue #15881: Inactive topic cleaning up does not work properly

Posted by GitBox <gi...@apache.org>.
truong-hua commented on issue #15881:
URL: https://github.com/apache/pulsar/issues/15881#issuecomment-1145600674

   FYI, this is the inactiveTopicPolicies configured at that namespace
   
   ```
   {"inactiveTopicDeleteMode":"delete_when_subscriptions_caught_up","maxInactiveDurationSeconds":3600,"deleteWhileInactive":true}
   ```


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] github-actions[bot] commented on issue #15881: Inactive topic cleaning up does not work properly

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on issue #15881:
URL: https://github.com/apache/pulsar/issues/15881#issuecomment-1174527839

   The issue had no activity for 30 days, mark with Stale label.


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org