You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by "waney316 (via GitHub)" <gi...@apache.org> on 2024/04/11 11:23:50 UTC

[D] pulsar broker load imbalance problem [pulsar]

GitHub user waney316 created a discussion: pulsar broker load imbalance problem

cluster version:3.1.1 with ack-aware placement policy
components:  3broker with 2 region , 3 bookie with 2 region,broker and bookie are mixed deploy.

problem: 
One of the brokers has a very high load, which causes its jvm heap memory and gc time to be long, seriously affecting the business.
,but its bundle_msg_in/out/producer_count/topic_count is not the largest,

The following is relevant information,can someone help me?

1: 10.50.100.197 broker has high publish latency
![image](https://github.com/apache/pulsar/assets/23314093/ed7c9319-a511-4e39-b334-a6976a4b45bc)

2: 10.50.100.197 broker has high jvm heap use and long gc time 
![image](https://github.com/apache/pulsar/assets/23314093/0d9aeeec-4913-4bee-b272-c9d275d9cb19)

3:broker has almost average bundle msg_rate/topic count 
![image](https://github.com/apache/pulsar/assets/23314093/7ccd6fda-74b9-4a6e-99d0-352fa2dc98b8)


4: broker config 
```
loadBalancerNamespaceBundleMaxBandwidthMbytes    100
managedLedgerMaxUnackedRangesToPersistInMetadataStore    1000
numExecutorThreadPoolSize    32
messagePublishBufferCheckIntervalInMillis    100
bookkeeperClientNumIoThreads    64
exposePreciseBacklogInPrometheus    false
managedLedgerCursorBackloggedThreshold    1000
transactionLogBatchedWriteMaxRecords    512
bookkeeperClientHealthCheckErrorThresholdPerInterval    5
brokerDeduplicationEntriesInterval    1000
isDelayedDeliveryDeliverAtTimeStrict    false
anonymousUserRole    
jvmGCMetricsLoggerClassName    
packagesManagementStorageProvider    org.apache.pulsar.packages.management.storage.bookkeeper.BookKeeperPackagesStorageProvider
brokerClientTlsKeyStoreType    JKS
loadBalancerAutoBundleSplitEnabled    false
loadBalancerSheddingBundlesWithPoliciesEnabled    false
managedLedgerDefaultAckQuorum    2
forceDeleteNamespaceAllowed    false
webSocketConnectionsPerBroker    32
loadBalancerSplitIntervalMinutes    1
delayedDeliveryEnabled    true
managedCursorInfoCompressionType    NONE
functionsWorkerEnabled    false
backlogQuotaDefaultLimitBytes    -1
bookkeeperClientEnforceMinNumRacksPerWriteQuorum    false
brokerClientTlsEnabled    false
kinitCommand    /usr/bin/kinit
replicatedSubscriptionsSnapshotMaxCachedPerSubscription    10
httpServerThreadPoolQueueSize    8192
loadBalancerReportUpdateMinIntervalMillis    5000
maxNumPartitionsPerPartitionedTopic    0
managedLedgerMinLedgerRolloverTimeMinutes    10
messagingProtocols    []
superUserRoles    [admin]
clusterName    pulsar-gz1
authenticationProviders    [org.apache.pulsar.broker.authentication.AuthenticationProviderToken]
protocolHandlerDirectory    ./protocols
dispatcherMaxReadSizeBytes    5242880
tlsCiphers    []
maxNamespacesPerTenant    0
exposeManagedLedgerMetricsInPrometheus    true
loadBalancerBrokerComfortLoadLevelPercentage    65
loadBalancerServiceUnitStateTombstoneDelayTimeInSeconds    3600
useSeparateThreadPoolForProtocolHandlers    true
maxSameAddressProducersPerTopic    0
loadBalancerDistributeBundlesEvenlyEnabled    true
brokerDeduplicationEnabled    false
topicOrderedExecutorThreadNum    32
transactionPendingAckBatchedWriteMaxDelayInMillis    1
bookkeeperTLSTrustCertTypes    PEM
advertisedListeners    
metricsBufferResponse    false
exposeManagedCursorMetricsInPrometheus    false
numHttpServerThreads    64
managedLedgerCacheEvictionWatermark    0.9
httpRequestsFailOnUnknownPropertiesEnabled    false
webSocketSessionIdleTimeoutMillis    300000
loadBalancerTransferEnabled    true
allowAutoTopicCreationType    non-partitioned
acknowledgmentAtBatchIndexLevelEnabled    false
managedLedgerMaxSizePerLedgerMbytes    2048
exposePublisherStats    true
tlsKeyStoreType    JKS
maxUnackedMessagesPerConsumer    50000
saslJaasServerSectionName    PulsarBroker
managedCursorInfoCompressionThresholdInBytes    16384
disableBrokerInterceptors    true
metadataStoreConfigPath    
maxConcurrentHttpRequests    1024
loadBalancerMsgThroughputMultiplierDifferenceShedderThreshold    4.0
dispatcherMaxReadBatchSize    500
managedLedgerMinimumBacklogCursorsForCaching    0
transactionLogBatchedWriteEnabled    false
brokerClientAuthenticationPlugin    org.apache.pulsar.client.impl.auth.AuthenticationToken
loadBalancerMemoryResourceWeight    1.0
subscribeThrottlingRatePerConsumer    0
brokerClientTlsKeyStorePassword    
numAcceptorThreads    1
proxyRoles    []
metadataStoreBatchingEnabled    true
loadBalancerReportUpdateThresholdPercentage    10
backlogQuotaDefaultRetentionPolicy    producer_request_hold
subscriptionKeySharedEnable    true
managedLedgerMinimumBacklogEntriesForCaching    1000
resourceUsageTransportClassName    
maxConcurrentTopicLoadRequest    5000
enablePersistentTopics    true
resourceUsageTransportPublishIntervalInSecs    60
persistentUnackedRangesWithMultipleEntriesEnabled    false
dispatcherEntryFilterRescheduledMessageDelay    1000
managedLedgerMaxReadsInFlightSizeInMB    0
dispatcherReadFailureBackoffMaxTimeInMs    60000
entryFilterNames    []
authenticationRefreshCheckSeconds    60
dispatcherMaxRoundRobinBatchSize    20
maxPendingPublishRequestsPerConnection    1000
delayedDeliveryFixedDelayDetectionLookahead    50000
enableBrokerSideSubscriptionPatternEvaluation    true
webSocketMaxTextFrameSize    1048576
backlogQuotaCheckEnabled    true
loadBalancerMaxNumberOfBundlesToSplitPerCycle    10
bookkeeperClientIsolationGroups    
strictBookieAffinityEnabled    false
namespaceBundleUnloadingTimeoutMs    60000
subscriptionPatternMaxLength    50
configurationStoreServers    
brokerInterceptors    []
brokerClientTlsProtocols    []
additionalServlets    []
topicBundleAssignmentStrategy    org.apache.pulsar.common.naming.ConsistentHashingTopicBundleAssigner
exposeConsumerLevelMetricsInPrometheus    false
loadBalancerSheddingIntervalMinutes    1
dispatcherDispatchMessagesInSubscriptionThread    true
loadManagerClassName    org.apache.pulsar.broker.loadbalance.impl.ModularLoadManagerImpl
metadataSyncEventTopic    
loadBalancerSheddingGracePeriodMinutes    30
numCacheExecutorThreadPoolSize    10
brokerDeduplicationSnapshotIntervalSeconds    120
loadBalancerBandwithOutResourceWeight    1.0
tlsTrustCertsFilePath    
maxPublishRatePerTopicInBytes    0
managedLedgerCacheEvictionFrequency    0.0
statsUpdateFrequencyInSecs    60
bookkeeperClientHealthCheckEnabled    true
managedLedgerInfoCompressionThresholdInBytes    16384
maxMessageSize    5242880
loadBalancerBrokerMaxTopics    50000
loadBalancerLoadSheddingStrategy    org.apache.pulsar.broker.loadbalance.impl.ThresholdShedder
loadBalancerSheddingConditionHitCountThreshold    3
configurationMetadataStoreUrl    xxxxx:2184
managedLedgerMaxEntriesPerLedger    50000
loadBalancerNamespaceMaximumBundles    128
managedLedgerCacheCopyEntries    false
tlsCertRefreshCheckDurationSec    300
saslJaasClientAllowedIds    .*pulsar.*
bindAddress    0.0.0.0
transactionMetadataStoreProviderClassName    org.apache.pulsar.transaction.coordinator.impl.MLTransactionMetadataStoreProvider
tlsAllowInsecureConnection    false
numWorkerThreadsForNonPersistentTopic    -1
httpMaxRequestSize    -1
bookkeeperClientAuthenticationParameters    
numIOThreads    64
delayedDeliveryMaxIndexesPerBucketSnapshotSegment    5000
loadBalancerBrokerLoadDataTTLInSeconds    1800
brokerDeleteInactiveTopicsMaxInactiveDurationSeconds    
enableReplicatedSubscriptions    true
maxSubscriptionsPerTopic    0
enableBusyWait    false
topicLevelPoliciesEnabled    true
bookkeeperClientSeparatedIoThreadsEnabled    false
dispatchThrottlingRatePerSubscriptionInMsg    0
maxConsumersPerTopic    0
backlogQuotaDefaultLimitSecond    -1
compactionServiceFactoryClassName    org.apache.pulsar.compaction.PulsarCompactionServiceFactory
managedLedgerMaxAckQuorum    5
exposeBundlesMetricsInPrometheus    true
metadataStoreUrl    xxxxx:2181
dispatchThrottlingRatePerTopicInByte    0
subscriptionRedeliveryTrackerEnabled    true
globalZookeeperServers    
activeConsumerFailoverDelayTimeMillis    1000
managedLedgerDefaultEnsembleSize    2
metadataStoreBatchingMaxOperations    1000
loadBalancerHistoryResourcePercentage    0.9
bookkeeperTlsCertFilesRefreshDurationSeconds    300
metadataStoreBatchingMaxSizeKb    128
transactionBufferSnapshotMaxTransactionCount    1000
loadBalancerOverrideBrokerNicSpeedGbps    Optional[20.0]
defaultRetentionSizeInMB    0
metadataStoreSessionTimeoutMillis    30000
allowOverrideEntryFilters    false
enableRunBookieAutoRecoveryTogether    false
topicFencingTimeoutSeconds    0
bookkeeperTLSProviderFactoryClass    org.apache.bookkeeper.tls.TLSContextFactory
managedLedgerOffloadThresholdInSeconds    -1
brokerClientSslProvider    
strictTopicNameEnabled    false
maxConcurrentLookupRequest    50000
bookkeeperClientLimitStatsLogging    true
replicationTlsEnabled    false
bookkeeperTLSKeyStorePasswordPath    
brokerDeduplicationSnapshotFrequencyInSeconds    120
bookkeeperExplicitLacIntervalInMills    0
replicatedSubscriptionsSnapshotTimeoutSeconds    30
managedLedgerDefaultWriteQuorum    2
statusFilePath    
brokerServicePurgeInactiveFrequencyInSeconds    60
loadBalancerPlacementStrategy    leastLoadedServer
webServiceTlsProvider    Conscrypt
brokerDeduplicationMaxNumberOfProducers    10000
bookkeeperClientAuthenticationPlugin    
metadataStoreCacheExpirySeconds    300
httpRequestsMaxPerSecond    100.0
systemTopicSchemaCompatibilityStrategy    ALWAYS_COMPATIBLE
topicFactoryClassName    
tlsTrustStoreType    JKS
statsUpdateInitialDelayInSecs    60
authorizationAllowWildcardsMatching    false
subscriptionKeySharedConsistentHashingReplicaPoints    100
bookkeeperClientQuarantineRatio    1.0
maxUnackedMessagesPerBroker    0
subscriptionExpiryCheckIntervalInMinutes    5
tlsKeyStore    
cacheEvictionByMarkDeletedPosition    false
splitTopicAndPartitionLabelInPrometheus    false
exposeSubscriptionBacklogSizeInPrometheus    false
numTransactionReplayThreadPoolSize    32
tlsEnabledWithKeyStore    false
managedLedgerCacheEvictionTimeThresholdMillis    1000
metadataStoreAllowReadOnlyOperations    false
brokerServicePortTls    Optional[6651]
bookkeeperClientSpeculativeReadTimeoutInMillis    0
brokerEntryMetadataInterceptors    []
managedLedgerCacheEvictionIntervalMs    10
functionsWorkerServiceNarPackage    
transactionPendingAckBatchedWriteMaxSize    4194304
retentionCheckIntervalInSeconds    120
loadBalancerBundleUnloadMinThroughputThreshold    10.0
bookkeeperTLSClientAuthentication    false
preferLaterVersions    false
metricsServletTimeoutMs    30000
maxActiveTransactionsPerCoordinator    0
loadBalancerMaxNumberOfBundlesInBundleLoadReport    10
maxUnackedMessagesPerSubscription    200000
tlsTrustStorePassword    
failureDomainsEnabled    false
internalListenerName    
brokerServiceCompactionPhaseOneLoopTimeInSeconds    30
forceDeleteTenantAllowed    false
webSocketNumIoThreads    32
subscriptionKeySharedUseConsistentHashing    true
brokerPublisherThrottlingTickTimeMillis    50
loadBalancerBrokerLoadTargetStd    0.25
loadBalancerLoadPlacementStrategy    org.apache.pulsar.broker.loadbalance.impl.LeastLongTermMessageRate
managedLedgerPassword    
allowAutoSubscriptionCreation    true
messageExpiryCheckIntervalInMinutes    5
loadBalancerHostUsageCheckIntervalMinutes    1
loadBalancerResourceQuotaUpdateIntervalMinutes    15
exposeTopicLevelMetricsInPrometheus    true
tlsCertificateFilePath    
dispatchThrottlingRatePerSubscriptionInByte    0
delayedDeliveryTickTimeMillis    1000
brokerClientCertificateFilePath    
brokerPublisherThrottlingMaxByteRate    0
supportedNamespaceBundleSplitAlgorithms    [range_equally_divide, topic_count_equally_divide, specified_positions_divide, flow_or_qps_equally_divide]
schemaCompatibilityStrategy    FULL
enableNamespaceIsolationUpdateOnTime    false
exposeProducerLevelMetricsInPrometheus    false
bookkeeperClientNumWorkerThreads    32
authenticateOriginalAuthData    false
transactionPendingAckBatchedWriteMaxRecords    512
managedLedgerDigestType    CRC32C
bootstrapNamespaces    []
bookkeeperClientAuthenticationParametersName    
subscriptionTypesEnabled    [Exclusive, Shared, Failover, Key_Shared]
loadBalancerBrokerThresholdShedderPercentage    10
tlsRequireTrustedClientCertOnConnect    false
brokerServicePort    Optional[6650]
loadBalancerReportUpdateMaxIntervalMinutes    15
tlsHostnameVerificationEnabled    false
managedLedgerNumSchedulerThreads    32
httpServerAcceptQueueSize    8192
transactionLogBatchedWriteMaxDelayInMillis    1
clientLibraryVersionCheckEnabled    false
topicLoadTimeoutSeconds    60
bookkeeperClientReorderReadSequenceEnabled    false
brokerShutdownTimeoutMs    60000
maxConsumersPerSubscription    0
transactionPendingAckLogIndexMinLag    500
minUnloadMessageThroughput    1048576
brokerDeleteInactivePartitionedTopicMetadataEnabled    false
maxConsumerMetadataSize    1024
brokerClientTlsTrustStore    
tlsEnabled    false
tlsProvider    
transactionCoordinatorEnabled    false
bookkeeperTLSTrustCertsFilePath    
loadBalancerMsgRateDifferenceShedderThreshold    50.0
delayedDeliveryTrackerFactoryClassName    org.apache.pulsar.broker.delayed.InMemoryDelayedDeliveryTrackerFactory
transactionBufferClientOperationTimeoutInMills    3000
brokerClientTlsCiphers    []
dispatchThrottlingRateRelativeToPublishRate    false
webServicePortTls    Optional[8443]
saslJaasServerRoleTokenSignerSecretPath    
schemaRegistryStorageClassName    org.apache.pulsar.broker.service.schema.BookkeeperSchemaStorageFactory
haProxyProtocolEnabled    false
metadataStoreBatchingMaxDelayMillis    5
bookkeeperUseV2WireProtocol    true
loadBalanceSheddingDelayInSeconds    180
disableHttpDebugMethods    false
offloadersDirectory    ./offloaders
managedLedgerCacheSizeMB    4915
maxUnloadPercentage    0.2
bookkeeperClientRegionawarePolicyEnabled    true
dispatchThrottlingRatePerReplicatorInByte    0
dispatchThrottlingOnNonBacklogConsumerEnabled    true
topicPublisherThrottlingTickTimeMillis    10
webSocketServiceEnabled    false
bookkeeperTLSCertificateFilePath    
loadBalancerNamespaceBundleMaxMsgRate    30000
bookkeeperClientHealthCheckIntervalSeconds    60
metadataStoreOperationTimeoutSeconds    30
replicationConnectionsPerBroker    16
bookkeeperClientExposeStatsToPrometheus    false
bookkeeperTLSTrustStorePasswordPath    
maxSameAddressConsumersPerTopic    0
managedLedgerMetadataOperationsTimeoutSeconds    60
webServicePort    Optional[8080]
brokerMaxConnections    0
brokerInterceptorsDirectory    ./interceptors
enableNonPersistentTopics    true
managedLedgerPrometheusStatsLatencyRolloverSeconds    60
loadBalancerNamespaceBundleMaxTopics    1000
brokerClientTlsTrustStorePassword    
managedLedgerCursorMaxEntriesPerLedger    50000
lowerBoundarySheddingEnabled    false
managedLedgerOffloadMaxThreads    2
dispatcherReadFailureBackoffInitialTimeInMs    15000
additionalServletDirectory    ./brokerAdditionalServlet
autoShrinkForConsumerPendingAcksMap    false
managedLedgerOffloadDriver    
brokerDeleteInactiveTopicsEnabled    false
configurationMetadataSyncEventTopic    
bookkeeperClientRackawarePolicyEnabled    true
maxUnloadBundleNumPerShedding    -1
managedLedgerMaxUnackedRangesToPersist    10000
delayedDeliveryMaxNumBuckets    -1
managedLedgerTraceTaskExecution    true
managedLedgerStorageClassName    org.apache.pulsar.broker.ManagedLedgerClientFactory
managedLedgerMaxEnsembleSize    5
loadBalancerAutoUnloadSplitBundlesEnabled    false
brokerClientKeyFilePath    
transactionBufferProviderClassName    org.apache.pulsar.broker.transaction.buffer.impl.TopicTransactionBufferProvider
encryptionRequireOnProducer    false
loadBalancerBrokerUnderloadedThresholdPercentage    50
delayedDeliveryMinIndexCountPerBucket    50000
replicatedSubscriptionsSnapshotFrequencyMillis    1000
zooKeeperOperationTimeoutSeconds    -1
bookkeeperTLSKeyFilePath    
managedLedgerDataReadPriority    tiered-storage-first
loadBalancerCPUResourceWeight    1.0
replicationPolicyCheckDurationSeconds    600
allowAutoTopicCreation    false
loadBalancerBrokerOverloadedThresholdPercentage    85
flowOrQpsDifferenceThresholdPercentage    10
dispatchThrottlingRateInByte    0
aggregatePublisherStatsByProducerName    false
maxUnackedMessagesPerSubscriptionOnBrokerBlocked    0.16
lazyCursorRecovery    false
httpRequestsLimitEnabled    false
brokerClientTlsKeyStore    
authorizationEnabled    true
managedLedgerCursorPositionFlushSeconds    60
bookkeeperClientHealthCheckQuarantineTimeInSeconds    1800
managedLedgerOffloadDeletionLagMs    14400000
narExtractionDirectory    /tmp
backlogQuotaCheckIntervalInSeconds    60
loadBalancerBandwithInResourceWeight    1.0
bookkeeperClientTimeoutInSeconds    30
transactionBufferClientMaxConcurrentRequests    1000
subscribeRatePeriodPerConsumerInSecond    30
brokerEntryPayloadProcessors    []
replicatorPrefix    pulsar.repl
autoSkipNonRecoverableData    false
dispatchThrottlingRateInMsg    0
dispatchThrottlingForFilteredEntriesEnabled    false
transactionPendingAckStoreProviderClassName    org.apache.pulsar.broker.transaction.pendingack.impl.MLPendingAckStoreProvider
managedLedgerOffloadPrefetchRounds    1
loadBalancerAverageResourceUsageDifferenceThresholdPercentage    10
brokerDeleteInactiveTopicsFrequencySeconds    60
managedLedgerReadEntryTimeoutSeconds    0
loadBalancerMaxNumberOfBrokerSheddingPerCycle    3
enablePackagesManagement    false
loadBalancerSheddingEnabled    true
managedLedgerAddEntryTimeoutSeconds    0
loadBalancerDirectMemoryResourceWeight    0.0
exposingBrokerEntryMetadataToClientEnabled    false
managedLedgerInactiveLedgerRolloverTimeSeconds    0
dispatchThrottlingRatePerReplicatorInMsg    0
isRunningStandalone    false
loadBalancerNamespaceBundleSplitConditionHitCountThreshold    3
maxMessagePublishBufferSizeInMB    12288
maxProducersPerTopic    0
subscriptionExpirationTimeMinutes    0
maxConcurrentNonPersistentMessagePerConnection    1000
clusterMigrationCheckDurationSeconds    0
dispatcherMinReadBatchSize    1
managedLedgerNewEntriesCheckDelayInMillis    10
webSocketPingDurationSeconds    -1
minUnloadMessage    1000
authenticationEnabled    true
isSchemaValidationEnforced    false
maxTopicsPerNamespace    0
bookkeeperMetadataServiceUri    
bookkeeperEnableStickyReads    true
brokerClientTlsEnabledWithKeyStore    false
brokerClientTlsTrustStoreType    JKS
bookkeeperClientGetBookieInfoIntervalSeconds    86400
bookkeeperClientGetBookieInfoRetryIntervalSeconds    60
transactionPendingAckBatchedWriteEnabled    false
bookkeeperClientMinNumRacksPerWriteQuorum    2
brokerDeleteInactiveTopicsMode    delete_when_no_subscriptions
numOrderedExecutorThreads    8
managedLedgerCursorRolloverTimeInSeconds    14400
defaultNumberOfNamespaceBundles    12
preciseTimeBasedBacklogQuotaCheck    false
loadBalancerEnabled    true
entryFiltersDirectory    
brokerServiceCompactionThresholdInBytes    0
dispatchThrottlingOnBatchMessageEnabled    false
authenticateMetricsEndpoint    false
bindAddresses    
authorizationProvider    org.apache.pulsar.broker.authorization.PulsarAuthorizationProvider
tlsTrustStore    
zookeeperServers    
transactionBufferSegmentedSnapshotEnabled    false
brokerClientTrustCertsFilePath    
maxHttpServerConnections    2048
schemaRegistryCompatibilityCheckers    [org.apache.pulsar.broker.service.schema.JsonSchemaCompatibilityCheck, org.apache.pulsar.broker.service.schema.AvroSchemaCompatibilityCheck, org.apache.pulsar.broker.service.schema.ProtobufNativeSchemaCompatibilityCheck]
defaultNumPartitions    1
dispatcherReadFailureBackoffMandatoryStopTimeInMs    0
packagesReplicas    1
bookkeeperNumberOfChannelsPerBookie    128
unblockStuckSubscriptionEnabled    false
subscriptionBacklogScanMaxEntries    10000
transactionBufferSnapshotMinTimeInMillis    5000
webServiceTlsCiphers    []
transactionBufferSnapshotSegmentSize    262144
inflightSaslContextExpiryMs    30000
preciseDispatcherFlowControl    false
skipBrokerShutdownOnOOM    false
webSocketNumServiceThreads    20
defaultRetentionTimeInMinutes    0
brokerPublisherThrottlingMaxMessageRate    0
replicationProducerQueueSize    1000
loadBalancerNamespaceBundleMaxSessions    1000
tlsProtocols    []
zooKeeperSessionTimeoutMillis    -1
webServiceTlsProtocols    []
managedLedgerStatsPeriodSeconds    60
bookkeeperClientSecondaryIsolationGroups    
brokerDeduplicationProducerInactivityTimeoutMinutes    360
tlsKeyFilePath    
managedLedgerMaxBacklogBetweenCursorsForCaching    1000
transactionLogBatchedWriteMaxSize    4194304
managedLedgerOffloadAutoTriggerSizeThresholdBytes    -1
packagesManagementLedgerRootPath    /ledgers
bookkeeperClientThrottleValue    0
bookkeeperDiskWeightBasedPlacementEnabled    true
subscriptionBacklogScanMaxTimeMs    120000
managedLedgerDefaultMarkDeleteRateLimit    1.0
managedLedgerMaxLedgerRolloverTimeMinutes    240
maxInflightSaslContext    50000
keepAliveIntervalSeconds    30
preciseTopicPublishRateLimiterEnable    false
zookeeperSessionExpiredPolicy    reconnect
connectionLivenessCheckTimeoutMillis    5000
functionsWorkerEnablePackageManagement    false
managedLedgerMaxWriteQuorum    5
maxPublishRatePerTopicInMessages    0
replicationMetricsEnabled    true
bookkeeperTLSKeyFileType    PEM
loadBalancerDebugModeEnabled    false
ttlDurationDefaultInSeconds    0
backlogQuotaDefaultLimitGB    -1.0
zooKeeperCacheExpirySeconds    -1
defaultNamespaceBundleSplitAlgorithm    range_equally_divide
managedLedgerMaxUnackedRangesToPersistInZooKeeper    -1
brokerServiceCompactionMonitorIntervalInSeconds    60
dispatchThrottlingRatePerTopicInMsg    0
managedLedgerInfoCompressionType    NONE
managedLedgerUnackedRangesOpenCacheSetEnabled    true
brokerMaxConnectionsPerIp    0
isAllowAutoUpdateSchemaEnabled    true
delayedDeliveryMaxTimeStepPerBucketSnapshotSegmentSeconds    300
enableRunBookieTogether    false
systemTopicEnabled    true
zooKeeperAllowReadOnlyOperations    false
tlsKeyStorePassword    
httpMaxRequestHeaderSize    8192
```





GitHub link: https://github.com/apache/pulsar/discussions/22483

----
This is an automatically sent email for commits@pulsar.apache.org.
To unsubscribe, please send an email to: commits-unsubscribe@pulsar.apache.org


Re: [D] pulsar broker load imbalance problem [pulsar]

Posted by "waney316 (via GitHub)" <gi...@apache.org>.
GitHub user waney316 added a comment to the discussion: pulsar broker load imbalance problem

> Any reason why these are disabled? I think [their defaults are to be enabled](https://github.com/apache/pulsar/blob/cea1a9ba9b576bf43f0a45ff8d65369b0f2bbb36/conf/broker.conf#L1296C1-L1296C35)...
> 
> ![image](https://private-user-images.githubusercontent.com/151540338/321645474-fca20ff4-f5e9-41ca-a30a-f1add502c748.png?jwt=eyJhbGciOiJIUzI1NiIsInR5cCI6IkpXVCJ9.eyJpc3MiOiJnaXRodWIuY29tIiwiYXVkIjoicmF3LmdpdGh1YnVzZXJjb250ZW50LmNvbSIsImtleSI6ImtleTUiLCJleHAiOjE3MTI5MDMyMzcsIm5iZiI6MTcxMjkwMjkzNywicGF0aCI6Ii8xNTE1NDAzMzgvMzIxNjQ1NDc0LWZjYTIwZmY0LWY1ZTktNDFjYS1hMzBhLWYxYWRkNTAyYzc0OC5wbmc_WC1BbXotQWxnb3JpdGhtPUFXUzQtSE1BQy1TSEEyNTYmWC1BbXotQ3JlZGVudGlhbD1BS0lBVkNPRFlMU0E1M1BRSzRaQSUyRjIwMjQwNDEyJTJGdXMtZWFzdC0xJTJGczMlMkZhd3M0X3JlcXVlc3QmWC1BbXotRGF0ZT0yMDI0MDQxMlQwNjIyMTdaJlgtQW16LUV4cGlyZXM9MzAwJlgtQW16LVNpZ25hdHVyZT1iZGFjN2U4OTNkMzdhZWI1YjVlMjU1NmI5ZDk1Zjg5OGVkOWFiOGE1ZjRkZjVhZGI4Zjk3ODNiZWVhMTBkNzlkJlgtQW16LVNpZ25lZEhlYWRlcnM9aG9zdCZhY3Rvcl9pZD0wJmtleV9pZD0wJnJlcG9faWQ9MCJ9.2czk2Ryxg1GlLHZdfZh5bOKSBn5lcwVhhV-WdvityO8)
> 
> Is it possible those switches being off prevents the system from splitting load across brokers?
> 
> See here -> https://pulsar.apache.org/docs/3.2.x/administration-load-balance/#split-namespace-bundles

Because  previously considered turning on loadBalancerAutoBundleSplitEnabled would cause client fluctuations,so the configuration items of 6 brokers are all true,If there is a problem with this configuration, why there is no big difference in the number of bundles on each broker?


GitHub link: https://github.com/apache/pulsar/discussions/22483#discussioncomment-9091871

----
This is an automatically sent email for commits@pulsar.apache.org.
To unsubscribe, please send an email to: commits-unsubscribe@pulsar.apache.org


Re: [D] pulsar broker load imbalance problem [pulsar]

Posted by "slawrencemd (via GitHub)" <gi...@apache.org>.
GitHub user slawrencemd added a comment to the discussion: pulsar broker load imbalance problem

Any reason why these are disabled? I think [their defaults are to be enabled](https://github.com/apache/pulsar/blob/cea1a9ba9b576bf43f0a45ff8d65369b0f2bbb36/conf/broker.conf#L1296C1-L1296C35)...


![image](https://github.com/apache/pulsar/assets/151540338/fca20ff4-f5e9-41ca-a30a-f1add502c748)

Is it possible those switches being off prevents the system from splitting load across brokers?

See here -> https://pulsar.apache.org/docs/3.2.x/administration-load-balance/#split-namespace-bundles

GitHub link: https://github.com/apache/pulsar/discussions/22483#discussioncomment-9084264

----
This is an automatically sent email for commits@pulsar.apache.org.
To unsubscribe, please send an email to: commits-unsubscribe@pulsar.apache.org