streamnative / pulsar-archived

Apache Pulsar - distributed pub-sub messaging system
https://pulsar.apache.org
Apache License 2.0
72 stars 25 forks source link

EnforceMinNumRacksPerWriteQuorum - not working as expected #3149

Open disserakt opened 3 years ago

disserakt commented 3 years ago

Hi to all! This issues based on a slack thread: https://apache-pulsar.slack.com/archives/C5Z4T36F7/p1633367470050400

Our installation Pulsar and Bookkeeper is on LXC containers. Pulsar version for now - 2.7.2 and Bookkeeper - 4.12.0 and cluster consists of 6 brokers Pulsar and 9 bookies Bookkeeper. Bookkeeper bookies are distributed over 3 racks and we set the parameter bookkeeperClientMinNumRacksPerWriteQuorum=2 on brokers and minNumRacksPerWriteQuorum=2 on bookies, to distribute ledgers evenly across at least 2 racks.

Moreover, the parameters for quorums and ensemble were as follows:

managedLedgerDefaultAckQuorum=3
managedLedgerDefaultEnsembleSize=3
managedLedgerDefaultWriteQuorum=3

But since this parameter bookkeeperClientMinNumRacksPerWriteQuorum is of a recommendation nature - we added parameter bookkeeperClientEnforceMinNumRacksPerWriteQuorum for guaranteed distribution across racks.

When we enable parameter bookkeeperClientEnforceMinNumRacksPerWriteQuorum=true in the broker configuration and enforceMinNumRacksPerWriteQuorum=true in the bookies configuration, we observe in logs the following behavior when we not safe shutdown bookkeepers belonging to the same rack (simulating not expected data center shutdown) - new ensemble and quorum cannot be selected:

{
    "log.level": "WARN",
    "message": "Failed to find 1 bookies : excludeBookies [<Bookie:bk-perf05.msk.ru:3181>, <Bookie:bk-perf02.msk.ru:3181>, <Bookie:bk-perf03.msk.ru:3181>, <Bookie:bk-perf06.msk.ru:3181>, <Bookie:bk-perf04.msk.ru:3181>, <Bookie:bk-perf08.msk.ru:3181>, <Bookie:bk-perf01.msk.ru:3181>], allBookies [<Bookie:bk-perf03.msk.ru:3181>, <Bookie:bk-perf06.msk.ru:3181>, <Bookie:bk-perf05.msk.ru:3181>, <Bookie:bk-perf01.msk.ru:3181>, <Bookie:bk-perf04.msk.ru:3181>, <Bookie:bk-perf02.msk.ru:3181>].",
    "process.thread.name": "BookKeeperClientWorker-OrderedExecutor-19-0",
    "log.logger": "org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl"
}
{
    "log.level": "ERROR",
    "message": "Failed to choose a bookie excluding Racks: [/dataline, /dataspace] Nodes: [<Bookie:bk-perf05.msk.ru:3181>, <Bookie:bk-perf03.msk.ru:3181>, <Bookie:bk-perf06.msk.ru:3181>, <Bookie:bk-perf04.msk.ru:3181>, <Bookie:bk-perf08.msk.ru:3181>], enforceMinNumRacksPerWriteQuorum is enabled so giving up.",
    "process.thread.name": "BookKeeperClientWorker-OrderedExecutor-19-0",
    "log.logger": "org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl"
}

And after that we can no longer continue writing to the some topics, as well as reading - where most likely AQ for ledgers ended up on those bookies that we turned off - that is, on bookies with same rack. In this case, we see the following errors in the logs:

