pulsar-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From mme...@apache.org
Subject [pulsar] branch master updated: Add configuration to enabled BK sticky reads (#3569)
Date Tue, 12 Feb 2019 02:50:37 GMT
This is an automated email from the ASF dual-hosted git repository.

mmerli 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 37224b3  Add configuration to enabled BK sticky reads (#3569)
37224b3 is described below

commit 37224b332f62bc8dfd6de3272aa6a28e7462bf2c
Author: Matteo Merli <mmerli@apache.org>
AuthorDate: Mon Feb 11 18:50:30 2019 -0800

    Add configuration to enabled BK sticky reads (#3569)
---
 conf/broker.conf                                                     | 5 +++++
 .../src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java | 5 +++++
 .../java/org/apache/pulsar/broker/BookKeeperClientFactoryImpl.java   | 1 +
 .../main/java/org/apache/pulsar/sql/presto/PulsarConnectorCache.java | 1 +
 .../main/java/org/apache/pulsar/sql/presto/PulsarSplitManager.java   | 1 +
 site2/docs/reference-configuration.md                                | 1 +
 6 files changed, 14 insertions(+)

diff --git a/conf/broker.conf b/conf/broker.conf
index df2955b..a80f808 100644
--- a/conf/broker.conf
+++ b/conf/broker.conf
@@ -366,6 +366,11 @@ bookkeeperClientReorderReadSequenceEnabled=false
 # outside the specified groups will not be used by the broker
 bookkeeperClientIsolationGroups=
 
+# 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.
+bookkeeperEnableStickyReads=true
+
 ### --- Managed Ledger --- ###
 
 # Number of bookies to use when creating a ledger
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 6d34f33..24f3e32 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
@@ -623,6 +623,11 @@ public class ServiceConfiguration implements PulsarConfiguration {
         doc = "Enable bookie isolation by specifying a list of bookie groups to choose from.
\n\n"
             + "Any bookie outside the specified groups will not be used by the broker")
     private String bookkeeperClientIsolationGroups;
+    @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 " +
+            "preference) to read all entries for a ledger.")
+    private boolean bookkeeperEnableStickyReads = true;
 
     /**** --- Managed Ledger --- ****/
     @FieldContext(
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 2ed20ac..5e6e59d 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
@@ -56,6 +56,7 @@ public class BookKeeperClientFactoryImpl implements BookKeeperClientFactory
{
         bkConf.setNumChannelsPerBookie(16);
         bkConf.setUseV2WireProtocol(conf.isBookkeeperUseV2WireProtocol());
         bkConf.setEnableDigestTypeAutodetection(true);
+        bkConf.setStickyReadsEnabled(conf.isBookkeeperEnableStickyReads());
 
         bkConf.setAllocatorPoolingPolicy(PoolingPolicy.UnpooledHeap);
         if (conf.isBookkeeperClientHealthCheckEnabled()) {
diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarConnectorCache.java
b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarConnectorCache.java
index 0057d44..ee775bb 100644
--- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarConnectorCache.java
+++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarConnectorCache.java
@@ -61,6 +61,7 @@ public class PulsarConnectorCache {
                 .setShadedLedgerManagerFactoryClassPrefix("org.apache.pulsar.shade.")
                 .setClientTcpNoDelay(false)
                 .setUseV2WireProtocol(true)
+                .setStickyReadsEnabled(true)
                 .setReadEntryTimeout(60);
         return new ManagedLedgerFactoryImpl(bkClientConfiguration);
     }
diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSplitManager.java
b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSplitManager.java
index 800399e..a2962e5 100644
--- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSplitManager.java
+++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSplitManager.java
@@ -129,6 +129,7 @@ public class PulsarSplitManager implements ConnectorSplitManager {
                 .setAllowShadedLedgerManagerFactoryClass(true)
                 .setShadedLedgerManagerFactoryClassPrefix("org.apache.pulsar.shade.")
                 .setClientTcpNoDelay(false)
+                .setStickyReadsEnabled(true)
                 .setUseV2WireProtocol(true);
         return new ManagedLedgerFactoryImpl(bkClientConfiguration);
     }
diff --git a/site2/docs/reference-configuration.md b/site2/docs/reference-configuration.md
index 7e8b81b..753a8fb 100644
--- a/site2/docs/reference-configuration.md
+++ b/site2/docs/reference-configuration.md
@@ -168,6 +168,7 @@ Pulsar brokers are responsible for handling incoming messages from producers,
di
 |bookkeeperClientRegionawarePolicyEnabled|  Enable region-aware bookie selection policy.
BK will chose bookies from different regions and racks when forming a new bookie ensemble.
If enabled, the value of bookkeeperClientRackawarePolicyEnabled is ignored  |false|
 |bookkeeperClientReorderReadSequenceEnabled|  Enable/disable reordering read sequence on
reading entries.  |false|
 |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
 ||
+|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|
 |managedLedgerDefaultAckQuorum| Number of guaranteed copies (acks to wait before write is
complete) |2|


Mime
View raw message