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/04/29 09:00:19 UTC

[GitHub] [pulsar] Shawyeok opened a new issue, #15390: Cursor recover took long time

Shawyeok opened a new issue, #15390:
URL: https://github.com/apache/pulsar/issues/15390

   **Describe the bug**
   In our production, after a node restart, a topic partition keeping load timed out:
   ```
   2022-04-26T21:39:30.444Z ERROR [pulsar-io-4-16] org.apache.pulsar.broker.service.ServerCnx - [/10.xxx:56961] Failed to create topic persistent://data/uu/feature.plain.result.queue-partition-31, producerId=31
   java.util.concurrent.CompletionException: org.apache.pulsar.common.util.FutureUtil$LowOverheadTimeoutException: Failed to load topic within timeout
   	at java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:292)
   	at java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:308)
   	at java.util.concurrent.CompletableFuture.uniCompose(CompletableFuture.java:943)
   	at java.util.concurrent.CompletableFuture$UniCompose.tryFire(CompletableFuture.java:926)
   	at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474)
   	at java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1977)
   	at org.apache.pulsar.common.util.FutureUtil.lambda$addTimeoutHandling$1(FutureUtil.java:141)
   	at io.netty.util.concurrent.PromiseTask.runTask(PromiseTask.java:98)
   	at io.netty.util.concurrent.ScheduledFutureTask.run(ScheduledFutureTask.java:170)
   	at io.netty.util.concurrent.AbstractEventExecutor.safeExecute(AbstractEventExecutor.java:164)
   	at io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:469)
   	at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:384)
   	at io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:986)
   	at io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
   	at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
   	at java.lang.Thread.run(Thread.java:748)
   Caused by: org.apache.pulsar.common.util.FutureUtil$LowOverheadTimeoutException: Failed to load topic within timeout
   	at org.apache.pulsar.broker.service.BrokerService.futureWithDeadline(...)(Unknown Source)
   ```
   The log shows there's one cursor recovery didn't callback, it seems cursor's ledger recover took very long time:
   ```
   2022-04-26T21:38:30.533Z INFO [bookkeeper-ml-scheduler-OrderedScheduler-4-0] org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl - [data/uu/persistent/feature.plain.result.queue-partition-31] Loading cursor data.feature.c.user.deliver.and.chat.features.to.hbase
   2022-04-26T21:38:30.533Z INFO [bookkeeper-ml-scheduler-OrderedScheduler-4-0] org.apache.bookkeeper.mledger.impl.ManagedCursorImpl - [data/uu/persistent/feature.plain.result.queue-partition-31] Recovering from bookkeeper ledger cursor: data.feature.c.user.deliver.and.chat.features.to.hbase
   2022-04-26T21:38:30.539Z INFO [bookkeeper-ml-scheduler-OrderedScheduler-4-0] org.apache.bookkeeper.mledger.impl.ManagedCursorImpl - [data/uu/persistent/feature.plain.result.queue-partition-31] Consumer data.feature.c.user.deliver.and.chat.features.to.hbase meta-data recover from ledger 110091138
   2022-04-26T21:39:30.560Z INFO [bookkeeper-ml-scheduler-OrderedScheduler-4-0] org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl - [data/uu/persistent/feature.plain.result.queue-partition-31] Loading cursor data.feature.c.user.deliver.and.chat.features.to.hbase
   2022-04-26T21:39:30.560Z INFO [bookkeeper-ml-scheduler-OrderedScheduler-4-0] org.apache.bookkeeper.mledger.impl.ManagedCursorImpl - [data/uu/persistent/feature.plain.result.queue-partition-31] Recovering from bookkeeper ledger cursor: data.feature.c.user.deliver.and.chat.features.to.hbase
   2022-04-26T21:39:30.561Z INFO [bookkeeper-ml-scheduler-OrderedScheduler-4-0] org.apache.bookkeeper.mledger.impl.ManagedCursorImpl - [data/uu/persistent/feature.plain.result.queue-partition-31] Consumer data.feature.c.user.deliver.and.chat.features.to.hbase meta-data recover from ledger 110091138
   2022-04-26T21:40:30.667Z INFO [bookkeeper-ml-scheduler-OrderedScheduler-4-0] org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl - [data/uu/persistent/feature.plain.result.queue-partition-31] Loading cursor data.feature.c.user.deliver.and.chat.features.to.hbase
   2022-04-26T21:40:30.667Z INFO [bookkeeper-ml-scheduler-OrderedScheduler-4-0] org.apache.bookkeeper.mledger.impl.ManagedCursorImpl - [data/uu/persistent/feature.plain.result.queue-partition-31] Recovering from bookkeeper ledger cursor: data.feature.c.user.deliver.and.chat.features.to.hbase
   2022-04-26T21:40:30.668Z INFO [bookkeeper-ml-scheduler-OrderedScheduler-4-0] org.apache.bookkeeper.mledger.impl.ManagedCursorImpl - [data/uu/persistent/feature.plain.result.queue-partition-31] Consumer data.feature.c.user.deliver.and.chat.features.to.hbase meta-data recover from ledger 110091138
   2022-04-26T21:41:30.773Z INFO [bookkeeper-ml-scheduler-OrderedScheduler-4-0] org.apache.bookkeeper.mledger.impl.ManagedCursorImpl - [data/uu/persistent/feature.plain.result.queue-partition-31] Consumer data.feature.c.user.deliver.and.chat.features.to.hbase meta-data recover from ledger 110091138
   2022-04-26T21:41:30.773Z INFO [bookkeeper-ml-scheduler-OrderedScheduler-4-0] org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl - [data/uu/persistent/feature.plain.result.queue-partition-31] Loading cursor data.feature.c.user.deliver.and.chat.features.to.hbase
   2022-04-26T21:41:30.773Z INFO [bookkeeper-ml-scheduler-OrderedScheduler-4-0] org.apache.bookkeeper.mledger.impl.ManagedCursorImpl - [data/uu/persistent/feature.plain.result.queue-partition-31] Recovering from bookkeeper ledger cursor: data.feature.c.user.deliver.and.chat.features.to.hbase
   2022-04-26T21:42:30.888Z INFO [bookkeeper-ml-scheduler-OrderedScheduler-4-0] org.apache.bookkeeper.mledger.impl.ManagedCursorImpl - [data/uu/persistent/feature.plain.result.queue-partition-31] Recovering from bookkeeper ledger cursor: data.feature.c.user.deliver.and.chat.features.to.hbase
   2022-04-26T21:42:30.888Z INFO [bookkeeper-ml-scheduler-OrderedScheduler-4-0] org.apache.bookkeeper.mledger.impl.ManagedCursorImpl - [data/uu/persistent/feature.plain.result.queue-partition-31] Consumer data.feature.c.user.deliver.and.chat.features.to.hbase meta-data recover from ledger 110091138
   2022-04-26T21:42:30.888Z INFO [bookkeeper-ml-scheduler-OrderedScheduler-4-0] org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl - [data/uu/persistent/feature.plain.result.queue-partition-31] Loading cursor data.feature.c.user.deliver.and.chat.features.to.hbase
   2022-04-26T21:43:31.106Z INFO [bookkeeper-ml-scheduler-OrderedScheduler-4-0] org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl - [data/uu/persistent/feature.plain.result.queue-partition-31] Loading cursor data.feature.c.user.deliver.and.chat.features.to.hbase
   2022-04-26T21:43:31.106Z INFO [bookkeeper-ml-scheduler-OrderedScheduler-4-0] org.apache.bookkeeper.mledger.impl.ManagedCursorImpl - [data/uu/persistent/feature.plain.result.queue-partition-31] Recovering from bookkeeper ledger cursor: data.feature.c.user.deliver.and.chat.features.to.hbase
   2022-04-26T21:43:31.106Z INFO [bookkeeper-ml-scheduler-OrderedScheduler-4-0] org.apache.bookkeeper.mledger.impl.ManagedCursorImpl - [data/uu/persistent/feature.plain.result.queue-partition-31] Consumer data.feature.c.user.deliver.and.chat.features.to.hbase meta-data recover from ledger 110091138
   2022-04-26T21:44:31.391Z INFO [bookkeeper-ml-scheduler-OrderedScheduler-4-0] org.apache.bookkeeper.mledger.impl.ManagedCursorImpl - [data/uu/persistent/feature.plain.result.queue-partition-31] Recovering from bookkeeper ledger cursor: data.feature.c.user.deliver.and.chat.features.to.hbase
   2022-04-26T21:44:31.391Z INFO [bookkeeper-ml-scheduler-OrderedScheduler-4-0] org.apache.bookkeeper.mledger.impl.ManagedCursorImpl - [data/uu/persistent/feature.plain.result.queue-partition-31] Consumer data.feature.c.user.deliver.and.chat.features.to.hbase meta-data recover from ledger 110091138
   2022-04-26T21:44:31.391Z INFO [bookkeeper-ml-scheduler-OrderedScheduler-4-0] org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl - [data/uu/persistent/feature.plain.result.queue-partition-31] Loading cursor data.feature.c.user.deliver.and.chat.features.to.hbase
   ```
   
   And it will start a new recover op after `managedLedgerMetadataOperationsTimeoutSeconds`, so bookie keep logging `Ledger fence request received for ledger`:
   <img width="1648" alt="image" src="https://user-images.githubusercontent.com/5058708/165911321-a717e5e1-c161-4631-b5e2-454e3a6321e1.png">
   
   After near 27h, the recover op finally callback (Actually the topic has moved to another node in this moment):
   ```
   2022-04-28T00:25:36.147Z INFO [BookKeeperClientWorker-OrderedExecutor-2-0] org.apache.bookkeeper.mledger.impl.ManagedCursorImpl - [data/uu/persistent/feature.plain.result.queue-partition-31] Opened ledger 110091138 for consumer data.feature.c.user.deliver.and.chat.features.to.hbase. rc=0
   2022-04-28T00:25:36.163Z INFO [BookKeeperClientWorker-OrderedExecutor-2-0] org.apache.bookkeeper.mledger.impl.ManagedCursorImpl - [data/uu/persistent/feature.plain.result.queue-partition-31] Opened ledger 110091138 for consumer data.feature.c.user.deliver.and.chat.features.to.hbase. rc=0
   2022-04-28T00:25:36.163Z INFO [BookKeeperClientWorker-OrderedExecutor-2-0] org.apache.bookkeeper.mledger.impl.ManagedCursorImpl - [data/uu/persistent/feature.plain.result.queue-partition-31] Opened ledger 110091138 for consumer data.feature.c.user.deliver.and.chat.features.to.hbase. rc=0
   2022-04-28T00:25:36.172Z INFO [BookKeeperClientWorker-OrderedExecutor-2-0] org.apache.bookkeeper.mledger.impl.ManagedCursorImpl - [data/uu/persistent/feature.plain.result.queue-partition-31] Cursor data.feature.c.user.deliver.and.chat.features.to.hbase recovered to position 110398886:151969
   2022-04-28T00:25:36.173Z INFO [BookKeeperClientWorker-OrderedExecutor-2-0] org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl - [data/uu/persistent/feature.plain.result.queue-partition-31] Recovery for cursor data.feature.c.user.deliver.and.chat.features.to.hbase completed. pos=110398886:151969 -- todo=0
   2022-04-28T00:25:36.188Z INFO [BookKeeperClientWorker-OrderedExecutor-2-0] org.apache.bookkeeper.mledger.impl.ManagedCursorImpl - [data/uu/persistent/feature.plain.result.queue-partition-31] Cursor data.feature.c.user.deliver.and.chat.features.to.hbase recovered to position 110398886:151969
   2022-04-28T00:25:36.188Z INFO [BookKeeperClientWorker-OrderedExecutor-2-0] org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl - [data/uu/persistent/feature.plain.result.queue-partition-31] Recovery for cursor data.feature.c.user.deliver.and.chat.features.to.hbase completed. pos=110398886:151969 -- todo=0
   2022-04-28T00:25:36.189Z INFO [BookKeeperClientWorker-OrderedExecutor-2-0] org.apache.bookkeeper.mledger.impl.ManagedCursorImpl - [data/uu/persistent/feature.plain.result.queue-partition-31] Cursor data.feature.c.user.deliver.and.chat.features.to.hbase recovered to position 110398886:151969
   2022-04-28T00:25:36.189Z INFO [BookKeeperClientWorker-OrderedExecutor-2-0] org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl - [data/uu/persistent/feature.plain.result.queue-partition-31] Recovery for cursor data.feature.c.user.deliver.and.chat.features.to.hbase completed. pos=110398886:151969 -- todo=0
   2022-04-28T00:25:36.479Z INFO [BookKeeperClientWorker-OrderedExecutor-2-0] org.apache.bookkeeper.mledger.impl.ManagedCursorImpl - [data/uu/persistent/feature.plain.result.queue-partition-31] Opened ledger 110091138 for consumer data.feature.c.user.deliver.and.chat.features.to.hbase. rc=0
   2022-04-28T00:25:36.480Z INFO [BookKeeperClientWorker-OrderedExecutor-2-0] org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl - [data/uu/persistent/feature.plain.result.queue-partition-31] Recovery for cursor data.feature.c.user.deliver.and.chat.features.to.hbase completed. pos=110398886:151969 -- todo=0
   2022-04-28T00:25:36.480Z INFO [BookKeeperClientWorker-OrderedExecutor-2-0] org.apache.bookkeeper.mledger.impl.ManagedCursorImpl - [data/uu/persistent/feature.plain.result.queue-partition-31] Cursor data.feature.c.user.deliver.and.chat.features.to.hbase recovered to position 110398886:151969
   2022-04-28T00:25:36.487Z INFO [BookKeeperClientWorker-OrderedExecutor-2-0] org.apache.bookkeeper.mledger.impl.ManagedCursorImpl - [data/uu/persistent/feature.plain.result.queue-partition-31] Opened ledger 110091138 for consumer data.feature.c.user.deliver.and.chat.features.to.hbase. rc=0
   2022-04-28T00:25:36.487Z INFO [BookKeeperClientWorker-OrderedExecutor-2-0] org.apache.bookkeeper.mledger.impl.ManagedCursorImpl - [data/uu/persistent/feature.plain.result.queue-partition-31] Opened ledger 110091138 for consumer data.feature.c.user.deliver.and.chat.features.to.hbase. rc=0
   2022-04-28T00:25:36.487Z INFO [BookKeeperClientWorker-OrderedExecutor-2-0] org.apache.bookkeeper.mledger.impl.ManagedCursorImpl - [data/uu/persistent/feature.plain.result.queue-partition-31] Opened ledger 110091138 for consumer data.feature.c.user.deliver.and.chat.features.to.hbase. rc=0
   2022-04-28T00:25:36.487Z INFO [BookKeeperClientWorker-OrderedExecutor-2-0] org.apache.bookkeeper.mledger.impl.ManagedCursorImpl - [data/uu/persistent/feature.plain.result.queue-partition-31] Opened ledger 110091138 for consumer data.feature.c.user.deliver.and.chat.features.to.hbase. rc=0
   2022-04-28T00:25:36.487Z INFO [BookKeeperClientWorker-OrderedExecutor-2-0] org.apache.bookkeeper.mledger.impl.ManagedCursorImpl - [data/uu/persistent/feature.plain.result.queue-partition-31] Opened ledger 110091138 for consumer data.feature.c.user.deliver.and.chat.features.to.hbase. rc=0
   2022-04-28T00:25:36.487Z INFO [BookKeeperClientWorker-OrderedExecutor-2-0] org.apache.bookkeeper.mledger.impl.ManagedCursorImpl - [data/uu/persistent/feature.plain.result.queue-partition-31] Opened ledger 110091138 for consumer data.feature.c.user.deliver.and.chat.features.to.hbase. rc=0
   2022-04-28T00:25:36.489Z INFO [BookKeeperClientWorker-OrderedExecutor-2-0] org.apache.bookkeeper.mledger.impl.ManagedCursorImpl - [data/uu/persistent/feature.plain.result.queue-partition-31] Opened ledger 110091138 for consumer data.feature.c.user.deliver.and.chat.features.to.hbase. rc=0
   2022-04-28T00:25:36.489Z INFO [BookKeeperClientWorker-OrderedExecutor-2-0] org.apache.bookkeeper.mledger.impl.ManagedCursorImpl - [data/uu/persistent/feature.plain.result.queue-partition-31] Opened ledger 110091138 for consumer data.feature.c.user.deliver.and.chat.features.to.hbase. rc=0
   ```
   
   **Additional context**
   - OS: `3.10.0-1160.53.1.el7.x86_64`
   - java: `1.8.0_181`
   - pulsar: `2.8.3`
   