{
    "log.level": "ERROR",
    "message": "Could not get additional bookie to remake ensemble, closing ledger: 2547",
    "process.thread.name": "BookKeeperClientWorker-OrderedExecutor-19-0",
    "log.logger": "org.apache.bookkeeper.client.ReadOnlyLedgerHandle"
}
{
    "log.level": "ERROR",
    "message": "Write of ledger entry to quorum failed: L2547 E561",
    "process.thread.name": "BookKeeperClientWorker-OrderedExecutor-19-0",
    "log.logger": "org.apache.bookkeeper.client.PendingAddOp"
}
{
    "log.level": "ERROR",
    "message": "Failure NotEnoughBookiesException: Not enough non-faulty bookies available while writing entry: 562 while recovering ledger: 2547",
    "process.thread.name": "BookKeeperClientWorker-OrderedExecutor-19-0",
    "log.logger": "org.apache.bookkeeper.client.LedgerRecoveryOp"
}
{
    "log.level": "ERROR",
    "message": "BKException while fencing the ledger 2547 for rereplication of postponed ledgers",
    "process.thread.name": "PendingReplicationTimer",
    "log.logger": "org.apache.bookkeeper.replication.ReplicationWorker",
    "error.type": "org.apache.bookkeeper.client.BKException.BKLedgerRecoveryException",
    "error.message": "Error while recovering ledger",
    "error.stack_trace": "org.apache.bookkeeper.client.BKException$BKLedgerRecoveryException: Error while recovering ledger\n\tat org.apache.bookkeeper.client.SyncCallbackUtils.finish(SyncCallbackUtils.java:83)\n\tat org.apache.bookkeeper.client.SyncCallbackUtils$SyncOpenCallback.openComplete(SyncCallbackUtils.java:157)\n\tat org.apache.bookkeeper.client.LedgerOpenOp.openComplete(LedgerOpenOp.java:232)\n\tat org.apache.bookkeeper.client.LedgerOpenOp$1.safeOperationComplete(LedgerOpenOp.java:201)\n\tat org.apache.bookkeeper.client.LedgerOpenOp$1.safeOperationComplete(LedgerOpenOp.java:193)\n\tat org.apache.bookkeeper.util.OrderedGenericCallback.operationComplete(OrderedGenericCallback.java:62)\n\tat org.apache.bookkeeper.proto.BookkeeperInternalCallbacks$TimedGenericCallback.operationComplete(BookkeeperInternalCallbacks.java:189)\n\tat org.apache.bookkeeper.client.ReadOnlyLedgerHandle.lambda$recover$5(ReadOnlyLedgerHandle.java:295)\n\tat java.base/java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:859)\n\tat java.base/java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:837)\n\tat java.base/java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:506)\n\tat java.base/java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:2088)\n\tat org.apache.bookkeeper.client.LedgerRecoveryOp.submitCallback(LedgerRecoveryOp.java:142)\n\tat org.apache.bookkeeper.client.LedgerRecoveryOp.addComplete(LedgerRecoveryOp.java:223)\n\tat org.apache.bookkeeper.client.AsyncCallback$AddCallback.addCompleteWithLatency(AsyncCallback.java:92)\n\tat org.apache.bookkeeper.client.PendingAddOp.submitCallback(PendingAddOp.java:431)\n\tat org.apache.bookkeeper.client.LedgerHandle.errorOutPendingAdds(LedgerHandle.java:1799)\n\tat org.apache.bookkeeper.client.LedgerHandle.errorOutPendingAdds(LedgerHandle.java:1784)\n\tat org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleUnrecoverableErrorDuringAdd(ReadOnlyLedgerHandle.java:244)\n\tat org.apache.bookkeeper.client.ReadOnlyLedgerHandle.handleBookieFailure(ReadOnlyLedgerHandle.java:236)\n\tat org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:378)\n\tat org.apache.bookkeeper.proto.BookieClientImpl$1.safeRun(BookieClientImpl.java:295)\n\tat org.apache.bookkeeper.common.util.SafeRunnable.run(SafeRunnable.java:36)\n\tat java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)\n\tat java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)\n\tat io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)\n\tat java.base/java.lang.Thread.run(Thread.java:834)\n"
}

But when we disable parameter bookkeeperClientEnforceMinNumRacksPerWriteQuorum=false on brokers and enforceMinNumRacksPerWriteQuorum=false on bookies - leaving all other parameters exactly the same, then the new ensemble and quorums are selected successfully. We see in logs the following:

{
    "log.level": "WARN",
    "message": "Failed to find 1 bookies : excludeBookies [<Bookie:bk-perf05.msk.ru:3181>, <Bookie:bk-perf02.msk.ru:3181>, <Bookie:bk-perf07.msk.ru:3181>, <Bookie:bk-perf09.msk.ru:3181>, <Bookie:bk-perf03.msk.ru:3181>, <Bookie:bk-perf06.msk.ru:3181>, <Bookie:bk-perf04.msk.ru:3181>, <Bookie:bk-perf08.msk.ru:3181>, <Bookie:bk-perf01.msk.ru:3181>], allBookies [<Bookie:bk-perf04.msk.ru:3181>, <Bookie:bk-perf03.msk.ru:3181>, <Bookie:bk-perf01.msk.ru:3181>, <Bookie:bk-perf06.msk.ru:3181>, <Bookie:bk-perf02.msk.ru:3181>, <Bookie:bk-perf05.msk.ru:3181>].",
    "process.thread.name": "BookKeeperClientWorker-OrderedExecutor-6-0",
    "log.logger": "org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl"
}
{
    "log.level": "WARN",
    "message": "Failed to choose a bookie: excluded [<Bookie:bk-perf05.msk.ru:3181>, <Bookie:bk-perf07.msk.ru:3181>, <Bookie:bk-perf09.msk.ru:3181>, <Bookie:bk-perf03.msk.ru:3181>, <Bookie:bk-perf06.msk.ru:3181>, <Bookie:bk-perf04.msk.ru:3181>, <Bookie:bk-perf08.msk.ru:3181>], fallback to choose bookie randomly from the cluster.",
    "process.thread.name": "BookKeeperClientWorker-OrderedExecutor-6-0",
    "log.logger": "org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl"
}
{
    "log.level": "WARN",
    "message": "replaceBookie for bookie: bk-perf08.msk.ru:3181 in ensemble: [bk-perf03.msk.ru:3181, bk-perf04.msk.ru:3181, bk-perf05.msk.ru:3181, bk-perf06.msk.ru:3181, bk-perf08.msk.ru:3181] is not adhering to placement policy and chose bk-perf02.msk.ru:3181. excludedBookies [bk-perf08.msk.ru:3181] and quarantinedBookies [bk-perf08.msk.ru:3181, bk-perf09.msk.ru:3181, bk-perf07.msk.ru:3181]",
    "process.thread.name": "BookKeeperClientWorker-OrderedExecutor-6-0",
    "log.logger": "org.apache.bookkeeper.client.BookieWatcherImpl"
}

