This is an automated email from the ASF dual-hosted git repository. sijie pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/pulsar.git
The following commit(s) were added to refs/heads/master by this push: new c8d0c68 [broker] Expose BookKeeper GetBookieInfo settings (#6800) c8d0c68 is described below commit c8d0c686881e2adcfe02948b29a8c0cc4bb10fa9 Author: Addison Higham <addis...@gmail.com> AuthorDate: Tue Apr 28 22:29:29 2020 -0600 [broker] Expose BookKeeper GetBookieInfo settings (#6800) Bookkeeper has settings to allow for periodic calls to refresh the bookie info. In dynamic environments (like when running on k8s) fetching the bookie info more often can be helpful to ensure the brokers stay up to date on the list of bookies even in the event of zookeeper watch issues. Co-authored-by: Addison Higham <ahig...@instructure.com> --- conf/broker.conf | 5 +++++ .../main/java/org/apache/pulsar/broker/ServiceConfiguration.java | 7 +++++++ .../java/org/apache/pulsar/broker/BookKeeperClientFactoryImpl.java | 2 ++ site2/docs/reference-configuration.md | 2 ++ 4 files changed, 16 insertions(+) diff --git a/conf/broker.conf b/conf/broker.conf index 8aebf10..ef1cb89 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -515,6 +515,11 @@ bookkeeperClientHealthCheckIntervalSeconds=60 bookkeeperClientHealthCheckErrorThresholdPerInterval=5 bookkeeperClientHealthCheckQuarantineTimeInSeconds=1800 +# Specify options for the GetBookieInfo check. These settings can be useful +# to help ensure the list of bookies is up to date on the brokers. +bookkeeperGetBookieInfoIntervalSeconds=86400 +bookkeeperGetBookieInfoRetryIntervalSeconds=60 + # Enable rack-aware bookie selection policy. BK will chose bookies from different racks when # forming a new bookie ensemble bookkeeperClientRackawarePolicyEnabled=true diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java index 030e838..fafeeb0 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java @@ -900,6 +900,13 @@ public class ServiceConfiguration implements PulsarConfiguration { doc = "Enable bookie secondary-isolation group if bookkeeperClientIsolationGroups doesn't have enough bookie available." ) private String bookkeeperClientSecondaryIsolationGroups; + + @FieldContext(category = CATEGORY_STORAGE_BK, doc = "Set the interval to periodically check bookie info") + private int bookkeeperClientGetBookieInfoIntervalSeconds = 60 * 60 * 24; // defaults to 24 hours + + @FieldContext(category = CATEGORY_STORAGE_BK, doc = "Set the interval to retry a failed bookie info lookup") + private int bookkeeperClientGetBookieInfoRetryIntervalSeconds = 60; + @FieldContext(category = CATEGORY_STORAGE_BK, doc = "Enable/disable having read operations for a ledger to be sticky to " + "a single bookie.\n" + "If this flag is enabled, the client will use one single bookie (by " + diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/BookKeeperClientFactoryImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/BookKeeperClientFactoryImpl.java index 095ecd0..37e8494 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/BookKeeperClientFactoryImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/BookKeeperClientFactoryImpl.java @@ -123,6 +123,8 @@ public class BookKeeperClientFactoryImpl implements BookKeeperClientFactory { bkConf.setReorderReadSequenceEnabled(conf.isBookkeeperClientReorderReadSequenceEnabled()); bkConf.setExplictLacInterval(conf.getBookkeeperExplicitLacIntervalInMills()); + bkConf.setGetBookieInfoIntervalSeconds(conf.getBookkeeperClientGetBookieInfoIntervalSeconds(), TimeUnit.SECONDS); + bkConf.setGetBookieInfoRetryIntervalSeconds(conf.getBookkeeperClientGetBookieInfoRetryIntervalSeconds(), TimeUnit.SECONDS); return bkConf; } diff --git a/site2/docs/reference-configuration.md b/site2/docs/reference-configuration.md index 6bc94ae..d0fe84f 100644 --- a/site2/docs/reference-configuration.md +++ b/site2/docs/reference-configuration.md @@ -186,6 +186,8 @@ Pulsar brokers are responsible for handling incoming messages from producers, di |bookkeeperClientIsolationGroups| Enable bookie isolation by specifying a list of bookie groups to choose from. Any bookie outside the specified groups will not be used by the broker || |bookkeeperClientSecondaryIsolationGroups| Enable bookie secondary-isolation group if bookkeeperClientIsolationGroups doesn't have enough bookie available. || |bookkeeperClientMinAvailableBookiesInIsolationGroups| Minimum bookies that should be available as part of bookkeeperClientIsolationGroups else broker will include bookkeeperClientSecondaryIsolationGroups bookies in isolated list. || +|bookkeeperClientGetBookieInfoIntervalSeconds| Set the interval to periodically check bookie info |86400| +|bookkeeperClientGetBookieInfoRetryIntervalSeconds| Set the interval to retry a failed bookie info lookup |60| |bookkeeperEnableStickyReads | Enable/disable having read operations for a ledger to be sticky to a single bookie. If this flag is enabled, the client will use one single bookie (by preference) to read all entries for a ledger. | true | |managedLedgerDefaultEnsembleSize| Number of bookies to use when creating a ledger |2| |managedLedgerDefaultWriteQuorum| Number of copies to store for each message |2|