-- 
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] Shawyeok commented on issue #15390: Cursor recover took long time

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

   broker.conf
   ```
   zookeeperServers=xxx
   configurationStoreServers=xxx
   brokerServicePort=6650
   webServicePort=8080
   bindAddress=0.0.0.0
   advertisedAddress=172.30.92.31
   haProxyProtocolEnabled=false
   numAcceptorThreads=
   numIOThreads=32
   numOrderedExecutorThreads=8
   numHttpServerThreads=
   numExecutorThreadPoolSize=
   numCacheExecutorThreadPoolSize=10
   enableBusyWait=false
   maxConcurrentHttpRequests=1024
   isRunningStandalone=
   clusterName=pulsar-idc-b
   maxTenants=0
   failureDomainsEnabled=false
   zooKeeperSessionTimeoutMillis=30000
   zooKeeperOperationTimeoutSeconds=30
   zooKeeperCacheExpirySeconds=300
   brokerShutdownTimeoutMs=60000
   skipBrokerShutdownOnOOM=false
   backlogQuotaCheckEnabled=true
   backlogQuotaCheckIntervalInSeconds=60
   backlogQuotaDefaultLimitGB=20
   backlogQuotaDefaultLimitSecond=-1
   backlogQuotaDefaultRetentionPolicy=consumer_backlog_eviction
   ttlDurationDefaultInSeconds=0
   allowAutoTopicCreation=true
   allowAutoTopicCreationType=partitioned
   allowAutoSubscriptionCreation=true
   defaultNumPartitions=3
   brokerDeleteInactiveTopicsEnabled=true
   brokerDeleteInactiveTopicsFrequencySeconds=60
   brokerDeleteInactiveTopicsMode=delete_when_subscriptions_caught_up
   brokerDeleteInactivePartitionedTopicMetadataEnabled=false
   brokerDeleteInactiveTopicsMaxInactiveDurationSeconds=259200
   forceDeleteTenantAllowed=false
   forceDeleteNamespaceAllowed=false
   maxPendingPublishRequestsPerConnection=1000
   messageExpiryCheckIntervalInMinutes=5
   activeConsumerFailoverDelayTimeMillis=1000
   subscriptionExpirationTimeMinutes=4320
   subscriptionRedeliveryTrackerEnabled=true
   subscriptionExpiryCheckIntervalInMinutes=5
   subscriptionTypesEnabled=Exclusive,Shared,Failover,Key_Shared
   subscriptionKeySharedEnable=true
   subscriptionKeySharedUseConsistentHashing=false
   subscriptionKeySharedConsistentHashingReplicaPoints=100
   brokerDeduplicationEnabled=false
   brokerDeduplicationMaxNumberOfProducers=10000
   brokerDeduplicationSnapshotFrequencyInSeconds=10
   brokerDeduplicationSnapshotIntervalSeconds=120
   brokerDeduplicationEntriesInterval=1000
   brokerDeduplicationProducerInactivityTimeoutMinutes=360
   defaultNumberOfNamespaceBundles=4
   maxNamespacesPerTenant=0
   maxTopicsPerNamespace=0
   brokerMaxConnections=0
   brokerMaxConnectionsPerIp=0
   isAllowAutoUpdateSchemaEnabled=true
   clientLibraryVersionCheckEnabled=false
   statusFilePath=
   preferLaterVersions=false
   maxUnackedMessagesPerConsumer=500
   maxUnackedMessagesPerSubscription=5000
   maxUnackedMessagesPerBroker=0
   maxUnackedMessagesPerSubscriptionOnBrokerBlocked=0.16
   unblockStuckSubscriptionEnabled=false
   topicPublisherThrottlingTickTimeMillis=10
   preciseTopicPublishRateLimiterEnable=false
   brokerPublisherThrottlingTickTimeMillis=50
   brokerPublisherThrottlingMaxMessageRate=0
   brokerPublisherThrottlingMaxByteRate=0
   maxPublishRatePerTopicInMessages=0
   maxPublishRatePerTopicInBytes=0
   subscribeThrottlingRatePerConsumer=0
   subscribeRatePeriodPerConsumerInSecond=30
   dispatchThrottlingRatePerTopicInMsg=0
   dispatchThrottlingRatePerTopicInByte=0
   dispatchThrottlingRatePerSubscriptionInMsg=0
   dispatchThrottlingRatePerSubscriptionInByte=0
   dispatchThrottlingRatePerReplicatorInMsg=0
   dispatchThrottlingRatePerReplicatorInByte=0
   dispatchThrottlingRateRelativeToPublishRate=false
   dispatchThrottlingOnNonBacklogConsumerEnabled=true
   dispatcherMaxReadBatchSize=100
   dispatcherMaxReadSizeBytes=5242880
   dispatcherMinReadBatchSize=1
   dispatcherMaxRoundRobinBatchSize=20
   dispatcherReadFailureBackoffInitialTimeInMs=15000
   dispatcherReadFailureBackoffMaxTimeInMs=60000
   dispatcherReadFailureBackoffMandatoryStopTimeInMs=0
   preciseDispatcherFlowControl=false
   maxConcurrentLookupRequest=50000
   maxConcurrentTopicLoadRequest=5000
   maxConcurrentNonPersistentMessagePerConnection=1000
   numWorkerThreadsForNonPersistentTopic=8
   enablePersistentTopics=true
   enableNonPersistentTopics=true
   enableRunBookieTogether=false
   enableRunBookieAutoRecoveryTogether=false
   maxProducersPerTopic=10000
   maxSameAddressProducersPerTopic=0
   encryptionRequireOnProducer=false
   maxConsumersPerTopic=10000
   maxSameAddressConsumersPerTopic=0
   maxSubscriptionsPerTopic=0
   maxConsumersPerSubscription=2000
   maxMessageSize=5242880
   brokerServiceCompactionMonitorIntervalInSeconds=60
   brokerServiceCompactionThresholdInBytes=0
   brokerServiceCompactionPhaseOneLoopTimeInSeconds=30
   delayedDeliveryEnabled=true
   delayedDeliveryTickTimeMillis=1000
   acknowledgmentAtBatchIndexLevelEnabled=false
   enableReplicatedSubscriptions=true
   replicatedSubscriptionsSnapshotFrequencyMillis=1000
   replicatedSubscriptionsSnapshotTimeoutSeconds=30
   replicatedSubscriptionsSnapshotMaxCachedPerSubscription=10
   maxMessagePublishBufferSizeInMB=
   retentionCheckIntervalInSeconds=120
   maxNumPartitionsPerPartitionedTopic=0
   zookeeperSessionExpiredPolicy=shutdown
   systemTopicEnabled=true
   systemTopicSchemaCompatibilityStrategy=ALWAYS_COMPATIBLE
   topicLevelPoliciesEnabled=true
   topicFencingTimeoutSeconds=0
   proxyRoles=
   authenticateOriginalAuthData=false
   tlsEnabled=false
   tlsCertRefreshCheckDurationSec=300
   tlsCertificateFilePath=
   tlsKeyFilePath=
   tlsTrustCertsFilePath=
   tlsAllowInsecureConnection=false
   tlsProtocols=
   tlsCiphers=
   tlsRequireTrustedClientCertOnConnect=false
   tlsEnabledWithKeyStore=false
   tlsProvider=
   tlsKeyStoreType=JKS
   tlsKeyStore=
   tlsKeyStorePassword=
   tlsTrustStoreType=JKS
   tlsTrustStore=
   tlsTrustStorePassword=
   brokerClientTlsEnabledWithKeyStore=false
   brokerClientSslProvider=
   brokerClientTlsTrustStoreType=JKS
   brokerClientTlsTrustStore=
   brokerClientTlsTrustStorePassword=
   brokerClientTlsCiphers=
   brokerClientTlsProtocols=
   authenticationEnabled=false
   authenticationProviders=
   authenticationRefreshCheckSeconds=60
   authorizationEnabled=false
   authorizationProvider=org.apache.pulsar.broker.authorization.PulsarAuthorizationProvider
   authorizationAllowWildcardsMatching=false
   superUserRoles=
   brokerClientTlsEnabled=false
   brokerClientAuthenticationPlugin=
   brokerClientAuthenticationParameters=
   brokerClientTrustCertsFilePath=
   athenzDomainNames=
   anonymousUserRole=
   tokenSecretKey=
   tokenPublicKey=
   tokenAuthClaim=
   tokenAudienceClaim=
   tokenAudience=
   saslJaasClientAllowedIds=
   saslJaasBrokerSectionName=
   httpMaxRequestSize=-1
   disableHttpDebugMethods=false
   httpRequestsLimitEnabled=false
   httpRequestsMaxPerSecond=100.0
   bookkeeperMetadataServiceUri=
   bookkeeperClientAuthenticationPlugin=
   bookkeeperClientAuthenticationParametersName=
   bookkeeperClientAuthenticationParameters=
   bookkeeperClientTimeoutInSeconds=30
   bookkeeperClientNumWorkerThreads=
   bookkeeperClientSpeculativeReadTimeoutInMillis=0
   bookkeeperNumberOfChannelsPerBookie=32
   bookkeeperUseV2WireProtocol=false
   bookkeeperClientHealthCheckEnabled=true
   bookkeeperClientHealthCheckIntervalSeconds=60
   bookkeeperClientHealthCheckErrorThresholdPerInterval=5
   bookkeeperClientHealthCheckQuarantineTimeInSeconds=1800
   bookkeeperClientQuarantineRatio=1.0
   bookkeeperClientGetBookieInfoIntervalSeconds=86400
   bookkeeperClientGetBookieInfoRetryIntervalSeconds=60
   bookkeeperClientRackawarePolicyEnabled=true
   bookkeeperClientRegionawarePolicyEnabled=false
   bookkeeperClientMinNumRacksPerWriteQuorum=2
   bookkeeperClientEnforceMinNumRacksPerWriteQuorum=false
   bookkeeperClientReorderReadSequenceEnabled=false
   bookkeeperClientIsolationGroups=
   bookkeeperClientSecondaryIsolationGroups=
   bookkeeperClientMinAvailableBookiesInIsolationGroups=
   bookkeeperEnableStickyReads=false
   bookkeeperTLSProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory
   bookkeeperTLSClientAuthentication=false
   bookkeeperTLSKeyFileType=PEM
   bookkeeperTLSTrustCertTypes=PEM
   bookkeeperTLSKeyStorePasswordPath=
   bookkeeperTLSTrustStorePasswordPath=
   bookkeeperTLSKeyFilePath=
   bookkeeperTLSCertificateFilePath=
   bookkeeperTLSTrustCertsFilePath=
   bookkeeperDiskWeightBasedPlacementEnabled=false
   bookkeeperExplicitLacIntervalInMills=0
   bookkeeperClientExposeStatsToPrometheus=true
   managedLedgerDefaultEnsembleSize=2
   managedLedgerDefaultWriteQuorum=2
   managedLedgerDefaultAckQuorum=2
   bookkeeper_enableTaskExecutionStats=true
   bookkeeper_taskExecutionWarnTimeMicros=50000
   managedLedgerCursorPositionFlushSeconds = 60
   managedLedgerDigestType=CRC32C
   managedLedgerNumWorkerThreads=16
   managedLedgerNumSchedulerThreads=8
   managedLedgerCacheSizeMB=1024
   managedLedgerCacheCopyEntries=false
   managedLedgerCacheEvictionWatermark=0.9
   managedLedgerCacheEvictionFrequency=100.0
   managedLedgerCacheEvictionTimeThresholdMillis=1000
   managedLedgerCursorBackloggedThreshold=1000
   managedLedgerDefaultMarkDeleteRateLimit=1.0
   managedLedgerMaxEntriesPerLedger=50000
   managedLedgerMinLedgerRolloverTimeMinutes=10
   managedLedgerMaxLedgerRolloverTimeMinutes=240
   managedLedgerMaxSizePerLedgerMbytes=2048
   managedLedgerOffloadDeletionLagMs=14400000
   managedLedgerOffloadAutoTriggerSizeThresholdBytes=-1
   managedLedgerCursorMaxEntriesPerLedger=50000
   managedLedgerCursorRolloverTimeInSeconds=14400
   managedLedgerMaxUnackedRangesToPersist=1024
   managedLedgerMaxUnackedRangesToPersistInZooKeeper=1000
   autoSkipNonRecoverableData=true
   lazyCursorRecovery=false
   managedLedgerMetadataOperationsTimeoutSeconds=60
   managedLedgerReadEntryTimeoutSeconds=0
   managedLedgerAddEntryTimeoutSeconds=0
   managedLedgerPrometheusStatsLatencyRolloverSeconds=60
   managedLedgerTraceTaskExecution=true
   managedLedgerNewEntriesCheckDelayInMillis=10
   loadBalancerEnabled=true
   loadBalancerReportUpdateThresholdPercentage=10
   loadBalancerReportUpdateMaxIntervalMinutes=15
   loadBalancerHostUsageCheckIntervalMinutes=1
   loadBalancerSheddingEnabled=true
   loadBalancerSheddingIntervalMinutes=1
   loadBalancerSheddingGracePeriodMinutes=30
   loadBalancerBrokerMaxTopics=50000
   loadBalancerBrokerOverloadedThresholdPercentage=85
   loadBalancerResourceQuotaUpdateIntervalMinutes=15
   loadBalancerAutoBundleSplitEnabled=true
   loadBalancerAutoUnloadSplitBundlesEnabled=true
   loadBalancerNamespaceBundleMaxTopics=1000
   loadBalancerNamespaceBundleMaxSessions=1000
   loadBalancerNamespaceBundleMaxMsgRate=30000
   loadBalancerNamespaceBundleMaxBandwidthMbytes=100
   loadBalancerNamespaceMaximumBundles=128
   loadBalancerOverrideBrokerNicSpeedGbps=10
   loadManagerClassName=org.apache.pulsar.broker.loadbalance.impl.ModularLoadManagerImpl
   supportedNamespaceBundleSplitAlgorithms=range_equally_divide,topic_count_equally_divide
   defaultNamespaceBundleSplitAlgorithm=range_equally_divide
   loadBalancerLoadSheddingStrategy=org.apache.pulsar.broker.loadbalance.impl.OverloadShedder
   loadBalancerBrokerThresholdShedderPercentage=10
   loadBalancerHistoryResourcePercentage=0.9
   loadBalancerBandwithInResourceWeight=1.0
   loadBalancerBandwithOutResourceWeight=1.0
   loadBalancerCPUResourceWeight=1.0
   loadBalancerMemoryResourceWeight=1.0
   loadBalancerDirectMemoryResourceWeight=1.0
   loadBalancerBundleUnloadMinThroughputThreshold=10
   replicationMetricsEnabled=true
   replicationConnectionsPerBroker=16
   replicationProducerQueueSize=1000
   replicatorPrefix=pulsar.repl
   replicationPolicyCheckDurationSeconds=600
   defaultRetentionTimeInMinutes=0
   defaultRetentionSizeInMB=0
   keepAliveIntervalSeconds=30
   bootstrapNamespaces=
   webSocketServiceEnabled=false
   webSocketNumIoThreads=8
   webSocketConnectionsPerBroker=8
   webSocketSessionIdleTimeoutMillis=300000
   webSocketMaxTextFrameSize=1048576
   exposeTopicLevelMetricsInPrometheus=true
   exposeConsumerLevelMetricsInPrometheus=false
   exposeProducerLevelMetricsInPrometheus=false
   exposeManagedLedgerMetricsInPrometheus=true
   exposeManagedCursorMetricsInPrometheus=true
   metricsServletTimeoutMs=30000
   functionsWorkerEnabled=false
   exposePublisherStats=true
   statsUpdateFrequencyInSecs=60
   statsUpdateInitialDelayInSecs=60
   exposePreciseBacklogInPrometheus=false
   splitTopicAndPartitionLabelInPrometheus=false
   schemaRegistryStorageClassName=org.apache.pulsar.broker.service.schema.BookkeeperSchemaStorageFactory
   isSchemaValidationEnforced=false
   schemaCompatibilityStrategy=FULL
   offloadersDirectory=./offloaders
   managedLedgerOffloadDriver=
   managedLedgerOffloadMaxThreads=2
   managedLedgerOffloadPrefetchRounds=1
   managedLedgerUnackedRangesOpenCacheSetEnabled=true
   s3ManagedLedgerOffloadRegion=
   s3ManagedLedgerOffloadBucket=
   s3ManagedLedgerOffloadServiceEndpoint=
   s3ManagedLedgerOffloadMaxBlockSizeInBytes=67108864
   s3ManagedLedgerOffloadReadBufferSizeInBytes=1048576
   gcsManagedLedgerOffloadRegion=
   gcsManagedLedgerOffloadBucket=
   gcsManagedLedgerOffloadMaxBlockSizeInBytes=67108864
   gcsManagedLedgerOffloadReadBufferSizeInBytes=1048576
   gcsManagedLedgerOffloadServiceAccountKeyFile=
   fileSystemProfilePath=../conf/filesystem_offload_core_site.xml
   fileSystemURI=
   globalZookeeperServers=
   replicationTlsEnabled=false
   brokerServicePurgeInactiveFrequencyInSeconds=60
   transactionCoordinatorEnabled=false
   transactionMetadataStoreProviderClassName=org.apache.pulsar.transaction.coordinator.impl.MLTransactionMetadataStoreProvider
   transactionBufferSnapshotMaxTransactionCount=1000
   transactionBufferSnapshotMinTimeInMillis=5000
   enablePackagesManagement=false
   packagesManagementStorageProvider=org.apache.pulsar.packages.management.storage.bookkeeper.BookKeeperPackagesStorageProvider
   packagesReplicas=1
   packagesManagementLedgerRootPath=/ledgers
   ```


-- 
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 #15390: Cursor recover took extreme long time

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

   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


[GitHub] [pulsar] Shawyeok commented on issue #15390: Cursor recover took extreme time

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

   In this case we may set `lazyCursorRecovery=true` to avoid impact the whole topic, just note this topic has many subs.


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