Our broker configuration:

brokerServicePort=6650
webServicePort=8080
bindAddress=0.0.0.0
journalDirectories=/var/lib/pulsar
ledgerDirectories=/var/lib/pulsar
indexDirectories=/var/lib/pulsar
clusterName=perf-rs01
configurationStoreServers=zk-perf01.msk.ru:2181,zk-perf02.msk.ru:2181,zk-perf03.msk.ru:2181
zookeeperServers=zk-perf01.msk.ru:2181,zk-perf02.msk.ru:2181,zk-perf03.msk.ru:2181
allowAutoTopicCreationType=partitioned
functionsWorkerEnabled=false
managedLedgerDefaultAckQuorum=3
managedLedgerDefaultEnsembleSize=3
managedLedgerDefaultWriteQuorum=3
defaultRetentionTimeInMinutes=5760
defaultRetentionSizeInMB=1025
defaultNumPartitions=3
brokerDeleteInactiveTopicsEnabled=false
systemTopicEnabled=false
topicLevelPoliciesEnabled=false
backlogQuotaCheckEnabled=false
backlogQuotaDefaultLimitGB=1
backlogQuotaDefaultRetentionPolicy=producer_exception
bookkeeperClientRackawarePolicyEnabled=true
bookkeeperClientRegionawarePolicyEnabled=false
bookkeeperClientMinNumRacksPerWriteQuorum=2
bookkeeperClientEnforceMinNumRacksPerWriteQuorum=true
topicFencingTimeoutSeconds=60
maxMessageSize=10485760
loadBalancerNamespaceMaximumBundles=512
loadBalancerLoadSheddingStrategy=org.apache.pulsar.broker.loadbalance.impl.ThresholdShedder
loadBalancerBrokerThresholdShedderPercentage=5
loadBalancerHistoryResourcePercentage=0.9
loadBalancerBandwithInResourceWeight=1.0
loadBalancerBandwithOutResourceWeight=1.0
loadBalancerCPUResourceWeight=1.0
loadBalancerMemoryResourceWeight=1.0
loadBalancerDirectMemoryResourceWeight=1.0
loadBalancerBundleUnloadMinThroughputThreshold=10
loadBalancerReportUpdateMaxIntervalMinutes=10
autoSkipNonRecoverableData=true
exposeTopicLevelMetricsInPrometheus=true
exposeConsumerLevelMetricsInPrometheus=true
bookkeeperClientExposeStatsToPrometheus=true

Our bookkeeper configuration:

bookiePort=3181
extraServerComponents=
httpServerEnabled=true
httpServerPort=8080
httpServerClass=org.apache.bookkeeper.http.vertx.VertxHttpServer
useHostNameAsBookieID=true
journalDirectories=/var/lib/bookkeeper/journal
ledgerDirectories=/var/lib/bookkeeper/data
indexDirectories=/var/lib/bookkeeper/index
metadataServiceUri=zk+hierarchical://zk-perf01.msk.ru:2181;zk-perf02.msk.ru:2181;zk-perf03.msk.ru:2181/ledgers
zkTimeout=5000
zkEnableSecurity=false
enableStatistics=true
statsProviderClass=org.apache.bookkeeper.stats.prometheus.PrometheusMetricsProvider
prometheusStatsHttpPort=8080
isForceGCAllowWhenNoSpace=true
verifyMetadataOnGC=true
ensemblePlacementPolicy=org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicy
reppDnsResolverClass=org.apache.pulsar.zookeeper.ZkBookieRackAffinityMapping
autoRecoveryDaemonEnabled=false
minNumRacksPerWriteQuorum=2
enforceMinNumRacksPerWriteQuorum=true
ignoreLocalNodeInPlacementPolicy=true
auditorPeriodicPlacementPolicyCheckInterval=86400
lostBookieRecoveryDelay=43200
zkServers=zk-perf01.msk.ru:2181,zk-perf02.msk.ru:2181,zk-perf03.msk.ru:2181
zkLedgersRootPath=/ledgers

As a result of this testing, I opened this issue to fix the behavior with bookkeeperClientEnforceMinNumRacksPerWriteQuorum and enforceMinNumRacksPerWriteQuorum.

To Reproduce My colleagues prepared a docker compose file to reproduce this behavior (i put it in my git):

  1. pulsar-enforce-rack-reproduce

Expected behavior bookkeeperClientEnforceMinNumRacksPerWriteQuorum and enforceMinNumRacksPerWriteQuorum - works as expected, the new ensemble and quorums are selected successfully and writing and reading to topics continues when shutdown bookkeepers belonging to the same rack.

Desktop:

Server where LXC containers were running:

github-actions[bot] commented 2 years ago

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