From 91f7cb83c226ad9bb36e35018147d990e3103aa2 Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Wed, 25 Oct 2017 15:45:27 +0200 Subject: [PATCH 01/15] ISSUE #606: Enhancements in developer environment Improvements in maven configuration in order to speed up development Author: Enrico Olivelli Reviewers: Ivan Kelly This closes #661 from eolivelli/plugins-upgrade, closes #606 --- bookkeeper-server/pom.xml | 53 +++++++++++++++++++ .../src/test/resources/log4j.properties | 41 ++------------ pom.xml | 4 +- 3 files changed, 59 insertions(+), 39 deletions(-) diff --git a/bookkeeper-server/pom.xml b/bookkeeper-server/pom.xml index a7e70e9e8d8..2adc3eeabbc 100644 --- a/bookkeeper-server/pom.xml +++ b/bookkeeper-server/pom.xml @@ -378,6 +378,7 @@ 0.5.0 com.google.protobuf:protoc:3.4.0:exe:${os.detected.classifier} + true @@ -434,6 +435,58 @@ + + dev + + + + org.apache.maven.plugins + maven-checkstyle-plugin + + true + + + + org.apache.maven.plugins + maven-surefire-plugin + ${maven-surefire-plugin.version} + + -Xmx2G -Djava.net.preferIPv4Stack=true + false + false + 1800 + false + + + + + + + dev-debug + + + + org.apache.maven.plugins + maven-checkstyle-plugin + + true + + + + org.apache.maven.plugins + maven-surefire-plugin + ${maven-surefire-plugin.version} + + -Xmx2G -Djava.net.preferIPv4Stack=true -Dbookkeeper.root.logger=DEBUG,CONSOLE + false + false + 1800 + false + + + + + codahale-metrics-provider diff --git a/bookkeeper-server/src/test/resources/log4j.properties b/bookkeeper-server/src/test/resources/log4j.properties index 21d973a7b45..10ae6bfcbba 100644 --- a/bookkeeper-server/src/test/resources/log4j.properties +++ b/bookkeeper-server/src/test/resources/log4j.properties @@ -25,20 +25,14 @@ # Format is " (, )+ -# DEFAULT: console appender only -log4j.rootLogger=INFO, CONSOLE - -# Example with rolling log file -#log4j.rootLogger=DEBUG, CONSOLE, ROLLINGFILE - -# Example with rolling log file and tracing -#log4j.rootLogger=TRACE, CONSOLE, ROLLINGFILE, TRACEFILE +# DEFAULT: console appender only, level INFO +bookkeeper.root.logger=INFO,CONSOLE +log4j.rootLogger=${bookkeeper.root.logger} # # Log INFO level and above messages to the console # log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender -log4j.appender.CONSOLE.Threshold=INFO log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout log4j.appender.CONSOLE.layout.ConversionPattern=%d{ISO8601} - %-5p - [%t:%C{1}@%L] - %m%n @@ -46,32 +40,3 @@ log4j.appender.CONSOLE.layout.ConversionPattern=%d{ISO8601} - %-5p - [%t:%C{1}@% log4j.logger.org.apache.zookeeper=OFF log4j.logger.org.apache.bookkeeper.bookie=INFO log4j.logger.org.apache.bookkeeper.meta=INFO - -# -# Add ROLLINGFILE to rootLogger to get log file output -# Log DEBUG level and above messages to a log file -log4j.appender.ROLLINGFILE=org.apache.log4j.DailyRollingFileAppender -log4j.appender.ROLLINGFILE.Threshold=DEBUG -log4j.appender.ROLLINGFILE.File=target/bookkeeper-server.log -log4j.appender.ROLLINGFILE.layout=org.apache.log4j.PatternLayout -log4j.appender.ROLLINGFILE.layout.ConversionPattern=%d{ISO8601} - %-5p - [%t:%C{1}@%L] - %m%n - -# Max log file size of 10MB -log4j.appender.ROLLINGFILE.MaxFileSize=10MB -# uncomment the next line to limit number of backup files -#log4j.appender.ROLLINGFILE.MaxBackupIndex=10 - -log4j.appender.ROLLINGFILE.layout=org.apache.log4j.PatternLayout -log4j.appender.ROLLINGFILE.layout.ConversionPattern=%d{ISO8601} - %-5p [%t:%C{1}@%L] - %m%n - - -# -# Add TRACEFILE to rootLogger to get log file output -# Log DEBUG level and above messages to a log file -log4j.appender.TRACEFILE=org.apache.log4j.FileAppender -log4j.appender.TRACEFILE.Threshold=TRACE -log4j.appender.TRACEFILE.File=bookkeeper_trace.log - -log4j.appender.TRACEFILE.layout=org.apache.log4j.PatternLayout -### Notice we are including log4j's NDC here (%x) -log4j.appender.TRACEFILE.layout.ConversionPattern=%d{ISO8601} - %-5p [%t:%C{1}@%L][%x] - %m%n diff --git a/pom.xml b/pom.xml index 7a4dddca78a..2e1cc9569d6 100644 --- a/pom.xml +++ b/pom.xml @@ -117,7 +117,7 @@ 3.0.5 6.19 2.17 - 3.5.1 + 3.7.0 2.7 2.2 2.10.4 @@ -206,6 +206,8 @@ -Werror -Xlint:deprecation -Xlint:unchecked + + -Xpkginfo:always From 6fc34a5bc14f4542fb8b627e8d50a89068ceb7f8 Mon Sep 17 00:00:00 2001 From: Aaron Gresch Date: Wed, 25 Oct 2017 15:59:40 +0200 Subject: [PATCH 02/15] Fail earlier when encountering DNS issues We had some DNS issues today and various TestRegionAwareEnsemblePlacementPolicy tests were failing for me. The LOG warning in RackawareEnsemblePlacementPolicyIml was not printing the right variable, which would have given a clue. Additionally, the repp.myRegion was set to UNKNOWN, causing incorrect code flow. Added an assert to kill the test as soon as this occurs. Author: Aaron Gresch Reviewers: Ivan Kelly , Enrico Olivelli , Sijie Guo This closes #658 from agresch/agresch_dns_failure --- .../client/RackawareEnsemblePlacementPolicyImpl.java | 2 +- .../TestRegionAwareEnsemblePlacementPolicy.java | 11 +++++++---- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RackawareEnsemblePlacementPolicyImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RackawareEnsemblePlacementPolicyImpl.java index a7a59db46a7..527f180ca2c 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RackawareEnsemblePlacementPolicyImpl.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RackawareEnsemblePlacementPolicyImpl.java @@ -131,7 +131,7 @@ public List resolve(List names) { for (int i = 0; i < rNames.size(); ++i) { if (rNames.get(i) == null) { LOG.warn("Failed to resolve network location for {}, using default rack for it : {}.", - rNames.get(i), defaultRack); + names.get(i), defaultRack); rNames.set(i, defaultRack); } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRegionAwareEnsemblePlacementPolicy.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRegionAwareEnsemblePlacementPolicy.java index 538782470b0..da11e140dd9 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRegionAwareEnsemblePlacementPolicy.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRegionAwareEnsemblePlacementPolicy.java @@ -129,6 +129,9 @@ public void testNodeInSameRegion() throws Exception { repp = new RegionAwareEnsemblePlacementPolicy(); repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); + // make sure we've detected the right region + assertEquals("r1", repp.myRegion); + Set addrs = new HashSet(); addrs.add(addr1); addrs.add(addr2); @@ -136,15 +139,15 @@ public void testNodeInSameRegion() throws Exception { addrs.add(addr4); repp.onClusterChanged(addrs, new HashSet()); - List reoderSet = repp.reorderReadSequence(ensemble, writeSet, new HashMap()); + List reorderSet = repp.reorderReadSequence(ensemble, writeSet, new HashMap()); List expectedSet = new ArrayList(); expectedSet.add(0); expectedSet.add(3); expectedSet.add(1); expectedSet.add(2); - LOG.info("reorder set : {}", reoderSet); - assertFalse(reoderSet == writeSet); - assertEquals(expectedSet, reoderSet); + LOG.info("reorder set : {}", reorderSet); + assertFalse(reorderSet == writeSet); + assertEquals(expectedSet, reorderSet); } @Test From 988a7b63d98aa58c6973bdfafdf9a12bba9f44f3 Mon Sep 17 00:00:00 2001 From: Aaron Gresch Date: Fri, 27 Oct 2017 11:51:32 -0700 Subject: [PATCH 03/15] Recycle instances of WriteEntryProcessor and ReadEntryProcessor merge of some changes from Yahoo's repo Author: Aaron Gresch Reviewers: Sijie Guo , Matteo Merli This closes #669 from agresch/agresch_recycle_entryprocessor --- .../proto/BookieRequestProcessor.java | 4 +-- .../bookkeeper/proto/PacketProcessorBase.java | 17 +++++++--- .../bookkeeper/proto/ReadEntryProcessor.java | 27 +++++++++++++-- .../bookkeeper/proto/WriteEntryProcessor.java | 33 +++++++++++++++++-- 4 files changed, 70 insertions(+), 11 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieRequestProcessor.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieRequestProcessor.java index 8d719e6d0dd..81208f3d187 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieRequestProcessor.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieRequestProcessor.java @@ -374,7 +374,7 @@ private void processGetBookieInfoRequestV3(final BookkeeperProtocol.Request r, f } private void processAddRequest(final BookieProtocol.Request r, final Channel c) { - WriteEntryProcessor write = new WriteEntryProcessor(r, c, this); + WriteEntryProcessor write = WriteEntryProcessor.create(r, c, this); if (null == writeThreadPool) { write.run(); } else { @@ -383,7 +383,7 @@ private void processAddRequest(final BookieProtocol.Request r, final Channel c) } private void processReadRequest(final BookieProtocol.Request r, final Channel c) { - ReadEntryProcessor read = new ReadEntryProcessor(r, c, this); + ReadEntryProcessor read = ReadEntryProcessor.create(r, c, this); if (null == readThreadPool) { read.run(); } else { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PacketProcessorBase.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PacketProcessorBase.java index 4f14dcf78c7..5c9d8d26756 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PacketProcessorBase.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PacketProcessorBase.java @@ -30,18 +30,25 @@ abstract class PacketProcessorBase extends SafeRunnable { private final static Logger logger = LoggerFactory.getLogger(PacketProcessorBase.class); - final Request request; - final Channel channel; - final BookieRequestProcessor requestProcessor; - final long enqueueNanos; + Request request; + Channel channel; + BookieRequestProcessor requestProcessor; + long enqueueNanos; - PacketProcessorBase(Request request, Channel channel, BookieRequestProcessor requestProcessor) { + protected void init(Request request, Channel channel, BookieRequestProcessor requestProcessor) { this.request = request; this.channel = channel; this.requestProcessor = requestProcessor; this.enqueueNanos = MathUtils.nowInNano(); } + protected void reset() { + request = null; + channel = null; + requestProcessor = null; + enqueueNanos = -1; + } + protected boolean isVersionCompatible() { byte version = request.getProtocolVersion(); if (version < BookieProtocol.LOWEST_COMPAT_PROTOCOL_VERSION diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadEntryProcessor.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadEntryProcessor.java index f65455cde0b..f651e13dcfa 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadEntryProcessor.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadEntryProcessor.java @@ -19,7 +19,9 @@ import io.netty.buffer.ByteBuf; import io.netty.channel.Channel; +import io.netty.util.Recycler; import io.netty.util.ReferenceCountUtil; +import io.netty.util.Recycler.Handle; import java.io.IOException; import java.util.concurrent.ExecutionException; @@ -37,9 +39,11 @@ class ReadEntryProcessor extends PacketProcessorBase { private final static Logger LOG = LoggerFactory.getLogger(ReadEntryProcessor.class); - public ReadEntryProcessor(Request request, Channel channel, + public static ReadEntryProcessor create(Request request, Channel channel, BookieRequestProcessor requestProcessor) { - super(request, channel, requestProcessor); + ReadEntryProcessor rep = RECYCLER.get(); + rep.init(request, channel, requestProcessor); + return rep; } @Override @@ -144,10 +148,29 @@ protected void processPacket() { sendResponse(errorCode, ResponseBuilder.buildErrorResponse(errorCode, read), requestProcessor.readRequestStats); } + recycle(); } @Override public String toString() { return String.format("ReadEntry(%d, %d)", request.getLedgerId(), request.getEntryId()); } + + private void recycle() { + super.reset(); + this.recyclerHandle.recycle(this); + } + + private final Recycler.Handle recyclerHandle; + + private ReadEntryProcessor(Recycler.Handle recyclerHandle) { + this.recyclerHandle = recyclerHandle; + } + + private static final Recycler RECYCLER = new Recycler() { + @Override + protected ReadEntryProcessor newObject(Recycler.Handle handle) { + return new ReadEntryProcessor(handle); + } + }; } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteEntryProcessor.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteEntryProcessor.java index 827aed9865a..c4b28406b02 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteEntryProcessor.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteEntryProcessor.java @@ -18,6 +18,8 @@ package org.apache.bookkeeper.proto; import io.netty.channel.Channel; +import io.netty.util.Recycler; +import io.netty.util.Recycler.Handle; import java.io.IOException; import java.util.concurrent.TimeUnit; @@ -39,9 +41,16 @@ class WriteEntryProcessor extends PacketProcessorBase implements WriteCallback { long startTimeNanos; - public WriteEntryProcessor(Request request, Channel channel, + protected void reset() { + super.reset(); + startTimeNanos = -1L; + } + + public static WriteEntryProcessor create(Request request, Channel channel, BookieRequestProcessor requestProcessor) { - super(request, channel, requestProcessor); + WriteEntryProcessor wep = RECYCLER.get(); + wep.init(request, channel, requestProcessor); + return wep; } @Override @@ -55,6 +64,7 @@ protected void processPacket() { sendResponse(BookieProtocol.EREADONLY, ResponseBuilder.buildErrorResponse(BookieProtocol.EREADONLY, add), requestProcessor.addRequestStats); + add.release(); return; } @@ -101,6 +111,7 @@ public void writeComplete(int rc, long ledgerId, long entryId, sendResponse(rc, ResponseBuilder.buildAddResponse(request), requestProcessor.addRequestStats); + recycle(); } @Override @@ -108,4 +119,22 @@ public String toString() { return String.format("WriteEntry(%d, %d)", request.getLedgerId(), request.getEntryId()); } + + private void recycle() { + reset(); + recyclerHandle.recycle(this); + } + + private final Recycler.Handle recyclerHandle; + + private WriteEntryProcessor(Recycler.Handle recyclerHandle) { + this.recyclerHandle = recyclerHandle; + } + + private static final Recycler RECYCLER = new Recycler() { + @Override + protected WriteEntryProcessor newObject(Recycler.Handle handle) { + return new WriteEntryProcessor(handle); + } + }; } From 621844e4343d8cef37af437be226e1e351539a99 Mon Sep 17 00:00:00 2001 From: Jia Zhai Date: Fri, 27 Oct 2017 11:58:19 -0700 Subject: [PATCH 04/15] ISSUE #662: Introduce Bookie Registration Manager for bookie server Descriptions of the changes in this PR: While in BOOKKEEPER-628, It is trying to improve/generalize the bookie registration process. This PR is for bookie side, it tries to introduce Bookie registration manager for bookie server. Author: Jia Zhai Reviewers: Enrico Olivelli , Sijie Guo This closes #663 from zhaijack/bookie_registration, closes #662 --- .../org/apache/bookkeeper/bookie/Bookie.java | 354 +++++----------- .../bookkeeper/bookie/BookieException.java | 86 +++- .../apache/bookkeeper/bookie/BookieShell.java | 102 ++--- .../org/apache/bookkeeper/bookie/Cookie.java | 177 +++----- .../bookkeeper/bookie/FileSystemUpgrade.java | 53 +-- .../bookkeeper/conf/ServerConfiguration.java | 28 ++ .../discover/RegistrationClient.java | 36 ++ .../discover/RegistrationManager.java | 108 +++++ .../discover/ZKRegistrationManager.java | 391 ++++++++++++++++++ .../bookkeeper/discover/package-info.java | 22 + .../bookkeeper/http/ExpandStorageService.java | 9 +- .../http/RecoveryBookieService.java | 12 +- .../bookie/BookieInitializationTest.java | 79 +++- .../apache/bookkeeper/bookie/CookieTest.java | 36 +- .../bookie/UpdateCookieCmdTest.java | 39 +- 15 files changed, 1040 insertions(+), 492 deletions(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/RegistrationClient.java create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/RegistrationManager.java create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/ZKRegistrationManager.java create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/package-info.java diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Bookie.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Bookie.java index 220aa4cb0ab..953fb17d301 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Bookie.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Bookie.java @@ -21,13 +21,11 @@ package org.apache.bookkeeper.bookie; -import static com.google.common.base.Charsets.UTF_8; import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIE_ADD_ENTRY; import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIE_ADD_ENTRY_BYTES; import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIE_READ_ENTRY; import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIE_READ_ENTRY_BYTES; import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIE_RECOVERY_ADD_ENTRY; -import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIE_SCOPE; import static org.apache.bookkeeper.bookie.BookKeeperServerStats.JOURNAL_SCOPE; import static org.apache.bookkeeper.bookie.BookKeeperServerStats.LD_INDEX_SCOPE; import static org.apache.bookkeeper.bookie.BookKeeperServerStats.LD_LEDGER_SCOPE; @@ -54,25 +52,28 @@ import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Observable; import java.util.Observer; import java.util.Set; import java.util.concurrent.Callable; -import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.Collectors; +import org.apache.bookkeeper.bookie.BookieException.BookieIllegalOpException; +import org.apache.bookkeeper.bookie.BookieException.CookieNotFoundException; import org.apache.bookkeeper.bookie.BookieException.DiskPartitionDuplicationException; +import org.apache.bookkeeper.bookie.BookieException.MetadataStoreException; import org.apache.bookkeeper.bookie.Journal.JournalScanner; import org.apache.bookkeeper.bookie.LedgerDirsManager.LedgerDirsListener; import org.apache.bookkeeper.bookie.LedgerDirsManager.NoWritableLedgerDirException; import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.discover.RegistrationManager; +import org.apache.bookkeeper.discover.ZKRegistrationManager; import org.apache.bookkeeper.meta.LedgerManager; import org.apache.bookkeeper.meta.LedgerManagerFactory; import org.apache.bookkeeper.net.BookieSocketAddress; @@ -87,24 +88,15 @@ import org.apache.bookkeeper.util.DiskChecker; import org.apache.bookkeeper.util.IOUtils; import org.apache.bookkeeper.util.MathUtils; -import org.apache.bookkeeper.util.ZkUtils; +import org.apache.bookkeeper.util.ReflectionUtils; import org.apache.bookkeeper.util.collections.ConcurrentLongHashMap; import org.apache.bookkeeper.versioning.Version; import org.apache.bookkeeper.versioning.Versioned; -import org.apache.bookkeeper.zookeeper.BoundExponentialBackoffRetryPolicy; -import org.apache.bookkeeper.zookeeper.ZooKeeperClient; +import org.apache.commons.configuration.ConfigurationException; import org.apache.commons.io.FileUtils; import org.apache.commons.lang3.mutable.MutableBoolean; -import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; -import org.apache.zookeeper.KeeperException.NodeExistsException; -import org.apache.zookeeper.WatchedEvent; -import org.apache.zookeeper.Watcher; -import org.apache.zookeeper.Watcher.Event.EventType; -import org.apache.zookeeper.Watcher.Event.KeeperState; import org.apache.zookeeper.ZooKeeper; -import org.apache.zookeeper.data.ACL; -import org.apache.zookeeper.data.Stat; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -129,18 +121,14 @@ public class Bookie extends BookieCriticalThread { static final long METAENTRY_ID_LEDGER_KEY = -0x1000; static final long METAENTRY_ID_FENCE_KEY = -0x2000; - // ZK registration path for this bookie - protected final String bookieRegistrationPath; - protected final String bookieReadonlyRegistrationPath; - private final LedgerDirsManager ledgerDirsManager; private LedgerDirsManager indexDirsManager; LedgerDirsMonitor ledgerMonitor; LedgerDirsMonitor idxMonitor; - // ZooKeeper client instance for the Bookie - ZooKeeper zk; + // Registration Manager for managing registration + RegistrationManager registrationManager; // Running flag private volatile boolean running = false; @@ -151,11 +139,9 @@ public class Bookie extends BookieCriticalThread { private final ConcurrentLongHashMap masterKeyCache = new ConcurrentLongHashMap<>(); - protected final String zkBookieRegPath; - protected final String zkBookieReadOnlyPath; - protected final List zkAcls; + protected final String bookieId; - private final AtomicBoolean zkRegistered = new AtomicBoolean(false); + private final AtomicBoolean rmRegistered = new AtomicBoolean(false); protected final AtomicBoolean readOnly = new AtomicBoolean(false); // executor to manage the state changes for a bookie. final ExecutorService stateService = Executors.newSingleThreadExecutor( @@ -254,19 +240,29 @@ public boolean accept(File dir, String name) { } } + @VisibleForTesting + public void setRegistrationManager(RegistrationManager rm) { + this.registrationManager = rm; + } + + @VisibleForTesting + public RegistrationManager getRegistrationManager() { + return this.registrationManager; + } + /** * Check that the environment for the bookie is correct. * This means that the configuration has stayed the same as the * first run and the filesystem structure is up to date. */ - private void checkEnvironment(ZooKeeper zk) throws BookieException, IOException { + private void checkEnvironment(RegistrationManager rm) throws BookieException, IOException { List allLedgerDirs = new ArrayList(ledgerDirsManager.getAllLedgerDirs().size() + indexDirsManager.getAllLedgerDirs().size()); allLedgerDirs.addAll(ledgerDirsManager.getAllLedgerDirs()); if (indexDirsManager != ledgerDirsManager) { allLedgerDirs.addAll(indexDirsManager.getAllLedgerDirs()); } - if (zk == null) { // exists only for testing, just make sure directories are correct + if (rm == null) { // exists only for testing, just make sure directories are correct for (File journalDirectory : journalDirectories) { checkDirectoryStructure(journalDirectory); @@ -279,7 +275,7 @@ private void checkEnvironment(ZooKeeper zk) throws BookieException, IOException } if (conf.getAllowStorageExpansion()) { - checkEnvironmentWithStorageExpansion(conf, zk, journalDirectories, allLedgerDirs); + checkEnvironmentWithStorageExpansion(conf, rm, journalDirectories, allLedgerDirs); return; } @@ -303,20 +299,20 @@ private void checkEnvironment(ZooKeeper zk) throws BookieException, IOException } } - String instanceId = getInstanceId(conf, zk); + String instanceId = rm.getClusterInstanceId(); Cookie.Builder builder = Cookie.generateCookie(conf); if (null != instanceId) { builder.setInstanceId(instanceId); } Cookie masterCookie = builder.build(); - Versioned zkCookie = null; + Versioned rmCookie = null; try { - zkCookie = Cookie.readFromZooKeeper(zk, conf); + rmCookie = Cookie.readFromRegistrationManager(rm, conf); // If allowStorageExpansion option is set, we should // make sure that the new set of ledger/index dirs // is a super set of the old; else, we fail the cookie check - masterCookie.verifyIsSuperSet(zkCookie.getValue()); - } catch (KeeperException.NoNodeException nne) { + masterCookie.verifyIsSuperSet(rmCookie.getValue()); + } catch (CookieNotFoundException e) { // can occur in cases: // 1) new environment or // 2) done only metadata format and started bookie server. @@ -381,7 +377,7 @@ private void checkEnvironment(ZooKeeper zk) throws BookieException, IOException for (File dir : allLedgerDirs) { masterCookie.writeToDirectory(dir); } - masterCookie.writeToZooKeeper(zk, conf, zkCookie != null ? zkCookie.getVersion() : Version.NEW); + masterCookie.writeToRegistrationManager(rm, conf, rmCookie != null ? rmCookie.getVersion() : Version.NEW); } List ledgerDirs = ledgerDirsManager.getAllLedgerDirs(); @@ -389,18 +385,10 @@ private void checkEnvironment(ZooKeeper zk) throws BookieException, IOException List indexDirs = indexDirsManager.getAllLedgerDirs(); checkIfDirsOnSameDiskPartition(indexDirs); checkIfDirsOnSameDiskPartition(journalDirectories); - } catch (KeeperException ke) { - LOG.error("Couldn't access cookie in zookeeper", ke); - throw new BookieException.InvalidCookieException(ke); - } catch (UnknownHostException uhe) { - LOG.error("Couldn't check cookies, networking is broken", uhe); - throw new BookieException.InvalidCookieException(uhe); + } catch (IOException ioe) { LOG.error("Error accessing cookie on disks", ioe); throw new BookieException.InvalidCookieException(ioe); - } catch (InterruptedException ie) { - LOG.error("Thread interrupted while checking cookies, exiting", ie); - throw new BookieException.InvalidCookieException(ie); } } @@ -450,8 +438,11 @@ private void checkIfDirsOnSameDiskPartition(List dirs) throws DiskPartitio } } - public static void checkEnvironmentWithStorageExpansion(ServerConfiguration conf, - ZooKeeper zk, List journalDirectories, List allLedgerDirs) throws BookieException, IOException { + public static void checkEnvironmentWithStorageExpansion( + ServerConfiguration conf, + RegistrationManager rm, + List journalDirectories, + List allLedgerDirs) throws BookieException { try { boolean newEnv = false; List missedCookieDirs = new ArrayList(); @@ -472,20 +463,20 @@ public static void checkEnvironmentWithStorageExpansion(ServerConfiguration conf } } - String instanceId = getInstanceId(conf, zk); + String instanceId = rm.getClusterInstanceId(); Cookie.Builder builder = Cookie.generateCookie(conf); if (null != instanceId) { builder.setInstanceId(instanceId); } Cookie masterCookie = builder.build(); - Versioned zkCookie = null; + Versioned rmCookie = null; try { - zkCookie = Cookie.readFromZooKeeper(zk, conf); + rmCookie = Cookie.readFromRegistrationManager(rm, conf); // If allowStorageExpansion option is set, we should // make sure that the new set of ledger/index dirs // is a super set of the old; else, we fail the cookie check - masterCookie.verifyIsSuperSet(zkCookie.getValue()); - } catch (KeeperException.NoNodeException nne) { + masterCookie.verifyIsSuperSet(rmCookie.getValue()); + } catch (CookieNotFoundException e) { // can occur in cases: // 1) new environment or // 2) done only metadata format and started bookie server. @@ -550,20 +541,11 @@ public static void checkEnvironmentWithStorageExpansion(ServerConfiguration conf for (File dir : allLedgerDirs) { masterCookie.writeToDirectory(dir); } - masterCookie.writeToZooKeeper(zk, conf, zkCookie != null ? zkCookie.getVersion() : Version.NEW); + masterCookie.writeToRegistrationManager(rm, conf, rmCookie != null ? rmCookie.getVersion() : Version.NEW); } - } catch (KeeperException ke) { - LOG.error("Couldn't access cookie in zookeeper", ke); - throw new BookieException.InvalidCookieException(ke); - } catch (UnknownHostException uhe) { - LOG.error("Couldn't check cookies, networking is broken", uhe); - throw new BookieException.InvalidCookieException(uhe); } catch (IOException ioe) { LOG.error("Error accessing cookie on disks", ioe); throw new BookieException.InvalidCookieException(ioe); - } catch (InterruptedException ie) { - LOG.error("Thread interrupted while checking cookies, exiting", ie); - throw new BookieException.InvalidCookieException(ie); } } @@ -605,25 +587,6 @@ public static BookieSocketAddress getBookieAddress(ServerConfiguration conf) return addr; } - private static String getInstanceId(ServerConfiguration conf, ZooKeeper zk) throws KeeperException, - InterruptedException { - String instanceId = null; - if (zk.exists(conf.getZkLedgersRootPath(), null) == null) { - LOG.error("BookKeeper metadata doesn't exist in zookeeper. " - + "Has the cluster been initialized? " - + "Try running bin/bookkeeper shell metaformat"); - throw new KeeperException.NoNodeException("BookKeeper metadata"); - } - try { - byte[] data = zk.getData(conf.getZkLedgersRootPath() + "/" - + BookKeeperConstants.INSTANCEID, false, null); - instanceId = new String(data, UTF_8); - } catch (KeeperException.NoNodeException e) { - LOG.info("INSTANCEID not exists in zookeeper. Not considering it for data verification"); - } - return instanceId; - } - public LedgerDirsManager getLedgerDirsManager() { return ledgerDirsManager; } @@ -653,18 +616,14 @@ public static File[] getCurrentDirectories(File[] dirs) { } public Bookie(ServerConfiguration conf) - throws IOException, KeeperException, InterruptedException, BookieException { + throws IOException, InterruptedException, BookieException { this(conf, NullStatsLogger.INSTANCE); } public Bookie(ServerConfiguration conf, StatsLogger statsLogger) - throws IOException, KeeperException, InterruptedException, BookieException { + throws IOException, InterruptedException, BookieException { super("Bookie-" + conf.getBookiePort()); this.statsLogger = statsLogger; - this.zkAcls = ZkUtils.getACLs(conf); - this.bookieRegistrationPath = conf.getZkAvailableBookiesPath() + "/"; - this.bookieReadonlyRegistrationPath = - this.bookieRegistrationPath + BookKeeperConstants.READONLY; this.conf = conf; this.journalDirectories = Lists.newArrayList(); for (File journalDirectory : conf.getJournalDirs()) { @@ -683,9 +642,20 @@ public Bookie(ServerConfiguration conf, StatsLogger statsLogger) } // instantiate zookeeper client to initialize ledger manager - this.zk = instantiateZookeeperClient(conf); - checkEnvironment(this.zk); - ledgerManagerFactory = LedgerManagerFactory.newLedgerManagerFactory(conf, this.zk); + this.registrationManager = instantiateRegistrationManager(conf); + checkEnvironment(this.registrationManager); + try { + ZooKeeper zooKeeper = null; // ZooKeeper is null existing only for testing + if (registrationManager != null) { + zooKeeper = ((ZKRegistrationManager) this.registrationManager).getZk(); + } + // current the registration manager is zookeeper only + ledgerManagerFactory = LedgerManagerFactory.newLedgerManagerFactory( + conf, + zooKeeper); + } catch (KeeperException e) { + throw new MetadataStoreException("Failed to initialize ledger manager", e); + } LOG.info("instantiate ledger manager {}", ledgerManagerFactory.getClass().getName()); ledgerManager = ledgerManagerFactory.newLedgerManager(); @@ -721,9 +691,7 @@ public Bookie(ServerConfiguration conf, StatsLogger statsLogger) } // ZK ephemeral node for this Bookie. - String myID = getMyId(); - zkBookieRegPath = this.bookieRegistrationPath + myID; - zkBookieReadOnlyPath = this.bookieReadonlyRegistrationPath + "/" + myID; + this.bookieId = getMyId(); // instantiate the journals journals = Lists.newArrayList(); @@ -760,7 +728,7 @@ public Number getDefaultValue() { @Override public Number getSample() { - return zkRegistered.get() ? (readOnly.get() ? 0 : 1) : -1; + return rmRegistered.get() ? (readOnly.get() ? 0 : 1) : -1; } }); } @@ -932,65 +900,23 @@ public void diskJustWritable(File disk) { } /** - * Instantiate the ZooKeeper client for the Bookie. + * Instantiate the registration manager for the Bookie. */ - private ZooKeeper instantiateZookeeperClient(ServerConfiguration conf) - throws IOException, InterruptedException, KeeperException { - if (conf.getZkServers() == null) { - LOG.warn("No ZK servers passed to Bookie constructor so BookKeeper clients won't know about this server!"); - return null; - } - // Create the ZooKeeper client instance - return newZookeeper(conf); - } - - /** - * Check existence of regPath and wait it expired if possible. - * - * @param regPath reg node path. - * @return true if regPath exists, otherwise return false - * @throws IOException if can't create reg path - */ - protected boolean checkRegNodeAndWaitExpired(String regPath) throws IOException { - final CountDownLatch prevNodeLatch = new CountDownLatch(1); - Watcher zkPrevRegNodewatcher = new Watcher() { - @Override - public void process(WatchedEvent event) { - // Check for prev znode deletion. Connection expiration is - // not handling, since bookie has logic to shutdown. - if (EventType.NodeDeleted == event.getType()) { - prevNodeLatch.countDown(); - } - } - }; + private RegistrationManager instantiateRegistrationManager(ServerConfiguration conf) throws BookieException { + // Create the registration manager instance + Class managerCls; try { - Stat stat = zk.exists(regPath, zkPrevRegNodewatcher); - if (null != stat) { - // if the ephemeral owner isn't current zookeeper client - // wait for it to be expired. - if (stat.getEphemeralOwner() != zk.getSessionId()) { - LOG.info("Previous bookie registration znode: {} exists, so waiting zk sessiontimeout:" - + " {} ms for znode deletion", regPath, conf.getZkTimeout()); - // waiting for the previous bookie reg znode deletion - if (!prevNodeLatch.await(conf.getZkTimeout(), TimeUnit.MILLISECONDS)) { - throw new NodeExistsException(regPath); - } else { - return false; - } - } - return true; - } else { - return false; - } - } catch (KeeperException ke) { - LOG.error("ZK exception checking and wait ephemeral znode {} expired : ", regPath, ke); - throw new IOException("ZK exception checking and wait ephemeral znode " - + regPath + " expired", ke); - } catch (InterruptedException ie) { - LOG.error("Interrupted checking and wait ephemeral znode {} expired : ", regPath, ie); - throw new IOException("Interrupted checking and wait ephemeral znode " - + regPath + " expired", ie); + managerCls = conf.getRegistrationManagerClass(); + } catch (ConfigurationException e) { + throw new BookieIllegalOpException(e); } + + RegistrationManager manager = ReflectionUtils.newInstance(managerCls); + return manager.initialize(conf, () -> { + rmRegistered.set(false); + // schedule a re-register operation + registerBookie(false); + }, statsLogger); } /** @@ -1016,40 +942,28 @@ public Void call() throws IOException { } protected void doRegisterBookie() throws IOException { - doRegisterBookie(readOnly.get() ? zkBookieReadOnlyPath : zkBookieRegPath); + doRegisterBookie(readOnly.get()); } - private void doRegisterBookie(final String regPath) throws IOException { - if (null == zk) { - // zookeeper instance is null, means not register itself to zk + private void doRegisterBookie(boolean isReadOnly) throws IOException { + if (null == registrationManager || + ((ZKRegistrationManager) this.registrationManager).getZk() == null) { + // registration manager is null, means not register itself to zk. + // ZooKeeper is null existing only for testing. + LOG.info("null zk while do register"); return; } - zkRegistered.set(false); - - // ZK ephemeral node for this Bookie. + rmRegistered.set(false); try { - if (!checkRegNodeAndWaitExpired(regPath)) { - // Create the ZK ephemeral node for this Bookie. - zk.create(regPath, new byte[0], zkAcls, CreateMode.EPHEMERAL); - LOG.info("Registered myself in ZooKeeper at {}.", regPath); - } - zkRegistered.set(true); - } catch (KeeperException ke) { - LOG.error("ZK exception registering ephemeral Znode for Bookie!", ke); - // Throw an IOException back up. This will cause the Bookie - // constructor to error out. Alternatively, we could do a System - // exit here as this is a fatal error. - throw new IOException(ke); - } catch (InterruptedException ie) { - LOG.error("Interrupted exception registering ephemeral Znode for Bookie!", ie); - // Throw an IOException back up. This will cause the Bookie - // constructor to error out. Alternatively, we could do a System - // exit here as this is a fatal error. - throw new IOException(ie); + registrationManager.registerBookie(bookieId, isReadOnly); + rmRegistered.set(true); + } catch (BookieException e) { + throw new IOException(e); } } + /** * Transition the bookie from readOnly mode to writable. */ @@ -1073,11 +987,11 @@ public void doTransitionToWritableMode() { } LOG.info("Transitioning Bookie to Writable mode and will serve read/write requests."); // change zookeeper state only when using zookeeper - if (null == zk) { + if (null == registrationManager) { return; } try { - doRegisterBookie(zkBookieRegPath); + doRegisterBookie(false); } catch (IOException e) { LOG.warn("Error in transitioning back to writable mode : ", e); transitionToReadOnlyMode(); @@ -1085,12 +999,8 @@ public void doTransitionToWritableMode() { } // clear the readonly state try { - zk.delete(zkBookieReadOnlyPath, -1); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - LOG.warn("Interrupted clearing readonly state while transitioning to writable mode : ", e); - return; - } catch (KeeperException e) { + registrationManager.unregisterBookie(bookieId, true); + } catch (BookieException e) { // if we failed when deleting the readonly flag in zookeeper, it is OK since client would // already see the bookie in writable list. so just log the exception LOG.warn("Failed to delete bookie readonly state in zookeeper : ", e); @@ -1130,40 +1040,16 @@ public void doTransitionToReadOnlyMode() { LOG.info("Transitioning Bookie to ReadOnly mode," + " and will serve only read requests from clients!"); // change zookeeper state only when using zookeeper - if (null == zk) { + if (null == registrationManager) { return; } try { - if (null == zk.exists(this.bookieReadonlyRegistrationPath, false)) { - try { - zk.create(this.bookieReadonlyRegistrationPath, new byte[0], - zkAcls, CreateMode.PERSISTENT); - } catch (NodeExistsException e) { - // this node is just now created by someone. - } - } - doRegisterBookie(zkBookieReadOnlyPath); - try { - // Clear the current registered node - zk.delete(zkBookieRegPath, -1); - } catch (KeeperException.NoNodeException nne) { - LOG.warn("No writable bookie registered node {} when transitioning to readonly", - zkBookieRegPath, nne); - } - } catch (IOException e) { - LOG.error("Error in transition to ReadOnly Mode." - + " Shutting down", e); - triggerBookieShutdown(ExitCode.BOOKIE_EXCEPTION); - return; - } catch (KeeperException e) { + registrationManager.registerBookie(bookieId, true); + } catch (BookieException e) { LOG.error("Error in transition to ReadOnly Mode." + " Shutting down", e); triggerBookieShutdown(ExitCode.BOOKIE_EXCEPTION); return; - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - LOG.warn("Interrupted Exception while transitioning to ReadOnly Mode."); - return; } } @@ -1174,54 +1060,6 @@ public boolean isReadOnly() { return readOnly.get(); } - /** - * Create a new zookeeper client to zk cluster. - * - *

- * Bookie Server just used zk client when syncing ledgers for garbage collection. - * So when zk client is expired, it means this bookie server is not available in - * bookie server list. The bookie client will be notified for its expiration. No - * more bookie request will be sent to this server. So it's better to exit when zk - * expired. - *

- *

- * Since there are lots of bk operations cached in queue, so we wait for all the operations - * are processed and quit. It is done by calling shutdown. - *

- * - * @param conf server configuration - * - * @return zk client instance - */ - private ZooKeeper newZookeeper(final ServerConfiguration conf) - throws IOException, InterruptedException, KeeperException { - Set watchers = new HashSet(); - watchers.add(new Watcher() { - @Override - public void process(WatchedEvent event) { - if (!running) { - // do nothing until first registration - return; - } - // Check for expired connection. - if (event.getType().equals(EventType.None) && event.getState().equals(KeeperState.Expired)) { - zkRegistered.set(false); - // schedule a re-register operation - registerBookie(false); - } - } - }); - return ZooKeeperClient.newBuilder() - .connectString(conf.getZkServers()) - .sessionTimeoutMs(conf.getZkTimeout()) - .watchers(watchers) - .operationRetryPolicy(new BoundExponentialBackoffRetryPolicy(conf.getZkRetryBackoffStartMs(), - conf.getZkRetryBackoffMaxMs(), Integer.MAX_VALUE)) - .requestRateLimit(conf.getZkRequestRateLimit()) - .statsLogger(this.statsLogger.scope(BOOKIE_SCOPE)) - .build(); - } - public boolean isRunning() { return running; } @@ -1327,8 +1165,8 @@ synchronized int shutdown(int exitCode) { stateService.shutdown(); } // Shutdown the ZK client - if (zk != null) { - zk.close(); + if (registrationManager != null) { + registrationManager.close(); } } catch (InterruptedException ie) { LOG.error("Interrupted during shutting down bookie : ", ie); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieException.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieException.java index 57bf708684b..99ae39e76fb 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieException.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieException.java @@ -40,6 +40,10 @@ public BookieException(int code, String reason) { super(reason); } + public BookieException(int code, String reason, Throwable t) { + super(reason, t); + } + public static BookieException create(int code) { switch(code) { case Code.UnauthorizedAccessException: @@ -52,6 +56,12 @@ public static BookieException create(int code) { return new UpgradeException(); case Code.DiskPartitionDuplicationException: return new DiskPartitionDuplicationException(); + case Code.CookieNotFoundException: + return new CookieNotFoundException(); + case Code.MetadataStoreException: + return new MetadataStoreException(); + case Code.UnknownBookieIdException: + return new UnknownBookieIdException(); default: return new BookieIllegalOpException(); } @@ -66,10 +76,12 @@ public interface Code { int IllegalOpException = -100; int LedgerFencedException = -101; - int InvalidCookieException = -102; int UpgradeException = -103; int DiskPartitionDuplicationException = -104; + int CookieNotFoundException = -105; + int MetadataStoreException = -106; + int UnknownBookieIdException = -107; } public void setCode(int code) { @@ -101,6 +113,15 @@ public String getMessage(int code) { case Code.DiskPartitionDuplicationException: err = "Disk Partition Duplication is not allowed"; break; + case Code.CookieNotFoundException: + err = "Cookie not found"; + break; + case Code.MetadataStoreException: + err = "Error performing metadata operations"; + break; + case Code.UnknownBookieIdException: + err = "Unknown bookie id"; + break; default: err = "Invalid operation"; break; @@ -132,7 +153,15 @@ public BookieUnauthorizedAccessException() { */ public static class BookieIllegalOpException extends BookieException { public BookieIllegalOpException() { - super(Code.UnauthorizedAccessException); + super(Code.IllegalOpException); + } + + public BookieIllegalOpException(String reason) { + super(Code.IllegalOpException, reason); + } + + public BookieIllegalOpException(Throwable cause) { + super(Code.IllegalOpException, cause); } } @@ -164,6 +193,23 @@ public InvalidCookieException(Throwable cause) { } } + /** + * Signal that no cookie is found when starting a bookie. + */ + public static class CookieNotFoundException extends BookieException { + public CookieNotFoundException() { + this(""); + } + + public CookieNotFoundException(String reason) { + super(Code.CookieNotFoundException, reason); + } + + public CookieNotFoundException(Throwable cause) { + super(Code.CookieNotFoundException, cause); + } + } + /** * Signals that an exception occurs on upgrading a bookie. */ @@ -197,4 +243,40 @@ public DiskPartitionDuplicationException(String reason) { super(Code.DiskPartitionDuplicationException, reason); } } + + /** + * Signal when bookie has problems on accessing metadata store. + */ + public static class MetadataStoreException extends BookieException { + + public MetadataStoreException() { + this(""); + } + + public MetadataStoreException(String reason) { + super(Code.MetadataStoreException, reason); + } + + public MetadataStoreException(Throwable cause) { + super(Code.MetadataStoreException, cause); + } + + public MetadataStoreException(String reason, Throwable cause) { + super(Code.MetadataStoreException, reason, cause); + } + } + + /** + * Signal when bookie has problems on accessing metadata store. + */ + public static class UnknownBookieIdException extends BookieException { + + public UnknownBookieIdException() { + super(Code.UnknownBookieIdException); + } + + public UnknownBookieIdException(Throwable cause) { + super(Code.UnknownBookieIdException, cause); + } + } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieShell.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieShell.java index 1c08eb208f0..481c6477cc0 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieShell.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieShell.java @@ -47,6 +47,7 @@ import java.util.Map; import java.util.concurrent.TimeUnit; import java.util.function.Predicate; +import org.apache.bookkeeper.bookie.BookieException.CookieNotFoundException; import org.apache.bookkeeper.bookie.BookieException.InvalidCookieException; import org.apache.bookkeeper.bookie.EntryLogger.EntryLogScanner; import org.apache.bookkeeper.bookie.Journal.JournalScanner; @@ -61,6 +62,8 @@ import org.apache.bookkeeper.client.UpdateLedgerOp; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.discover.RegistrationManager; +import org.apache.bookkeeper.discover.ZKRegistrationManager; import org.apache.bookkeeper.meta.LedgerManager; import org.apache.bookkeeper.meta.LedgerManager.LedgerRange; import org.apache.bookkeeper.meta.LedgerManager.LedgerRangeIterator; @@ -69,6 +72,7 @@ import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; import org.apache.bookkeeper.replication.AuditorElector; +import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.util.DiskChecker; import org.apache.bookkeeper.util.EntryFormatter; import org.apache.bookkeeper.util.IOUtils; @@ -88,7 +92,6 @@ import org.apache.commons.io.FileUtils; import org.apache.commons.lang.StringUtils; import org.apache.commons.lang.mutable.MutableBoolean; -import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.ZooKeeper; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -261,18 +264,15 @@ int runCmd(CommandLine cmdLine) throws Exception { boolean result = Bookie.format(conf, interactive, force); // delete cookie if (cmdLine.hasOption("d")) { - ZooKeeperClient zkc = - ZooKeeperClient.newBuilder() - .connectString(conf.getZkServers()) - .sessionTimeoutMs(conf.getZkTimeout()) - .build(); + RegistrationManager rm = new ZKRegistrationManager(); + rm.initialize(conf, () -> {}, NullStatsLogger.INSTANCE); try { - Versioned cookie = Cookie.readFromZooKeeper(zkc, conf); - cookie.getValue().deleteFromZooKeeper(zkc, conf, cookie.getVersion()); - } catch (KeeperException.NoNodeException nne) { + Versioned cookie = Cookie.readFromRegistrationManager(rm, conf); + cookie.getValue().deleteFromRegistrationManager(rm, conf, cookie.getVersion()); + } catch (CookieNotFoundException nne) { LOG.warn("No cookie to remove : ", nne); } finally { - zkc.close(); + rm.close(); } } return (result) ? 0 : 1; @@ -324,7 +324,7 @@ int runCmd(CommandLine cmdLine) throws Exception { private int bkRecovery(ClientConfiguration conf, BookKeeperAdmin bkAdmin, String[] args, boolean deleteCookie) - throws InterruptedException, BKException, KeeperException, IOException { + throws InterruptedException, BKException, BookieException, IOException { final String bookieSrcString[] = args[0].split(":"); if (bookieSrcString.length != 2) { System.err.println("BookieSrc inputted has invalid format" @@ -347,10 +347,14 @@ private int bkRecovery(ClientConfiguration conf, BookKeeperAdmin bkAdmin, bkAdmin.recoverBookieData(bookieSrc, bookieDest); if (deleteCookie) { + ServerConfiguration serverConf = new ServerConfiguration(); + serverConf.addConfiguration(conf); + RegistrationManager rm = new ZKRegistrationManager(); try { - Versioned cookie = Cookie.readFromZooKeeper(bkAdmin.getZooKeeper(), conf, bookieSrc); - cookie.getValue().deleteFromZooKeeper(bkAdmin.getZooKeeper(), conf, bookieSrc, cookie.getVersion()); - } catch (KeeperException.NoNodeException nne) { + rm.initialize(serverConf, () -> {}, NullStatsLogger.INSTANCE); + Versioned cookie = Cookie.readFromRegistrationManager(rm, bookieSrc); + cookie.getValue().deleteFromRegistrationManager(rm, bookieSrc, cookie.getVersion()); + } catch (CookieNotFoundException nne) { LOG.warn("No cookie to remove for {} : ", bookieSrc, nne); } } @@ -1506,22 +1510,19 @@ int runCmd(CommandLine cmdLine) throws Exception { return updateBookieIdInCookie(bookieId, useHostName); } - private int updateBookieIdInCookie(final String bookieId, final boolean useHostname) throws IOException, + private int updateBookieIdInCookie(final String bookieId, final boolean useHostname) throws BookieException, InterruptedException { - ZooKeeper zk = null; + RegistrationManager rm = new ZKRegistrationManager(); try { - zk = ZooKeeperClient.newBuilder() - .connectString(bkConf.getZkServers()) - .sessionTimeoutMs(bkConf.getZkTimeout()) - .build(); + rm.initialize(bkConf, () -> {}, NullStatsLogger.INSTANCE); ServerConfiguration conf = new ServerConfiguration(bkConf); String newBookieId = Bookie.getBookieAddress(conf).toString(); // read oldcookie Versioned oldCookie = null; try { conf.setUseHostNameAsBookieID(!useHostname); - oldCookie = Cookie.readFromZooKeeper(zk, conf); - } catch (KeeperException.NoNodeException nne) { + oldCookie = Cookie.readFromRegistrationManager(rm, conf); + } catch (CookieNotFoundException nne) { LOG.error("Either cookie already updated with UseHostNameAsBookieID={} or no cookie exists!", useHostname, nne); return -1; @@ -1540,12 +1541,12 @@ private int updateBookieIdInCookie(final String bookieId, final boolean useHostn if (hasCookieUpdatedInDirs) { try { conf.setUseHostNameAsBookieID(useHostname); - Cookie.readFromZooKeeper(zk, conf); + Cookie.readFromRegistrationManager(rm, conf); // since newcookie exists, just do cleanup of oldcookie and return conf.setUseHostNameAsBookieID(!useHostname); - oldCookie.getValue().deleteFromZooKeeper(zk, conf, oldCookie.getVersion()); + oldCookie.getValue().deleteFromRegistrationManager(rm, conf, oldCookie.getVersion()); return 0; - } catch (KeeperException.NoNodeException nne) { + } catch (CookieNotFoundException nne) { if (LOG.isDebugEnabled()) { LOG.debug("Ignoring, cookie will be written to zookeeper"); } @@ -1569,20 +1570,17 @@ private int updateBookieIdInCookie(final String bookieId, final boolean useHostn } // writes newcookie to zookeeper conf.setUseHostNameAsBookieID(useHostname); - newCookie.writeToZooKeeper(zk, conf, Version.NEW); + newCookie.writeToRegistrationManager(rm, conf, Version.NEW); // delete oldcookie conf.setUseHostNameAsBookieID(!useHostname); - oldCookie.getValue().deleteFromZooKeeper(zk, conf, oldCookie.getVersion()); - } catch (KeeperException ke) { - LOG.error("KeeperException during cookie updation!", ke); - return -1; + oldCookie.getValue().deleteFromRegistrationManager(rm, conf, oldCookie.getVersion()); } catch (IOException ioe) { LOG.error("IOException during cookie updation!", ioe); return -1; } finally { - if (zk != null) { - zk.close(); + if (rm != null) { + rm.close(); } } return 0; @@ -1628,31 +1626,33 @@ String getUsage() { @Override int runCmd(CommandLine cmdLine) { ServerConfiguration conf = new ServerConfiguration(bkConf); - ZooKeeper zk; + RegistrationManager rm = new ZKRegistrationManager(); try { - zk = ZooKeeperClient.newBuilder() - .connectString(bkConf.getZkServers()) - .sessionTimeoutMs(bkConf.getZkTimeout()).build(); - } catch (KeeperException | InterruptedException | IOException e) { - LOG.error("Exception while establishing zookeeper connection.", e); - return -1; - } + try { + rm.initialize(bkConf, () -> {}, NullStatsLogger.INSTANCE); + } catch (BookieException e) { + LOG.error("Exception while establishing zookeeper connection.", e); + return -1; + } - List allLedgerDirs = Lists.newArrayList(); - allLedgerDirs.addAll(Arrays.asList(ledgerDirectories)); - if (indexDirectories != ledgerDirectories) { - allLedgerDirs.addAll(Arrays.asList(indexDirectories)); - } + List allLedgerDirs = Lists.newArrayList(); + allLedgerDirs.addAll(Arrays.asList(ledgerDirectories)); + if (indexDirectories != ledgerDirectories) { + allLedgerDirs.addAll(Arrays.asList(indexDirectories)); + } - try { - Bookie.checkEnvironmentWithStorageExpansion(conf, zk, + try { + Bookie.checkEnvironmentWithStorageExpansion(conf, rm, Lists.newArrayList(journalDirectories), allLedgerDirs); - } catch (BookieException | IOException e) { - LOG.error( + } catch (BookieException e) { + LOG.error( "Exception while updating cookie for storage expansion", e); - return -1; + return -1; + } + return 0; + } finally { + rm.close(); } - return 0; } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Cookie.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Cookie.java index ec7793c0295..e2d4f1281f8 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Cookie.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Cookie.java @@ -37,28 +37,23 @@ import java.io.StringReader; import java.net.InetSocketAddress; import java.net.UnknownHostException; -import java.util.List; import java.util.Set; -import org.apache.bookkeeper.conf.AbstractConfiguration; +import org.apache.bookkeeper.bookie.BookieException.InvalidCookieException; +import org.apache.bookkeeper.bookie.BookieException.UnknownBookieIdException; import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.discover.RegistrationManager; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.DataFormats.CookieFormat; import org.apache.bookkeeper.util.BookKeeperConstants; -import org.apache.bookkeeper.util.ZkUtils; import org.apache.bookkeeper.versioning.LongVersion; import org.apache.bookkeeper.versioning.Version; import org.apache.bookkeeper.versioning.Versioned; -import org.apache.zookeeper.CreateMode; -import org.apache.zookeeper.KeeperException; -import org.apache.zookeeper.ZooKeeper; -import org.apache.zookeeper.data.ACL; -import org.apache.zookeeper.data.Stat; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** * When a bookie starts for the first time it generates a cookie, and stores - * the cookie in zookeeper as well as in the each of the local filesystem + * the cookie in registration manager as well as in the each of the local filesystem * directories it uses. This cookie is used to ensure that for the life of the * bookie, its configuration stays the same. If any of the bookie directories * becomes unavailable, the bookie becomes unavailable. If the bookie changes @@ -233,79 +228,61 @@ public void writeToDirectory(File directory) throws IOException { } /** - * Writes cookie details to ZooKeeper. + * Writes cookie details to registration manager. * - * @param zk ZooKeeper instance + * @param rm registration manager * @param conf configuration * @param version version - * @throws KeeperException - * @throws InterruptedException - * @throws UnknownHostException + * @throws BookieException when fail to write the cookie. */ - public void writeToZooKeeper(ZooKeeper zk, ServerConfiguration conf, Version version) - throws KeeperException, InterruptedException, UnknownHostException { - List zkAcls = ZkUtils.getACLs(conf); - String bookieCookiePath = conf.getZkLedgersRootPath() + "/" - + BookKeeperConstants.COOKIE_NODE; - String zkPath = getZkPath(conf); - byte[] data = toString().getBytes(UTF_8); - if (Version.NEW == version) { - if (zk.exists(bookieCookiePath, false) == null) { - try { - zk.create(bookieCookiePath, new byte[0], - zkAcls, CreateMode.PERSISTENT); - } catch (KeeperException.NodeExistsException nne) { - LOG.info("More than one bookie tried to create {} at once. Safe to ignore", - bookieCookiePath); - } - } - zk.create(zkPath, data, - zkAcls, CreateMode.PERSISTENT); - } else { - if (!(version instanceof LongVersion)) { - throw new IllegalArgumentException("Invalid version type, expected ZkVersion type"); - } - zk.setData(zkPath, data, (int) ((LongVersion) version).getLongVersion()); + public void writeToRegistrationManager(RegistrationManager rm, ServerConfiguration conf, Version version) + throws BookieException { + BookieSocketAddress address = null; + try { + address = Bookie.getBookieAddress(conf); + } catch (UnknownHostException e) { + throw new UnknownBookieIdException(e); } + byte[] data = toString().getBytes(UTF_8); + rm.writeCookie(address.toString(), new Versioned<>(data, version)); } /** - * Deletes cookie from ZooKeeper and sets znode version to DEFAULT_COOKIE_ZNODE_VERSION. + * Deletes cookie from registration manager. * - * @param zk ZooKeeper instance + * @param rm registration manager * @param conf configuration - * @param version zookeeper version - * @throws KeeperException - * @throws InterruptedException - * @throws UnknownHostException + * @param version cookie version + * @throws BookieException when fail to delete cookie. */ - public void deleteFromZooKeeper(ZooKeeper zk, ServerConfiguration conf, Version version) throws KeeperException, - InterruptedException, UnknownHostException { - BookieSocketAddress address = Bookie.getBookieAddress(conf); - deleteFromZooKeeper(zk, conf, address, version); + public void deleteFromRegistrationManager(RegistrationManager rm, + ServerConfiguration conf, + Version version) throws BookieException { + BookieSocketAddress address = null; + try { + address = Bookie.getBookieAddress(conf); + } catch (UnknownHostException e) { + throw new UnknownBookieIdException(e); + } + deleteFromRegistrationManager(rm, address, version); } /** - * Delete cookie from zookeeper. + * Delete cookie from registration manager. * - * @param zk zookeeper client - * @param conf configuration instance + * @param rm registration manager * @param address bookie address * @param version cookie version - * @throws KeeperException - * @throws InterruptedException - * @throws UnknownHostException + * @throws BookieException when fail to delete cookie. */ - public void deleteFromZooKeeper(ZooKeeper zk, AbstractConfiguration conf, - BookieSocketAddress address, Version version) - throws KeeperException, InterruptedException, UnknownHostException { + public void deleteFromRegistrationManager(RegistrationManager rm, + BookieSocketAddress address, + Version version) throws BookieException { if (!(version instanceof LongVersion)) { throw new IllegalArgumentException("Invalid version type, expected ZkVersion type"); } - String zkPath = getZkPath(conf, address); - zk.delete(zkPath, (int) ((LongVersion) version).getLongVersion()); - LOG.info("Removed cookie from {} for bookie {}.", conf.getZkLedgersRootPath(), address); + rm.removeCookie(address.toString(), version); } /** @@ -326,48 +303,44 @@ static Builder generateCookie(ServerConfiguration conf) } /** - * Read cookie from ZooKeeper. + * Read cookie from registration manager. * - * @param zk ZooKeeper instance + * @param rm registration manager * @param conf configuration * @return versioned cookie object - * @throws KeeperException - * @throws InterruptedException - * @throws IOException - * @throws UnknownHostException + * @throws BookieException when fail to read cookie */ - public static Versioned readFromZooKeeper(ZooKeeper zk, ServerConfiguration conf) - throws KeeperException, InterruptedException, IOException, UnknownHostException { - return readFromZooKeeper(zk, conf, Bookie.getBookieAddress(conf)); + public static Versioned readFromRegistrationManager(RegistrationManager rm, ServerConfiguration conf) + throws BookieException { + try { + return readFromRegistrationManager(rm, Bookie.getBookieAddress(conf)); + } catch (UnknownHostException e) { + throw new UnknownBookieIdException(e); + } } /** - * Read cookie from zookeeper for a given bookie address. + * Read cookie from registration manager for a given bookie address. * - * @param zk zookeeper client - * @param conf configuration instance + * @param rm registration manager * @param address bookie address * @return versioned cookie object - * @throws KeeperException - * @throws InterruptedException - * @throws IOException - * @throws UnknownHostException + * @throws BookieException when fail to read cookie */ - public static Versioned readFromZooKeeper(ZooKeeper zk, AbstractConfiguration conf, BookieSocketAddress address) - throws KeeperException, InterruptedException, IOException, UnknownHostException { - String zkPath = getZkPath(conf, address); - - Stat stat = zk.exists(zkPath, false); - byte[] data = zk.getData(zkPath, false, stat); - BufferedReader reader = new BufferedReader(new StringReader(new String(data, UTF_8))); + public static Versioned readFromRegistrationManager(RegistrationManager rm, + BookieSocketAddress address) throws BookieException { + Versioned cookieData = rm.readCookie(address.toString()); + BufferedReader reader = new BufferedReader(new StringReader(new String(cookieData.getValue(), UTF_8))); try { - Builder builder = parse(reader); - Cookie cookie = builder.build(); - // sets stat version from ZooKeeper - LongVersion version = new LongVersion(stat.getVersion()); - return new Versioned(cookie, version); - } finally { - reader.close(); + try { + Builder builder = parse(reader); + Cookie cookie = builder.build(); + return new Versioned(cookie, cookieData.getVersion()); + } finally { + reader.close(); + } + } catch (IOException ioe) { + throw new InvalidCookieException(ioe); } } @@ -389,30 +362,6 @@ public static Cookie readFromDirectory(File directory) throws IOException { } } - /** - * Returns cookie path in zookeeper. - * - * @param conf configuration - * @return cookie zk path - * @throws UnknownHostException - */ - static String getZkPath(ServerConfiguration conf) - throws UnknownHostException { - return getZkPath(conf, Bookie.getBookieAddress(conf)); - } - - /** - * Return cookie path for a given bookie address. - * - * @param conf configuration - * @param address bookie address - * @return cookie path for bookie - */ - static String getZkPath(AbstractConfiguration conf, BookieSocketAddress address) { - String bookieCookiePath = conf.getZkLedgersRootPath() + "/" + BookKeeperConstants.COOKIE_NODE; - return bookieCookiePath + "/" + address; - } - /** * Check whether the 'bookieHost' was created using a hostname or an IP * address. Represent as 'hostname/IPaddress' if the InetSocketAddress was diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/FileSystemUpgrade.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/FileSystemUpgrade.java index 045f8b2efbd..4158abddaf8 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/FileSystemUpgrade.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/FileSystemUpgrade.java @@ -36,20 +36,19 @@ import java.util.NoSuchElementException; import java.util.Scanner; import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.discover.RegistrationManager; +import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.util.BookKeeperConstants; import org.apache.bookkeeper.util.HardLink; +import org.apache.bookkeeper.util.ReflectionUtils; import org.apache.bookkeeper.versioning.Version; import org.apache.bookkeeper.versioning.Versioned; -import org.apache.bookkeeper.zookeeper.BoundExponentialBackoffRetryPolicy; -import org.apache.bookkeeper.zookeeper.ZooKeeperClient; import org.apache.commons.cli.BasicParser; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.HelpFormatter; import org.apache.commons.cli.Options; import org.apache.commons.configuration.ConfigurationException; import org.apache.commons.io.FileUtils; -import org.apache.zookeeper.KeeperException; -import org.apache.zookeeper.ZooKeeper; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -128,22 +127,15 @@ private static int detectPreviousVersion(File directory) throws IOException { } } - private static ZooKeeper newZookeeper(final ServerConfiguration conf) + private static RegistrationManager newRegistrationManager(final ServerConfiguration conf) throws BookieException.UpgradeException { + try { - int zkTimeout = conf.getZkTimeout(); - return ZooKeeperClient.newBuilder() - .connectString(conf.getZkServers()) - .sessionTimeoutMs(zkTimeout) - .operationRetryPolicy( - new BoundExponentialBackoffRetryPolicy(zkTimeout, zkTimeout, Integer.MAX_VALUE)) - .build(); - } catch (InterruptedException ie) { - throw new BookieException.UpgradeException(ie); - } catch (IOException ioe) { - throw new BookieException.UpgradeException(ioe); - } catch (KeeperException ke) { - throw new BookieException.UpgradeException(ke); + Class rmClass = conf.getRegistrationManagerClass(); + RegistrationManager rm = ReflectionUtils.newInstance(rmClass); + return rm.initialize(conf, () -> {}, NullStatsLogger.INSTANCE); + } catch (Exception e) { + throw new BookieException.UpgradeException(e); } } @@ -177,7 +169,7 @@ public static void upgrade(ServerConfiguration conf) throws BookieException.UpgradeException, InterruptedException { LOG.info("Upgrading..."); - ZooKeeper zk = newZookeeper(conf); + RegistrationManager rm = newRegistrationManager(conf); try { Map deferredMoves = new HashMap(); Cookie.Builder cookieBuilder = Cookie.generateCookie(conf); @@ -229,15 +221,15 @@ public boolean accept(File dir, String name) { } try { - c.writeToZooKeeper(zk, conf, Version.NEW); - } catch (KeeperException ke) { - LOG.error("Error writing cookie to zookeeper"); + c.writeToRegistrationManager(rm, conf, Version.NEW); + } catch (BookieException ke) { + LOG.error("Error writing cookie to registration manager"); throw new BookieException.UpgradeException(ke); } } catch (IOException ioe) { throw new BookieException.UpgradeException(ioe); } finally { - zk.close(); + rm.close(); } LOG.info("Done"); } @@ -283,7 +275,7 @@ public static void finalizeUpgrade(ServerConfiguration conf) public static void rollback(ServerConfiguration conf) throws BookieException.UpgradeException, InterruptedException { LOG.info("Rolling back upgrade..."); - ZooKeeper zk = newZookeeper(conf); + RegistrationManager rm = newRegistrationManager(conf); try { for (File d : getAllDirectories(conf)) { LOG.info("Rolling back {}", d); @@ -305,17 +297,14 @@ public static void rollback(ServerConfiguration conf) } } try { - Versioned cookie = Cookie.readFromZooKeeper(zk, conf); - cookie.getValue().deleteFromZooKeeper(zk, conf, cookie.getVersion()); - } catch (KeeperException ke) { - LOG.error("Error deleting cookie from ZooKeeper"); + Versioned cookie = Cookie.readFromRegistrationManager(rm, conf); + cookie.getValue().deleteFromRegistrationManager(rm, conf, cookie.getVersion()); + } catch (BookieException ke) { + LOG.error("Error deleting cookie from Registration Manager"); throw new BookieException.UpgradeException(ke); - } catch (IOException ioe) { - LOG.error("I/O Error deleting cookie from ZooKeeper"); - throw new BookieException.UpgradeException(ioe); } } finally { - zk.close(); + rm.close(); } LOG.info("Done"); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ServerConfiguration.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ServerConfiguration.java index aae3e7df199..10fb329b43c 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ServerConfiguration.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ServerConfiguration.java @@ -26,6 +26,8 @@ import org.apache.bookkeeper.bookie.InterleavedLedgerStorage; import org.apache.bookkeeper.bookie.LedgerStorage; import org.apache.bookkeeper.bookie.SortedLedgerStorage; +import org.apache.bookkeeper.discover.RegistrationManager; +import org.apache.bookkeeper.discover.ZKRegistrationManager; import org.apache.bookkeeper.server.component.ServerLifecycleComponent; import org.apache.bookkeeper.stats.NullStatsProvider; import org.apache.bookkeeper.stats.StatsProvider; @@ -173,6 +175,9 @@ public class ServerConfiguration extends AbstractConfiguration { // Lifecycle Components protected final static String EXTRA_SERVER_COMPONENTS = "extraServerComponents"; + // Registration + protected final static String REGISTRATION_MANAGER_CLASS = "registrationManagerClass"; + /** * Construct a default configuration object */ @@ -2384,4 +2389,27 @@ public ServerConfiguration setExtraServerComponents(String[] componentClasses) { return this; } + /** + * Set registration manager class + * + * @param regManagerClass + * ManagerClass + */ + public void setRegistrationManagerClass( + Class regManagerClass) { + setProperty(REGISTRATION_MANAGER_CLASS, regManagerClass); + } + + /** + * Get Registration Manager Class. + * + * @return registration manager class. + */ + public Class getRegistrationManagerClass() + throws ConfigurationException { + return ReflectionUtils.getClass(this, REGISTRATION_MANAGER_CLASS, + ZKRegistrationManager.class, RegistrationManager.class, + defaultLoader); + } + } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/RegistrationClient.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/RegistrationClient.java new file mode 100644 index 00000000000..7f6160adb63 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/RegistrationClient.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.bookkeeper.discover; + +import java.util.List; +import java.util.concurrent.CompletableFuture; + +/** + * A registration client, which the bookkeeper client will use to interact with registration service. + */ +public interface RegistrationClient { + + /** + * Get the list of available bookie identifiers. + * + * @return a future represents the list of available bookies + */ + CompletableFuture> getAvailableBookies(); + +} diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/RegistrationManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/RegistrationManager.java new file mode 100644 index 00000000000..2b274d752a4 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/RegistrationManager.java @@ -0,0 +1,108 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.bookkeeper.discover; + +import org.apache.bookkeeper.bookie.BookieException; +import org.apache.bookkeeper.common.annotation.InterfaceAudience.LimitedPrivate; +import org.apache.bookkeeper.common.annotation.InterfaceStability.Evolving; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.stats.StatsLogger; +import org.apache.bookkeeper.versioning.Version; +import org.apache.bookkeeper.versioning.Versioned; + +/** + * Registration manager interface, which a bookie server will use to do the registration process. + */ +@LimitedPrivate +@Evolving +public interface RegistrationManager extends AutoCloseable { + + /** + * Registration Listener on listening the registration state. + */ + @FunctionalInterface + interface RegistrationListener { + + /** + * Signal when registration is expired. + */ + void onRegistrationExpired(); + + } + + RegistrationManager initialize(ServerConfiguration conf, + RegistrationListener listener, + StatsLogger statsLogger) throws BookieException; + + @Override + void close(); + + /** + * Return the cluster instance id. + * + * @return the cluster instance id. + */ + String getClusterInstanceId() throws BookieException; + + /** + * Registering the bookie server as bookieId. + * + * @param bookieId bookie id + * @param readOnly whether to register it as writable or readonly + * @throws BookieException when fail to register a bookie. + */ + void registerBookie(String bookieId, boolean readOnly) throws BookieException; + + /** + * Unregistering the bookie server as bookieId. + * + * @param bookieId bookie id + * @param readOnly whether to register it as writable or readonly + * @throws BookieException when fail to unregister a bookie. + */ + void unregisterBookie(String bookieId, boolean readOnly) throws BookieException; + + /** + * Write the cookie data, which will be used for verifying the integrity of the bookie environment. + * + * @param bookieId bookie id + * @param cookieData cookie data + * @throws BookieException when fail to write cookie + */ + void writeCookie(String bookieId, Versioned cookieData) throws BookieException; + + /** + * Read the cookie data, which will be used for verifying the integrity of the bookie environment. + * + * @param bookieId bookie id + * @return versioned cookie data + * @throws BookieException when fail to read cookie + */ + Versioned readCookie(String bookieId) throws BookieException; + + /** + * Remove the cookie data. + * + * @param bookieId bookie id + * @param version version of the cookie data + * @throws BookieException when fail to remove cookie + */ + void removeCookie(String bookieId, Version version) throws BookieException; + +} diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/ZKRegistrationManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/ZKRegistrationManager.java new file mode 100644 index 00000000000..dbe924d075b --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/ZKRegistrationManager.java @@ -0,0 +1,391 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.bookkeeper.discover; + +import static java.nio.charset.StandardCharsets.UTF_8; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIE_SCOPE; +import static org.apache.bookkeeper.util.BookKeeperConstants.COOKIE_NODE; +import static org.apache.bookkeeper.util.BookKeeperConstants.INSTANCEID; +import static org.apache.bookkeeper.util.BookKeeperConstants.READONLY; + +import com.google.common.annotations.VisibleForTesting; +import java.io.IOException; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.bookie.BookieException; +import org.apache.bookkeeper.bookie.BookieException.BookieIllegalOpException; +import org.apache.bookkeeper.bookie.BookieException.CookieNotFoundException; +import org.apache.bookkeeper.bookie.BookieException.MetadataStoreException; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.stats.StatsLogger; +import org.apache.bookkeeper.util.ZkUtils; +import org.apache.bookkeeper.versioning.LongVersion; +import org.apache.bookkeeper.versioning.Version; +import org.apache.bookkeeper.versioning.Versioned; +import org.apache.bookkeeper.zookeeper.BoundExponentialBackoffRetryPolicy; +import org.apache.bookkeeper.zookeeper.ZooKeeperClient; +import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.KeeperException.NoNodeException; +import org.apache.zookeeper.KeeperException.NodeExistsException; +import org.apache.zookeeper.WatchedEvent; +import org.apache.zookeeper.Watcher; +import org.apache.zookeeper.Watcher.Event.EventType; +import org.apache.zookeeper.Watcher.Event.KeeperState; +import org.apache.zookeeper.ZooKeeper; +import org.apache.zookeeper.data.ACL; +import org.apache.zookeeper.data.Stat; + +/** + * ZooKeeper Based {@link RegistrationManager}. + */ +@Slf4j +public class ZKRegistrationManager implements RegistrationManager { + + private ServerConfiguration conf; + private ZooKeeper zk; + private List zkAcls; + private volatile boolean running = false; + + // cookie path + private String cookiePath; + // registration paths + protected String bookieRegistrationPath; + protected String bookieReadonlyRegistrationPath; + + private StatsLogger statsLogger; + + @Override + public RegistrationManager initialize(ServerConfiguration conf, + RegistrationListener listener, + StatsLogger statsLogger) + throws BookieException { + if (null == conf.getZkServers()) { + log.warn("No ZK servers passed to Bookie constructor so BookKeeper clients won't know about this server!"); + return null; + } + + this.conf = conf; + this.zkAcls = ZkUtils.getACLs(conf); + this.statsLogger = statsLogger; + + this.cookiePath = conf.getZkLedgersRootPath() + "/" + COOKIE_NODE; + this.bookieRegistrationPath = conf.getZkAvailableBookiesPath(); + this.bookieReadonlyRegistrationPath = this.bookieRegistrationPath + "/" + READONLY; + + try { + this.zk = newZookeeper(conf, listener); + } catch (InterruptedException | KeeperException | IOException e) { + throw new MetadataStoreException(e); + } + return this; + } + + @VisibleForTesting + public void setZk(ZooKeeper zk) { + this.zk = zk; + } + + @VisibleForTesting + public ZooKeeper getZk() { + return this.zk; + } + + /** + * Create a new zookeeper client to zk cluster. + * + *

+ * Bookie Server just used zk client when syncing ledgers for garbage collection. + * So when zk client is expired, it means this bookie server is not available in + * bookie server list. The bookie client will be notified for its expiration. No + * more bookie request will be sent to this server. So it's better to exit when zk + * expired. + *

+ *

+ * Since there are lots of bk operations cached in queue, so we wait for all the operations + * are processed and quit. It is done by calling shutdown. + *

+ * + * @param conf server configuration + * + * @return zk client instance + */ + private ZooKeeper newZookeeper(final ServerConfiguration conf, RegistrationListener listener) + throws InterruptedException, KeeperException, IOException { + Set watchers = new HashSet(); + watchers.add(event -> { + if (!running) { + // do nothing until first registration + return; + } + // Check for expired connection. + if (event.getType().equals(EventType.None) && event.getState().equals(KeeperState.Expired)) { + listener.onRegistrationExpired(); + } + }); + return ZooKeeperClient.newBuilder() + .connectString(conf.getZkServers()) + .sessionTimeoutMs(conf.getZkTimeout()) + .watchers(watchers) + .operationRetryPolicy(new BoundExponentialBackoffRetryPolicy(conf.getZkRetryBackoffStartMs(), + conf.getZkRetryBackoffMaxMs(), Integer.MAX_VALUE)) + .requestRateLimit(conf.getZkRequestRateLimit()) + .statsLogger(this.statsLogger.scope(BOOKIE_SCOPE)) + .build(); + } + + @Override + public void close() { + if (null != zk) { + try { + zk.close(); + } catch (InterruptedException e) { + log.warn("Interrupted on closing zookeeper client", e); + } + } + } + + private String getCookiePath(String bookieId) { + return this.cookiePath + "/" + bookieId; + } + + // + // Registration Management + // + + /** + * Check existence of regPath and wait it expired if possible. + * + * @param regPath reg node path. + * @return true if regPath exists, otherwise return false + * @throws IOException if can't create reg path + */ + protected boolean checkRegNodeAndWaitExpired(String regPath) throws IOException { + final CountDownLatch prevNodeLatch = new CountDownLatch(1); + Watcher zkPrevRegNodewatcher = new Watcher() { + @Override + public void process(WatchedEvent event) { + // Check for prev znode deletion. Connection expiration is + // not handling, since bookie has logic to shutdown. + if (EventType.NodeDeleted == event.getType()) { + prevNodeLatch.countDown(); + } + } + }; + try { + Stat stat = zk.exists(regPath, zkPrevRegNodewatcher); + if (null != stat) { + // if the ephemeral owner isn't current zookeeper client + // wait for it to be expired. + if (stat.getEphemeralOwner() != zk.getSessionId()) { + log.info("Previous bookie registration znode: {} exists, so waiting zk sessiontimeout:" + + " {} ms for znode deletion", regPath, conf.getZkTimeout()); + // waiting for the previous bookie reg znode deletion + if (!prevNodeLatch.await(conf.getZkTimeout(), TimeUnit.MILLISECONDS)) { + throw new NodeExistsException(regPath); + } else { + return false; + } + } + return true; + } else { + return false; + } + } catch (KeeperException ke) { + log.error("ZK exception checking and wait ephemeral znode {} expired : ", regPath, ke); + throw new IOException("ZK exception checking and wait ephemeral znode " + + regPath + " expired", ke); + } catch (InterruptedException ie) { + log.error("Interrupted checking and wait ephemeral znode {} expired : ", regPath, ie); + throw new IOException("Interrupted checking and wait ephemeral znode " + + regPath + " expired", ie); + } + } + + @Override + public void registerBookie(String bookieId, boolean readOnly) throws BookieException { + if (!readOnly) { + String regPath = bookieRegistrationPath + "/" + bookieId; + doRegisterBookie(regPath); + } else { + doRegisterReadOnlyBookie(bookieId); + } + } + + private void doRegisterBookie(String regPath) throws BookieException { + // ZK ephemeral node for this Bookie. + try { + if (!checkRegNodeAndWaitExpired(regPath)) { + // Create the ZK ephemeral node for this Bookie. + zk.create(regPath, new byte[0], zkAcls, CreateMode.EPHEMERAL); + } + } catch (KeeperException ke) { + log.error("ZK exception registering ephemeral Znode for Bookie!", ke); + // Throw an IOException back up. This will cause the Bookie + // constructor to error out. Alternatively, we could do a System + // exit here as this is a fatal error. + throw new MetadataStoreException(ke); + } catch (InterruptedException ie) { + log.error("Interrupted exception registering ephemeral Znode for Bookie!", ie); + // Throw an IOException back up. This will cause the Bookie + // constructor to error out. Alternatively, we could do a System + // exit here as this is a fatal error. + throw new MetadataStoreException(ie); + } catch (IOException e) { + throw new MetadataStoreException(e); + } + } + + private void doRegisterReadOnlyBookie(String bookieId) throws BookieException { + try { + if (null == zk.exists(this.bookieReadonlyRegistrationPath, false)) { + try { + zk.create(this.bookieReadonlyRegistrationPath, new byte[0], + zkAcls, CreateMode.PERSISTENT); + } catch (NodeExistsException e) { + // this node is just now created by someone. + } + } + + String regPath = bookieReadonlyRegistrationPath + "/" + bookieId; + doRegisterBookie(regPath); + // clear the write state + regPath = bookieRegistrationPath + "/" + bookieId; + try { + // Clear the current registered node + zk.delete(regPath, -1); + } catch (KeeperException.NoNodeException nne) { + log.warn("No writable bookie registered node {} when transitioning to readonly", + regPath, nne); + } + } catch (KeeperException | InterruptedException e) { + throw new MetadataStoreException(e); + } + } + + @Override + public void unregisterBookie(String bookieId, boolean readOnly) throws BookieException { + String regPath; + if (!readOnly) { + regPath = bookieRegistrationPath + "/" + bookieId; + } else { + regPath = bookieReadonlyRegistrationPath + "/" + bookieId; + } + doUnregisterBookie(regPath); + } + + private void doUnregisterBookie(String regPath) throws BookieException { + try { + zk.delete(regPath, -1); + } catch (InterruptedException | KeeperException e) { + throw new MetadataStoreException(e); + } + } + + // + // Cookie Management + // + + @Override + public void writeCookie(String bookieId, + Versioned cookieData) throws BookieException { + String zkPath = getCookiePath(bookieId); + try { + if (Version.NEW == cookieData.getVersion()) { + if (zk.exists(cookiePath, false) == null) { + try { + zk.create(cookiePath, new byte[0], zkAcls, CreateMode.PERSISTENT); + } catch (NodeExistsException nne) { + log.info("More than one bookie tried to create {} at once. Safe to ignore.", + cookiePath); + } + } + zk.create(zkPath, cookieData.getValue(), zkAcls, CreateMode.PERSISTENT); + } else { + if (!(cookieData.getVersion() instanceof LongVersion)) { + throw new BookieIllegalOpException("Invalid version type, expected it to be LongVersion"); + } + zk.setData( + zkPath, + cookieData.getValue(), + (int) ((LongVersion) cookieData.getVersion()).getLongVersion()); + } + } catch (InterruptedException | KeeperException e) { + throw new MetadataStoreException("Failed to write cookie for bookie " + bookieId); + } + } + + @Override + public Versioned readCookie(String bookieId) throws BookieException { + String zkPath = getCookiePath(bookieId); + try { + Stat stat = zk.exists(zkPath, false); + byte[] data = zk.getData(zkPath, false, stat); + // sets stat version from ZooKeeper + LongVersion version = new LongVersion(stat.getVersion()); + return new Versioned<>(data, version); + } catch (NoNodeException nne) { + throw new CookieNotFoundException(bookieId); + } catch (KeeperException | InterruptedException e) { + throw new MetadataStoreException("Failed to read cookie for bookie " + bookieId); + } + } + + @Override + public void removeCookie(String bookieId, Version version) throws BookieException { + String zkPath = getCookiePath(bookieId); + try { + zk.delete(zkPath, (int) ((LongVersion) version).getLongVersion()); + } catch (NoNodeException e) { + throw new CookieNotFoundException(bookieId); + } catch (InterruptedException | KeeperException e) { + throw new MetadataStoreException("Failed to delete cookie for bookie " + bookieId); + } + + log.info("Removed cookie from {} for bookie {}.", cookiePath, bookieId); + } + + + @Override + public String getClusterInstanceId() throws BookieException { + String instanceId = null; + try { + if (zk.exists(conf.getZkLedgersRootPath(), null) == null) { + log.error("BookKeeper metadata doesn't exist in zookeeper. " + + "Has the cluster been initialized? " + + "Try running bin/bookkeeper shell metaformat"); + throw new KeeperException.NoNodeException("BookKeeper metadata"); + } + try { + byte[] data = zk.getData(conf.getZkLedgersRootPath() + "/" + + INSTANCEID, false, null); + instanceId = new String(data, UTF_8); + } catch (KeeperException.NoNodeException e) { + log.info("INSTANCEID not exists in zookeeper. Not considering it for data verification"); + } + } catch (KeeperException | InterruptedException e) { + throw new MetadataStoreException("Failed to get cluster instance id", e); + } + return instanceId; + } +} diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/package-info.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/package-info.java new file mode 100644 index 00000000000..ecc26f55577 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/package-info.java @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Classes related to service discovery. + */ +package org.apache.bookkeeper.discover; \ No newline at end of file diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/http/ExpandStorageService.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/http/ExpandStorageService.java index e51ca124d29..cd87a24a58d 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/http/ExpandStorageService.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/http/ExpandStorageService.java @@ -29,9 +29,12 @@ import org.apache.bookkeeper.bookie.Bookie; import org.apache.bookkeeper.bookie.BookieException; import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.discover.RegistrationManager; +import org.apache.bookkeeper.discover.ZKRegistrationManager; import org.apache.bookkeeper.http.service.HttpEndpointService; import org.apache.bookkeeper.http.service.HttpServiceRequest; import org.apache.bookkeeper.http.service.HttpServiceResponse; +import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.zookeeper.ZooKeeper; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -80,9 +83,11 @@ public HttpServiceResponse handle(HttpServiceRequest request) throws Exception { } try { - Bookie.checkEnvironmentWithStorageExpansion(conf, zk, + RegistrationManager rm = new ZKRegistrationManager(); + rm.initialize(conf, () -> { }, NullStatsLogger.INSTANCE); + Bookie.checkEnvironmentWithStorageExpansion(conf, rm, Lists.newArrayList(journalDirectories), allLedgerDirs); - } catch (BookieException | IOException e) { + } catch (BookieException e) { LOG.error("Exception occurred while updating cookie for storage expansion", e); response.setCode(HttpServer.StatusCode.INTERNAL_ERROR); response.setBody("Exception while updating cookie for storage expansion"); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/http/RecoveryBookieService.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/http/RecoveryBookieService.java index 74ffa3336a0..b079f1b36bc 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/http/RecoveryBookieService.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/http/RecoveryBookieService.java @@ -27,11 +27,14 @@ import org.apache.bookkeeper.client.BookKeeperAdmin; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.discover.RegistrationManager; import org.apache.bookkeeper.http.service.HttpEndpointService; import org.apache.bookkeeper.http.service.HttpServiceRequest; import org.apache.bookkeeper.http.service.HttpServiceResponse; import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.util.JsonUtil; +import org.apache.bookkeeper.util.ReflectionUtils; import org.apache.bookkeeper.versioning.Versioned; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -103,7 +106,10 @@ public HttpServiceResponse handle(HttpServiceRequest request) throws Exception { if (HttpServer.Method.PUT == request.getMethod() && !requestJsonBody.bookie_src.isEmpty()) { - ClientConfiguration adminConf = new ClientConfiguration(conf); + + Class rmClass = conf.getRegistrationManagerClass(); + RegistrationManager rm = ReflectionUtils.newInstance(rmClass); + rm.initialize(conf, () -> {}, NullStatsLogger.INSTANCE); String bookieSrcString[] = requestJsonBody.bookie_src.get(0).split(":"); BookieSocketAddress bookieSrc = new BookieSocketAddress( @@ -122,8 +128,8 @@ public HttpServiceResponse handle(HttpServiceRequest request) throws Exception { LOG.info("Start recovering bookie."); bka.recoverBookieData(bookieSrc, bookieDest); if (deleteCookie) { - Versioned cookie = Cookie.readFromZooKeeper(bka.getZooKeeper(), adminConf, bookieSrc); - cookie.getValue().deleteFromZooKeeper(bka.getZooKeeper(), adminConf, bookieSrc, cookie.getVersion()); + Versioned cookie = Cookie.readFromRegistrationManager(rm, bookieSrc); + cookie.getValue().deleteFromRegistrationManager(rm, bookieSrc, cookie.getVersion()); } LOG.info("Complete recovering bookie"); } catch (Exception e) { diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieInitializationTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieInitializationTest.java index 135736fa221..3783c57ccf2 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieInitializationTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieInitializationTest.java @@ -38,6 +38,8 @@ import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.conf.TestBKConfiguration; +import org.apache.bookkeeper.discover.RegistrationManager; +import org.apache.bookkeeper.discover.ZKRegistrationManager; import org.apache.bookkeeper.proto.BookieServer; import org.apache.bookkeeper.replication.ReplicationException.CompatibilityException; import org.apache.bookkeeper.replication.ReplicationException.UnavailableException; @@ -49,6 +51,7 @@ import org.apache.bookkeeper.util.DiskChecker; import org.apache.bookkeeper.zookeeper.ZooKeeperClient; import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.ZooKeeper; import org.apache.zookeeper.data.Stat; import org.junit.Assert; import org.junit.Rule; @@ -66,10 +69,11 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase { @Rule public final TestName runtime = new TestName(); + RegistrationManager rm; public BookieInitializationTest() { super(0); - String ledgersPath = "/" + runtime.getMethodName(); + String ledgersPath = "/" + "ledgers" + runtime.getMethodName(); baseClientConf.setZkLedgersRootPath(ledgersPath); baseConf.setZkLedgersRootPath(ledgersPath); } @@ -78,6 +82,15 @@ public BookieInitializationTest() { public void setUp() throws Exception { super.setUp(); zkUtil.createBKEnsemble("/" + runtime.getMethodName()); + rm = new ZKRegistrationManager(); + } + + @Override + public void tearDown() throws Exception { + super.tearDown(); + if(rm != null) { + rm.close(); + } } private static class MockBookie extends Bookie { @@ -109,8 +122,10 @@ protected Bookie newBookie(ServerConfiguration conf) throws IOException, KeeperException, InterruptedException, BookieException { MockBookie bookie = new MockBookie(conf); - bookie.zk = zkc; - zkc.close(); + rm.initialize(conf, () -> {}, NullStatsLogger.INSTANCE); + bookie.registrationManager = rm; + ((ZKRegistrationManager) bookie.registrationManager).setZk(zkc); + ((ZKRegistrationManager) bookie.registrationManager).getZk().close(); return bookie; } }; @@ -134,15 +149,19 @@ public void testBookieRegistrationWithSameZooKeeperClient() throws Exception { + conf.getBookiePort(); MockBookie b = new MockBookie(conf); - b.zk = zkc; + conf.setZkServers(zkUtil.getZooKeeperConnectString()); + rm.initialize(conf, () -> {}, NullStatsLogger.INSTANCE); + b.registrationManager = rm; + b.testRegisterBookie(conf); + ZooKeeper zooKeeper = ((ZKRegistrationManager) rm).getZk(); Assert.assertNotNull("Bookie registration node doesn't exists!", - zkc.exists(bkRegPath, false)); + zooKeeper.exists(bkRegPath, false)); // test register bookie again if the registeration node is created by itself. b.testRegisterBookie(conf); Assert.assertNotNull("Bookie registration node doesn't exists!", - zkc.exists(bkRegPath, false)); + zooKeeper.exists(bkRegPath, false)); } /** @@ -161,18 +180,24 @@ public void testBookieRegistration() throws Exception { final String bkRegPath = conf.getZkAvailableBookiesPath() + "/" + InetAddress.getLocalHost().getHostAddress() + ":" + conf.getBookiePort(); - MockBookie b = new MockBookie(conf); - b.zk = zkc; + + conf.setZkServers(zkUtil.getZooKeeperConnectString()); + rm.initialize(conf, () -> {}, NullStatsLogger.INSTANCE); + b.registrationManager = rm; + b.testRegisterBookie(conf); - Stat bkRegNode1 = zkc.exists(bkRegPath, false); + + Stat bkRegNode1 = ((ZKRegistrationManager) rm).getZk().exists(bkRegPath, false); Assert.assertNotNull("Bookie registration node doesn't exists!", bkRegNode1); // simulating bookie restart, on restart bookie will create new // zkclient and doing the registration. ZooKeeperClient newZk = createNewZKClient(); - b.zk = newZk; + RegistrationManager newRm = new ZKRegistrationManager(); + newRm.initialize(conf, () -> {}, NullStatsLogger.INSTANCE); + b.registrationManager = newRm; try { // deleting the znode, so that the bookie registration should @@ -228,7 +253,11 @@ public void testRegNodeExistsAfterSessionTimeOut() throws Exception { + conf.getBookiePort(); MockBookie b = new MockBookie(conf); - b.zk = zkc; + + conf.setZkServers(zkUtil.getZooKeeperConnectString()); + rm.initialize(conf, () -> {}, NullStatsLogger.INSTANCE); + b.registrationManager = rm; + b.testRegisterBookie(conf); Stat bkRegNode1 = zkc.exists(bkRegPath, false); Assert.assertNotNull("Bookie registration node doesn't exists!", @@ -237,14 +266,19 @@ public void testRegNodeExistsAfterSessionTimeOut() throws Exception { // simulating bookie restart, on restart bookie will create new // zkclient and doing the registration. ZooKeeperClient newzk = createNewZKClient(); - b.zk = newzk; + RegistrationManager newRm = new ZKRegistrationManager(); + newRm.initialize(conf, () -> {}, NullStatsLogger.INSTANCE); + b.registrationManager = newRm; try { b.testRegisterBookie(conf); fail("Should throw NodeExistsException as the znode is not getting expired"); } catch (IOException e) { - Throwable t = e.getCause(); - if (t instanceof KeeperException) { - KeeperException ke = (KeeperException) t; + Throwable t1 = e.getCause(); // BookieException.MetadataStoreException + Throwable t2 = t1.getCause(); // IOException + Throwable t3 = t2.getCause(); // KeeperException.NodeExistsException + + if (t3 instanceof KeeperException) { + KeeperException ke = (KeeperException) t3; Assert.assertTrue("ErrorCode:" + ke.code() + ", Registration node doesn't exists", ke.code() == KeeperException.Code.NODEEXISTS); @@ -263,6 +297,7 @@ public void testRegNodeExistsAfterSessionTimeOut() throws Exception { throw e; } finally { newzk.close(); + newRm.close(); } } @@ -280,11 +315,17 @@ public void testDuplicateBookieServerStartup() throws Exception { tmpDir.getPath()).setLedgerDirNames( new String[] { tmpDir.getPath() }); BookieServer bs1 = new BookieServer(conf); + conf.setZkServers(zkUtil.getZooKeeperConnectString()); + rm.initialize(conf, () -> {}, NullStatsLogger.INSTANCE); + bs1.getBookie().setRegistrationManager(rm); bs1.start(); // starting bk server with same conf try { BookieServer bs2 = new BookieServer(conf); + RegistrationManager newRm = new ZKRegistrationManager(); + newRm.initialize(conf, () -> {}, NullStatsLogger.INSTANCE); + bs2.getBookie().registrationManager = newRm; bs2.start(); fail("Should throw BindException, as the bk server is already running!"); } catch (BindException e) { @@ -313,6 +354,9 @@ public void testBookieServerStartupOnEphemeralPorts() throws Exception { ServerConfiguration conf1 = new ServerConfiguration(); conf1.addConfiguration(conf); BookieServer bs1 = new BookieServer(conf1); + conf.setZkServers(zkUtil.getZooKeeperConnectString()); + rm.initialize(conf, () -> {}, NullStatsLogger.INSTANCE); + bs1.getBookie().registrationManager = rm; bs1.start(); assertFalse(0 == conf1.getBookiePort()); @@ -320,6 +364,9 @@ public void testBookieServerStartupOnEphemeralPorts() throws Exception { ServerConfiguration conf2 = new ServerConfiguration(); conf2.addConfiguration(conf); BookieServer bs2 = new BookieServer(conf2); + RegistrationManager newRm = new ZKRegistrationManager(); + newRm.initialize(conf, () -> {}, NullStatsLogger.INSTANCE); + bs2.getBookie().registrationManager = newRm; bs2.start(); assertFalse(0 == conf2.getBookiePort()); @@ -343,7 +390,7 @@ public void testStartBookieWithoutZKServer() throws Exception { try { new Bookie(conf); fail("Should throw ConnectionLossException as ZKServer is not running!"); - } catch (KeeperException.ConnectionLossException e) { + } catch (BookieException.MetadataStoreException e) { // expected behaviour } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CookieTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CookieTest.java index f622448d000..e817776b550 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CookieTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CookieTest.java @@ -28,16 +28,21 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import com.google.common.base.Stopwatch; import com.google.common.collect.Sets; import java.io.File; import java.io.IOException; import java.util.HashSet; import java.util.List; import java.util.Set; +import java.util.concurrent.TimeUnit; import org.apache.bookkeeper.client.BookKeeperAdmin; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.conf.TestBKConfiguration; +import org.apache.bookkeeper.discover.RegistrationManager; +import org.apache.bookkeeper.discover.ZKRegistrationManager; +import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.apache.bookkeeper.test.PortManager; import org.apache.bookkeeper.util.IOUtils; @@ -45,6 +50,7 @@ import org.apache.bookkeeper.versioning.Version; import org.apache.bookkeeper.versioning.Versioned; import org.apache.commons.io.FileUtils; +import org.junit.After; import org.junit.Assert; import org.junit.Test; @@ -68,6 +74,24 @@ private String newDirectory(boolean createCurDir) throws IOException { return d.getPath(); } + RegistrationManager rm; + + @Override + public void setUp() throws Exception { + super.setUp(); + rm = new ZKRegistrationManager(); + baseConf.setZkServers(zkUtil.getZooKeeperConnectString()); + rm.initialize(baseConf, () -> {}, NullStatsLogger.INSTANCE); + } + + @Override + public void tearDown() throws Exception { + super.tearDown(); + if(rm != null) { + rm.close(); + } + } + /** * Test starting bookie with clean state. */ @@ -98,7 +122,7 @@ public void testBadJournalCookie() throws Exception { .setBookiePort(bookiePort); Cookie.Builder cookieBuilder = Cookie.generateCookie(conf1); Cookie c = cookieBuilder.build(); - c.writeToZooKeeper(zkc, conf1, Version.NEW); + c.writeToRegistrationManager(rm, conf1, Version.NEW); String journalDir = newDirectory(); String ledgerDir = newDirectory(); @@ -590,15 +614,15 @@ public void testWriteToZooKeeper() throws Exception { Bookie b = new Bookie(conf); // should work fine b.start(); b.shutdown(); - Versioned zkCookie = Cookie.readFromZooKeeper(zkc, conf); + Versioned zkCookie = Cookie.readFromRegistrationManager(rm, conf); Version version1 = zkCookie.getVersion(); Assert.assertTrue("Invalid type expected ZkVersion type", version1 instanceof LongVersion); LongVersion zkVersion1 = (LongVersion) version1; Cookie cookie = zkCookie.getValue(); - cookie.writeToZooKeeper(zkc, conf, version1); + cookie.writeToRegistrationManager(rm, conf, version1); - zkCookie = Cookie.readFromZooKeeper(zkc, conf); + zkCookie = Cookie.readFromRegistrationManager(rm, conf); Version version2 = zkCookie.getVersion(); Assert.assertTrue("Invalid type expected ZkVersion type", version2 instanceof LongVersion); @@ -620,8 +644,8 @@ public void testDeleteFromZooKeeper() throws Exception { Bookie b = new Bookie(conf); // should work fine b.start(); b.shutdown(); - Versioned zkCookie = Cookie.readFromZooKeeper(zkc, conf); + Versioned zkCookie = Cookie.readFromRegistrationManager(rm, conf); Cookie cookie = zkCookie.getValue(); - cookie.deleteFromZooKeeper(zkc, conf, zkCookie.getVersion()); + cookie.deleteFromRegistrationManager(rm, conf, zkCookie.getVersion()); } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/UpdateCookieCmdTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/UpdateCookieCmdTest.java index f5b646e7bd4..c11fbca6ee3 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/UpdateCookieCmdTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/UpdateCookieCmdTest.java @@ -20,17 +20,21 @@ */ package org.apache.bookkeeper.bookie; +import static org.apache.bookkeeper.util.BookKeeperConstants.COOKIE_NODE; + import java.io.File; import java.io.IOException; import java.net.UnknownHostException; import java.util.List; +import org.apache.bookkeeper.discover.RegistrationManager; +import org.apache.bookkeeper.discover.ZKRegistrationManager; +import org.apache.bookkeeper.stats.NullStatsLogger; import org.junit.Assert; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.proto.BookieServer; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; -import org.apache.bookkeeper.util.BookKeeperConstants; import org.apache.bookkeeper.versioning.Version; import org.apache.zookeeper.KeeperException; import org.junit.Test; @@ -44,10 +48,29 @@ public class UpdateCookieCmdTest extends BookKeeperClusterTestCase { private final static Logger LOG = LoggerFactory.getLogger(UpdateCookieCmdTest.class); + RegistrationManager rm; + public UpdateCookieCmdTest() { super(1); } + @Override + public void setUp() throws Exception { + super.setUp(); + LOG.info("setUp ZKRegistrationManager"); + rm = new ZKRegistrationManager(); + baseConf.setZkServers(zkUtil.getZooKeeperConnectString()); + rm.initialize(baseConf, () -> {}, NullStatsLogger.INSTANCE); + } + + @Override + public void tearDown() throws Exception { + super.tearDown(); + if(rm != null) { + rm.close(); + } + } + /** * updatecookie to hostname */ @@ -109,12 +132,12 @@ public void testWhenBothIPaddressAndHostNameCookiesExists() throws Exception { // creates cookie with ipaddress ServerConfiguration conf = bsConfs.get(0); conf.setUseHostNameAsBookieID(true); // sets to hostname - Cookie cookie = Cookie.readFromZooKeeper(zkc, conf).getValue(); + Cookie cookie = Cookie.readFromRegistrationManager(rm, conf).getValue(); Cookie.Builder cookieBuilder = Cookie.newBuilder(cookie); conf.setUseHostNameAsBookieID(false); // sets to hostname final String newBookieHost = Bookie.getBookieAddress(conf).toString(); cookieBuilder.setBookieHost(newBookieHost); - cookieBuilder.build().writeToZooKeeper(zkc, conf, Version.NEW); + cookieBuilder.build().writeToRegistrationManager(rm, conf, Version.NEW); verifyCookieInZooKeeper(conf, 2); // again issue hostname cmd @@ -125,7 +148,7 @@ public void testWhenBothIPaddressAndHostNameCookiesExists() throws Exception { Assert.assertEquals("Failed to return the error code!", 0, bkShell.run(argv)); conf.setUseHostNameAsBookieID(true); - cookie = Cookie.readFromZooKeeper(zkc, conf).getValue(); + cookie = Cookie.readFromRegistrationManager(rm, conf).getValue(); Assert.assertFalse("Cookie has created with IP!", cookie.isBookieHostCreatedFromIp()); // ensure the old cookie is deleted verifyCookieInZooKeeper(conf, 1); @@ -148,7 +171,7 @@ public void testWhenNoCookieExists() throws Exception { BookieServer bks = bs.get(0); bks.shutdown(); - String zkCookiePath = Cookie.getZkPath(conf); + String zkCookiePath = conf.getZkLedgersRootPath() + "/" + COOKIE_NODE + "/" + Bookie.getBookieAddress(conf); Assert.assertNotNull("Cookie path doesn't still exists!", zkc.exists(zkCookiePath, false)); zkc.delete(zkCookiePath, -1); Assert.assertNull("Cookie path still exists!", zkc.exists(zkCookiePath, false)); @@ -163,7 +186,7 @@ public void testWhenNoCookieExists() throws Exception { private void verifyCookieInZooKeeper(ServerConfiguration conf, int expectedCount) throws KeeperException, InterruptedException { List cookies; - String bookieCookiePath1 = conf.getZkLedgersRootPath() + "/" + BookKeeperConstants.COOKIE_NODE; + String bookieCookiePath1 = conf.getZkLedgersRootPath() + "/" + COOKIE_NODE; cookies = zkc.getChildren(bookieCookiePath1, false); Assert.assertEquals("Wrongly updated the cookie!", expectedCount, cookies.size()); } @@ -174,7 +197,7 @@ private void updateCookie(String option, String optionVal, boolean useHostNameAs bks.shutdown(); conf.setUseHostNameAsBookieID(!useHostNameAsBookieID); - Cookie cookie = Cookie.readFromZooKeeper(zkc, conf).getValue(); + Cookie cookie = Cookie.readFromRegistrationManager(rm, conf).getValue(); final boolean previousBookieID = cookie.isBookieHostCreatedFromIp(); Assert.assertEquals("Wrong cookie!", useHostNameAsBookieID, previousBookieID); @@ -187,7 +210,7 @@ private void updateCookie(String option, String optionVal, boolean useHostNameAs Assert.assertEquals("Failed to return exit code!", 0, bkShell.run(argv)); newconf.setUseHostNameAsBookieID(useHostNameAsBookieID); - cookie = Cookie.readFromZooKeeper(zkc, newconf).getValue(); + cookie = Cookie.readFromRegistrationManager(rm, newconf).getValue(); Assert.assertEquals("Wrongly updated cookie!", previousBookieID, !cookie.isBookieHostCreatedFromIp()); Assert.assertEquals("Wrongly updated cookie!", useHostNameAsBookieID, !cookie.isBookieHostCreatedFromIp()); verifyCookieInZooKeeper(newconf, 1); From 1ea27189c73869d0279f1b0d7e737f65933bbf21 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Fri, 27 Oct 2017 16:09:48 -0700 Subject: [PATCH 05/15] ISSUE #596 ISSUE #583: Auto replication should honor ensemble placement policy Descriptions of the changes in this PR: This pull request ports the changes from [twitter/bookkeeperfc7e171](https://github.com/twitter/bookkeeper/commit/fc7e171135a58cddb9e91f5f614b3ceb6f9f9fee) The changes include: 1. when bookkeeper admin re-replicates a ledger, it will pick a bookie from the available bookies in the cluster to satisfy the placement constraint. (#596) 2. hence, remove `targetBookie` from ReplicationWorker, because the parameter will volatile the placement constraint. (#583) 3. at the same time, change `LedgerFragement` to represent the number of bookies that need to be check and replicate. a) the ledger checker can use the correct bookie index for verifying the existence of entries in a bookie (for stripping case) b) only read entries one time when need to replicate them to multiple bookies. Author: Sijie Guo Reviewers: Ivan Kelly This closes #641 from sijie/twitter_autorecovery_fixes, closes #596, closes #583 --- bookkeeper-server/conf/log4j.shell.properties | 7 + .../bookkeeper/client/BookKeeperAdmin.java | 486 +++++++++++------- .../bookkeeper/client/LedgerChecker.java | 167 ++++-- .../bookkeeper/client/LedgerFragment.java | 106 +++- .../client/LedgerFragmentReplicator.java | 173 +++---- .../bookkeeper/conf/ServerConfiguration.java | 21 + .../bookkeeper/replication/Auditor.java | 12 +- .../replication/AutoRecoveryMain.java | 9 +- .../replication/ReplicationStats.java | 1 + .../replication/ReplicationWorker.java | 168 +++--- .../client/BookKeeperCloseTest.java | 21 +- .../bookkeeper/client/BookieRecoveryTest.java | 9 +- .../bookkeeper/client/TestLedgerChecker.java | 55 +- .../client/TestLedgerFragmentReplication.java | 15 +- .../replication/BookieAutoRecoveryTest.java | 4 +- .../replication/TestReplicationWorker.java | 64 +-- 16 files changed, 778 insertions(+), 540 deletions(-) diff --git a/bookkeeper-server/conf/log4j.shell.properties b/bookkeeper-server/conf/log4j.shell.properties index 58d6ea6891d..c2f2e0e9d11 100644 --- a/bookkeeper-server/conf/log4j.shell.properties +++ b/bookkeeper-server/conf/log4j.shell.properties @@ -36,6 +36,13 @@ log4j.appender.CONSOLE.Threshold=INFO log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout log4j.appender.CONSOLE.layout.ConversionPattern=%d{ABSOLUTE} %-5p %m%n +# verbose console logging +log4j.appender.VERBOSECONSOLE=org.apache.log4j.ConsoleAppender +log4j.appender.VERBOSECONSOLE.Threshold=INFO +log4j.appender.VERBOSECONSOLE.layout=org.apache.log4j.PatternLayout +log4j.appender.VERBOSECONSOLE.layout.ConversionPattern=%m%n + +log4j.logger.verbose=INFO,VERBOSECONSOLE log4j.logger.org.apache.zookeeper=ERROR log4j.logger.org.apache.bookkeeper=ERROR log4j.logger.org.apache.bookkeeper.bookie.BookieShell=INFO diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperAdmin.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperAdmin.java index a0d2fe2f309..d2e5db3f376 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperAdmin.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperAdmin.java @@ -23,18 +23,22 @@ import static com.google.common.base.Charsets.UTF_8; import static com.google.common.base.Preconditions.checkArgument; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; import com.google.common.util.concurrent.AbstractFuture; import java.io.IOException; -import java.net.UnknownHostException; import java.util.ArrayList; import java.util.Collection; import java.util.Enumeration; import java.util.HashMap; +import java.util.HashSet; import java.util.Iterator; import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.NoSuchElementException; +import java.util.Optional; import java.util.Random; import java.util.Set; import java.util.UUID; @@ -70,7 +74,6 @@ import org.apache.zookeeper.AsyncCallback; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; -import org.apache.zookeeper.KeeperException.Code; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.util.ZkUtils; @@ -84,7 +87,10 @@ * Admin client for BookKeeper clusters */ public class BookKeeperAdmin implements AutoCloseable { + private final static Logger LOG = LoggerFactory.getLogger(BookKeeperAdmin.class); + private final static Logger VERBOSE = LoggerFactory.getLogger("verbose"); + // ZK client instance private ZooKeeper zk; private final boolean ownsZK; @@ -495,9 +501,20 @@ public SyncObject() { */ public void recoverBookieData(final BookieSocketAddress bookieSrc, final BookieSocketAddress bookieDest) throws InterruptedException, BKException { + Set bookiesSrc = Sets.newHashSet(bookieSrc); + recoverBookieData(bookiesSrc); + } + + public void recoverBookieData(final Set bookiesSrc) + throws InterruptedException, BKException { + recoverBookieData(bookiesSrc, false, false); + } + + public void recoverBookieData(final Set bookiesSrc, boolean dryrun, boolean skipOpenLedgers) + throws InterruptedException, BKException { SyncObject sync = new SyncObject(); // Call the async method to recover bookie data. - asyncRecoverBookieData(bookieSrc, bookieDest, new RecoverCallback() { + asyncRecoverBookieData(bookiesSrc, dryrun, skipOpenLedgers, new RecoverCallback() { @Override public void recoverComplete(int rc, Object ctx) { LOG.info("Recover bookie operation completed with rc: " + rc); @@ -520,7 +537,7 @@ public void recoverComplete(int rc, Object ctx) { throw BKException.create(sync.rc); } } - + /** * Async method to rebuild and recover the ledger fragments data that was * stored on the source bookie. That bookie could have failed completely and @@ -535,89 +552,26 @@ public void recoverComplete(int rc, Object ctx) { * @param bookieSrc * Source bookie that had a failure. We want to replicate the * ledger fragments that were stored there. - * @param bookieDest - * Optional destination bookie that if passed, we will copy all - * of the ledger fragments from the source bookie over to it. * @param cb * RecoverCallback to invoke once all of the data on the dead * bookie has been recovered and replicated. * @param context * Context for the RecoverCallback to call. */ - public void asyncRecoverBookieData(final BookieSocketAddress bookieSrc, final BookieSocketAddress bookieDest, + public void asyncRecoverBookieData(final BookieSocketAddress bookieSrc, final RecoverCallback cb, final Object context) { - // Sync ZK to make sure we're reading the latest bookie data. - zk.sync(bookiesPath, new AsyncCallback.VoidCallback() { - @Override - public void processResult(int rc, String path, Object ctx) { - if (rc != Code.OK.intValue()) { - LOG.error("ZK error syncing: ", KeeperException.create(KeeperException.Code.get(rc), path)); - cb.recoverComplete(BKException.Code.ZKException, context); - return; - } - getAvailableBookies(bookieSrc, bookieDest, cb, context); - } + Set bookiesSrc = Sets.newHashSet(bookieSrc); + asyncRecoverBookieData(bookiesSrc, cb, context); + } - }, null); + public void asyncRecoverBookieData(final Set bookieSrc, + final RecoverCallback cb, final Object context) { + asyncRecoverBookieData(bookieSrc, false, false, cb, context); } - /** - * This method asynchronously gets the set of available Bookies that the - * dead input bookie's data will be copied over into. If the user passed in - * a specific destination bookie, then just use that one. Otherwise, we'll - * randomly pick one of the other available bookies to use for each ledger - * fragment we are replicating. - * - * @param bookieSrc - * Source bookie that had a failure. We want to replicate the - * ledger fragments that were stored there. - * @param bookieDest - * Optional destination bookie that if passed, we will copy all - * of the ledger fragments from the source bookie over to it. - * @param cb - * RecoverCallback to invoke once all of the data on the dead - * bookie has been recovered and replicated. - * @param context - * Context for the RecoverCallback to call. - */ - private void getAvailableBookies(final BookieSocketAddress bookieSrc, final BookieSocketAddress bookieDest, - final RecoverCallback cb, final Object context) { - final List availableBookies = new LinkedList(); - if (bookieDest != null) { - availableBookies.add(bookieDest); - // Now poll ZK to get the active ledgers - getActiveLedgers(bookieSrc, bookieDest, cb, context, availableBookies); - } else { - zk.getChildren(bookiesPath, null, new AsyncCallback.ChildrenCallback() { - @Override - public void processResult(int rc, String path, Object ctx, List children) { - if (rc != Code.OK.intValue()) { - LOG.error("ZK error getting bookie nodes: ", KeeperException.create(KeeperException.Code - .get(rc), path)); - cb.recoverComplete(BKException.Code.ZKException, context); - return; - } - for (String bookieNode : children) { - if (BookKeeperConstants.READONLY - .equals(bookieNode)) { - // exclude the readonly node from available bookies. - continue; - } - BookieSocketAddress addr; - try { - addr = new BookieSocketAddress(bookieNode); - } catch (UnknownHostException nhe) { - LOG.error("Bookie Node retrieved from ZK has invalid name format: " + bookieNode); - cb.recoverComplete(BKException.Code.ZKException, context); - return; - } - availableBookies.add(addr); - } - // Now poll ZK to get the active ledgers - getActiveLedgers(bookieSrc, null, cb, context, availableBookies); - } - }, null); - } + public void asyncRecoverBookieData(final Set bookieSrc, boolean dryrun, + final boolean skipOpenLedgers, final RecoverCallback cb, final Object context) { + getActiveLedgers(bookieSrc, dryrun, skipOpenLedgers, cb, context); } /** @@ -626,25 +580,21 @@ public void processResult(int rc, String path, Object ctx, List children * determine if any of the ledger fragments for it were stored at the dead * input bookie. * - * @param bookieSrc - * Source bookie that had a failure. We want to replicate the + * @param bookiesSrc + * Source bookies that had a failure. We want to replicate the * ledger fragments that were stored there. - * @param bookieDest - * Optional destination bookie that if passed, we will copy all - * of the ledger fragments from the source bookie over to it. + * @param dryrun + * dryrun the recover procedure. + * @param skipOpenLedgers + * Skip recovering open ledgers. * @param cb * RecoverCallback to invoke once all of the data on the dead * bookie has been recovered and replicated. * @param context * Context for the RecoverCallback to call. - * @param availableBookies - * List of Bookie Servers that are available to use for - * replicating data on the failed bookie. This could contain a - * single bookie server if the user explicitly chose a bookie - * server to replicate data to. */ - private void getActiveLedgers(final BookieSocketAddress bookieSrc, final BookieSocketAddress bookieDest, - final RecoverCallback cb, final Object context, final List availableBookies) { + private void getActiveLedgers(final Set bookiesSrc, final boolean dryrun, + final boolean skipOpenLedgers, final RecoverCallback cb, final Object context) { // Wrapper class around the RecoverCallback so it can be used // as the final VoidCallback to process ledgers class RecoverCallbackWrapper implements AsyncCallback.VoidCallback { @@ -663,7 +613,7 @@ public void processResult(int rc, String path, Object ctx) { Processor ledgerProcessor = new Processor() { @Override public void process(Long ledgerId, AsyncCallback.VoidCallback iterCallback) { - recoverLedger(bookieSrc, ledgerId, iterCallback, availableBookies); + recoverLedger(bookiesSrc, ledgerId, dryrun, skipOpenLedgers, iterCallback); } }; bkc.getLedgerManager().asyncProcessLedgers( @@ -671,42 +621,25 @@ ledgerProcessor, new RecoverCallbackWrapper(cb), context, BKException.Code.OK, BKException.Code.LedgerRecoveryException); } - /** - * Get a new random bookie, but ensure that it isn't one that is already - * in the ensemble for the ledger. - */ - private BookieSocketAddress getNewBookie(final List bookiesAlreadyInEnsemble, - final List availableBookies) - throws BKException.BKNotEnoughBookiesException { - ArrayList candidates = new ArrayList(); - candidates.addAll(availableBookies); - candidates.removeAll(bookiesAlreadyInEnsemble); - if (candidates.size() == 0) { - throw new BKException.BKNotEnoughBookiesException(); - } - return candidates.get(rand.nextInt(candidates.size())); - } - /** * This method asynchronously recovers a given ledger if any of the ledger * entries were stored on the failed bookie. * - * @param bookieSrc - * Source bookie that had a failure. We want to replicate the + * @param bookiesSrc + * Source bookies that had a failure. We want to replicate the * ledger fragments that were stored there. * @param lId * Ledger id we want to recover. - * @param ledgerIterCb + * @param dryrun + * printing the recovery plan without actually recovering bookies + * @param skipOpenLedgers + * Skip recovering open ledgers. + * @param finalLedgerIterCb * IterationCallback to invoke once we've recovered the current * ledger. - * @param availableBookies - * List of Bookie Servers that are available to use for - * replicating data on the failed bookie. This could contain a - * single bookie server if the user explicitly chose a bookie - * server to replicate data to. */ - private void recoverLedger(final BookieSocketAddress bookieSrc, final long lId, - final AsyncCallback.VoidCallback ledgerIterCb, final List availableBookies) { + private void recoverLedger(final Set bookiesSrc, final long lId, final boolean dryrun, + final boolean skipOpenLedgers, final AsyncCallback.VoidCallback finalLedgerIterCb) { if (LOG.isDebugEnabled()) { LOG.debug("Recovering ledger : {}", lId); } @@ -714,43 +647,73 @@ private void recoverLedger(final BookieSocketAddress bookieSrc, final long lId, asyncOpenLedgerNoRecovery(lId, new OpenCallback() { @Override public void openComplete(int rc, final LedgerHandle lh, Object ctx) { - if (rc != Code.OK.intValue()) { + if (rc != BKException.Code.OK) { LOG.error("BK error opening ledger: " + lId, BKException.create(rc)); - ledgerIterCb.processResult(rc, null, null); + finalLedgerIterCb.processResult(rc, null, null); return; } LedgerMetadata lm = lh.getLedgerMetadata(); - if (!lm.isClosed() && - lm.getEnsembles().size() > 0) { - Long lastKey = lm.getEnsembles().lastKey(); - ArrayList lastEnsemble = lm.getEnsembles().get(lastKey); - // the original write has not removed faulty bookie from - // current ledger ensemble. to avoid data loss issue in - // the case of concurrent updates to the ensemble composition, - // the recovery tool should first close the ledger - if (lastEnsemble.contains(bookieSrc)) { - // close opened non recovery ledger handle + if (skipOpenLedgers && !lm.isClosed() && !lm.isInRecovery()) { + LOG.info("Skip recovering open ledger {}.", lId); + try { + lh.close(); + } catch (InterruptedException ie) { + Thread.currentThread().interrupt(); + } catch (BKException bke) { + LOG.warn("Error on closing ledger handle for {}.", lId); + } + finalLedgerIterCb.processResult(BKException.Code.OK, null, null); + return; + } + + final boolean fenceRequired = !lm.isClosed() && containBookiesInLastEnsemble(lm, bookiesSrc); + // the original write has not removed faulty bookie from + // current ledger ensemble. to avoid data loss issue in + // the case of concurrent updates to the ensemble composition, + // the recovery tool should first close the ledger + if (!dryrun && fenceRequired) { + // close opened non recovery ledger handle + try { + lh.close(); + } catch (Exception ie) { + LOG.warn("Error closing non recovery ledger handle for ledger " + lId, ie); + } + asyncOpenLedger(lId, new OpenCallback() { + @Override + public void openComplete(int newrc, final LedgerHandle newlh, Object newctx) { + if (newrc != BKException.Code.OK) { + LOG.error("BK error close ledger: " + lId, BKException.create(newrc)); + finalLedgerIterCb.processResult(newrc, null, null); + return; + } + bkc.mainWorkerPool.submit(() -> { + // do recovery + recoverLedger(bookiesSrc, lId, dryrun, skipOpenLedgers, finalLedgerIterCb); + }); + } + }, null); + return; + } + + final AsyncCallback.VoidCallback ledgerIterCb = new AsyncCallback.VoidCallback() { + @Override + public void processResult(int rc, String path, Object ctx) { + if (BKException.Code.OK != rc) { + LOG.error("Failed to recover ledger {} : {}", lId, rc); + } else { + LOG.info("Recovered ledger {}.", lId); + } try { lh.close(); - } catch (Exception ie) { - LOG.warn("Error closing non recovery ledger handle for ledger " + lId, ie); + } catch (InterruptedException ie) { + Thread.currentThread().interrupt(); + } catch (BKException bke) { + LOG.warn("Error on cloing ledger handle for {}.", lId); } - asyncOpenLedger(lId, new OpenCallback() { - @Override - public void openComplete(int newrc, final LedgerHandle newlh, Object newctx) { - if (newrc != Code.OK.intValue()) { - LOG.error("BK error close ledger: " + lId, BKException.create(newrc)); - ledgerIterCb.processResult(newrc, null, null); - return; - } - // do recovery - recoverLedger(bookieSrc, lId, ledgerIterCb, availableBookies); - } - }, null); - return; + finalLedgerIterCb.processResult(rc, path, ctx); } - } + }; /* * This List stores the ledger fragments to recover indexed by @@ -773,7 +736,7 @@ public void openComplete(int newrc, final LedgerHandle newlh, Object newctx) { if (curEntryId != null) ledgerFragmentsRange.put(curEntryId, entry.getKey() - 1); curEntryId = entry.getKey(); - if (entry.getValue().contains(bookieSrc)) { + if (containBookies(entry.getValue(), bookiesSrc)) { /* * Current ledger fragment has entries stored on the * dead bookie so we'll need to recover them. @@ -797,6 +760,10 @@ public void openComplete(int newrc, final LedgerHandle newlh, Object newctx) { return; } + if (dryrun) { + VERBOSE.info("Recovered ledger {} : {}", lId, (fenceRequired ? "[fence required]" : "")); + } + /* * Multicallback for ledger. Once all fragments for the ledger have been recovered * trigger the ledgerIterCb @@ -810,47 +777,69 @@ public void openComplete(int newrc, final LedgerHandle newlh, Object newctx) { */ for (final Long startEntryId : ledgerFragmentsToRecover) { Long endEntryId = ledgerFragmentsRange.get(startEntryId); - BookieSocketAddress newBookie = null; + ArrayList ensemble = lh.getLedgerMetadata().getEnsembles().get(startEntryId); + // Get bookies to replace + Map targetBookieAddresses; try { - newBookie = getNewBookie(lh.getLedgerMetadata().getEnsembles().get(startEntryId), - availableBookies); - } catch (BKException.BKNotEnoughBookiesException bke) { - ledgerFragmentsMcb.processResult(BKException.Code.NotEnoughBookiesException, - null, null); + targetBookieAddresses = getReplacementBookies(lh, ensemble, bookiesSrc); + } catch (BKException.BKNotEnoughBookiesException e) { + if (!dryrun) { + ledgerFragmentsMcb.processResult(BKException.Code.NotEnoughBookiesException, null, null); + } else { + VERBOSE.info(" Fragment [{} - {}] : {}", startEntryId, endEntryId, + BKException.getMessage(BKException.Code.NotEnoughBookiesException)); + } continue; } - if (LOG.isDebugEnabled()) { - LOG.debug("Replicating fragment from [" + startEntryId - + "," + endEntryId + "] of ledger " + lh.getId() - + " to " + newBookie); - } - try { - LedgerFragmentReplicator.SingleFragmentCallback cb = new LedgerFragmentReplicator.SingleFragmentCallback( - ledgerFragmentsMcb, lh, startEntryId, bookieSrc, newBookie); - ArrayList currentEnsemble = lh.getLedgerMetadata().getEnsemble( - startEntryId); - int bookieIndex = -1; - if (null != currentEnsemble) { - for (int i = 0; i < currentEnsemble.size(); i++) { - if (currentEnsemble.get(i).equals(bookieSrc)) { - bookieIndex = i; - break; - } - } + if (dryrun) { + ArrayList newEnsemble = + replaceBookiesInEnsemble(ensemble, targetBookieAddresses); + VERBOSE.info(" Fragment [{} - {}] : ", startEntryId, endEntryId); + VERBOSE.info(" old ensemble : {}", formatEnsemble(ensemble, bookiesSrc, '*')); + VERBOSE.info(" new ensemble : {}", formatEnsemble(newEnsemble, bookiesSrc, '*')); + } else { + if (LOG.isDebugEnabled()) { + LOG.debug("Replicating fragment from [{}, {}] of ledger {} to {}", + startEntryId, endEntryId, lh.getId(), targetBookieAddresses); + } + try { + LedgerFragmentReplicator.SingleFragmentCallback cb = new LedgerFragmentReplicator.SingleFragmentCallback( + ledgerFragmentsMcb, lh, startEntryId, getReplacementBookiesMap(ensemble, targetBookieAddresses)); + LedgerFragment ledgerFragment = new LedgerFragment(lh, + startEntryId, endEntryId, targetBookieAddresses.keySet()); + asyncRecoverLedgerFragment(lh, ledgerFragment, cb, Sets.newHashSet(targetBookieAddresses.values())); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + return; } - LedgerFragment ledgerFragment = new LedgerFragment(lh, - startEntryId, endEntryId, bookieIndex); - asyncRecoverLedgerFragment(lh, ledgerFragment, cb, newBookie); - } catch(InterruptedException e) { - Thread.currentThread().interrupt(); - return; } } + if (dryrun) { + ledgerIterCb.processResult(BKException.Code.OK, null, null); + } } }, null); } + static String formatEnsemble(ArrayList ensemble, Set bookiesSrc, char marker) { + StringBuilder sb = new StringBuilder(); + sb.append("["); + for (int i = 0; i < ensemble.size(); i++) { + sb.append(ensemble.get(i)); + if (bookiesSrc.contains(ensemble.get(i))) { + sb.append(marker); + } else { + sb.append(' '); + } + if (i != ensemble.size() - 1) { + sb.append(", "); + } + } + sb.append("]"); + return sb.toString(); + } + /** * This method asynchronously recovers a ledger fragment which is a * contiguous portion of a ledger that was stored in an ensemble that @@ -863,16 +852,81 @@ public void openComplete(int newrc, final LedgerHandle newlh, Object newctx) { * @param ledgerFragmentMcb * - MultiCallback to invoke once we've recovered the current * ledger fragment. - * @param newBookie - * - New bookie we want to use to recover and replicate the + * @param newBookies + * - New bookies we want to use to recover and replicate the * ledger entries that were stored on the failed bookie. */ private void asyncRecoverLedgerFragment(final LedgerHandle lh, final LedgerFragment ledgerFragment, final AsyncCallback.VoidCallback ledgerFragmentMcb, - final BookieSocketAddress newBookie) - throws InterruptedException { - lfr.replicate(lh, ledgerFragment, ledgerFragmentMcb, newBookie); + final Set newBookies) throws InterruptedException { + lfr.replicate(lh, ledgerFragment, ledgerFragmentMcb, newBookies); + } + + private Map getReplacementBookies( + LedgerHandle lh, + List ensemble, + Set bookiesToRereplicate) + throws BKException.BKNotEnoughBookiesException { + Set bookieIndexesToRereplicate = Sets.newHashSet(); + for (int bookieIndex = 0; bookieIndex < ensemble.size(); bookieIndex++) { + BookieSocketAddress bookieInEnsemble = ensemble.get(bookieIndex); + if (bookiesToRereplicate.contains(bookieInEnsemble)) { + bookieIndexesToRereplicate.add(bookieIndex); + } + } + return getReplacementBookiesByIndexes( + lh, ensemble, bookieIndexesToRereplicate, Optional.of(bookiesToRereplicate)); + } + + private Map getReplacementBookiesByIndexes( + LedgerHandle lh, + List ensemble, + Set bookieIndexesToRereplicate, + Optional> excludedBookies) + throws BKException.BKNotEnoughBookiesException { + // target bookies to replicate + Map targetBookieAddresses = + Maps.newHashMapWithExpectedSize(bookieIndexesToRereplicate.size()); + // bookies to exclude for ensemble allocation + Set bookiesToExclude = Sets.newHashSet(); + if (excludedBookies.isPresent()) { + bookiesToExclude.addAll(excludedBookies.get()); + } + + // excluding bookies that need to be replicated + for (Integer bookieIndex : bookieIndexesToRereplicate) { + BookieSocketAddress bookie = ensemble.get(bookieIndex); + bookiesToExclude.add(bookie); + } + + // allocate bookies + for (Integer bookieIndex : bookieIndexesToRereplicate) { + BookieSocketAddress oldBookie = ensemble.get(bookieIndex); + BookieSocketAddress newBookie = + bkc.getPlacementPolicy().replaceBookie( + lh.getLedgerMetadata().getEnsembleSize(), + lh.getLedgerMetadata().getWriteQuorumSize(), + lh.getLedgerMetadata().getAckQuorumSize(), + lh.getLedgerMetadata().getCustomMetadata(), + ensemble, + oldBookie, + bookiesToExclude); + targetBookieAddresses.put(bookieIndex, newBookie); + bookiesToExclude.add(newBookie); + } + + return targetBookieAddresses; + } + + private ArrayList replaceBookiesInEnsemble( + List ensemble, + Map replacedBookies) { + ArrayList newEnsemble = Lists.newArrayList(ensemble); + for (Map.Entry entry : replacedBookies.entrySet()) { + newEnsemble.set(entry.getKey(), entry.getValue()); + } + return newEnsemble; } /** @@ -882,20 +936,32 @@ private void asyncRecoverLedgerFragment(final LedgerHandle lh, * - ledgerHandle * @param ledgerFragment * - LedgerFragment to replicate - * @param targetBookieAddress - * - target Bookie, to where entries should be replicated. */ public void replicateLedgerFragment(LedgerHandle lh, + final LedgerFragment ledgerFragment) + throws InterruptedException, BKException { + Optional> excludedBookies = Optional.empty(); + Map targetBookieAddresses = + getReplacementBookiesByIndexes(lh, ledgerFragment.getEnsemble(), + ledgerFragment.getBookiesIndexes(), excludedBookies); + replicateLedgerFragment(lh, ledgerFragment, targetBookieAddresses); + } + + private void replicateLedgerFragment(LedgerHandle lh, final LedgerFragment ledgerFragment, - final BookieSocketAddress targetBookieAddress) + final Map targetBookieAddresses) throws InterruptedException, BKException { CompletableFuture result = new CompletableFuture<>(); ResultCallBack resultCallBack = new ResultCallBack(result); - SingleFragmentCallback cb = new SingleFragmentCallback(resultCallBack, - lh, ledgerFragment.getFirstEntryId(), ledgerFragment - .getAddress(), targetBookieAddress); + SingleFragmentCallback cb = new SingleFragmentCallback( + resultCallBack, + lh, + ledgerFragment.getFirstEntryId(), + getReplacementBookiesMap(ledgerFragment, targetBookieAddresses)); - asyncRecoverLedgerFragment(lh, ledgerFragment, cb, targetBookieAddress); + Set targetBookieSet = Sets.newHashSet(); + targetBookieSet.addAll(targetBookieAddresses.values()); + asyncRecoverLedgerFragment(lh, ledgerFragment, cb, targetBookieSet); try { SyncCallbackUtils.waitForResult(result); @@ -903,6 +969,52 @@ public void replicateLedgerFragment(LedgerHandle lh, throw BKException.create(bkc.getReturnRc(err.getCode())); } } + + private static Map getReplacementBookiesMap( + ArrayList ensemble, + Map targetBookieAddresses) { + Map bookiesMap = + new HashMap(); + for (Map.Entry entry : targetBookieAddresses.entrySet()) { + BookieSocketAddress oldBookie = ensemble.get(entry.getKey()); + BookieSocketAddress newBookie = entry.getValue(); + bookiesMap.put(oldBookie, newBookie); + } + return bookiesMap; + } + + private static Map getReplacementBookiesMap( + LedgerFragment ledgerFragment, + Map targetBookieAddresses) { + Map bookiesMap = + new HashMap(); + for (Integer bookieIndex : ledgerFragment.getBookiesIndexes()) { + BookieSocketAddress oldBookie = ledgerFragment.getAddress(bookieIndex); + BookieSocketAddress newBookie = targetBookieAddresses.get(bookieIndex); + bookiesMap.put(oldBookie, newBookie); + } + return bookiesMap; + } + + private static boolean containBookiesInLastEnsemble(LedgerMetadata lm, + Set bookies) { + if (lm.getEnsembles().size() <= 0) { + return false; + } + Long lastKey = lm.getEnsembles().lastKey(); + ArrayList lastEnsemble = lm.getEnsembles().get(lastKey); + return containBookies(lastEnsemble, bookies); + } + + private static boolean containBookies(ArrayList ensemble, + Set bookies) { + for (BookieSocketAddress bookie : ensemble) { + if (bookies.contains(bookie)) { + return true; + } + } + return false; + } /** This is the class for getting the replication result */ static class ResultCallBack implements AsyncCallback.VoidCallback { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerChecker.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerChecker.java index a90a3662710..2ad80769871 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerChecker.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerChecker.java @@ -20,15 +20,14 @@ package org.apache.bookkeeper.client; import io.netty.buffer.ByteBuf; - import java.util.ArrayList; +import java.util.HashMap; import java.util.HashSet; import java.util.Map; import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; - import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookieClient; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; @@ -81,39 +80,125 @@ public void readEntryComplete(int rc, long ledgerId, long entryId, } } + /** + * This will collect the bad bookies inside a ledger fragment. + */ + private static class LedgerFragmentCallback implements GenericCallback { + + private final LedgerFragment fragment; + private final int bookieIndex; + // bookie index -> return code + private final Map badBookies; + private final AtomicInteger numBookies; + private final GenericCallback cb; + + LedgerFragmentCallback(LedgerFragment lf, + int bookieIndex, + GenericCallback cb, + Map badBookies, + AtomicInteger numBookies) { + this.fragment = lf; + this.bookieIndex = bookieIndex; + this.cb = cb; + this.badBookies = badBookies; + this.numBookies = numBookies; + } + + @Override + public void operationComplete(int rc, LedgerFragment lf) { + if (BKException.Code.OK != rc) { + synchronized (badBookies) { + badBookies.put(bookieIndex, rc); + } + } + if (numBookies.decrementAndGet() == 0) { + if (badBookies.isEmpty()) { + cb.operationComplete(BKException.Code.OK, fragment); + } else { + int rcToReturn = BKException.Code.NoBookieAvailableException; + for (Map.Entry entry : badBookies.entrySet()) { + rcToReturn = entry.getValue(); + if (entry.getValue() == BKException.Code.ClientClosedException) { + break; + } + } + cb.operationComplete(rcToReturn, + fragment.subset(badBookies.keySet())); + } + } + } + } + public LedgerChecker(BookKeeper bkc) { bookieClient = bkc.getBookieClient(); } + /** + * Verify a ledger fragment to collect bad bookies + * + * @param fragment + * fragment to verify + * @param cb + * callback + * @throws InvalidFragmentException + */ + private void verifyLedgerFragment(LedgerFragment fragment, + GenericCallback cb) + throws InvalidFragmentException, BKException { + Set bookiesToCheck = fragment.getBookiesIndexes(); + if (bookiesToCheck.isEmpty()) { + cb.operationComplete(BKException.Code.OK, fragment); + return; + } + + AtomicInteger numBookies = new AtomicInteger(bookiesToCheck.size()); + Map badBookies = new HashMap(); + for (Integer bookieIndex : bookiesToCheck) { + LedgerFragmentCallback lfCb = new LedgerFragmentCallback( + fragment, bookieIndex, cb, badBookies, numBookies); + verifyLedgerFragment(fragment, bookieIndex, lfCb); + } + } + + /** + * Verify a bookie inside a ledger fragment. + * + * @param fragment + * ledger fragment + * @param bookieIndex + * bookie index in the fragment + * @param cb + * callback + * @throws InvalidFragmentException + */ private void verifyLedgerFragment(LedgerFragment fragment, - GenericCallback cb) throws InvalidFragmentException { - long firstStored = fragment.getFirstStoredEntryId(); - long lastStored = fragment.getLastStoredEntryId(); - - // because of this if block, even if the bookie of the fragment is - // down, it considers Fragment is available/not-bad if firstStored - // and lastStored are LedgerHandle.INVALID_ENTRY_ID. - // So same logic is used in BookieShell.DecommissionBookieCmd.areEntriesOfSegmentStoredInTheBookie - // if any change is made here, then the changes should be in BookieShell also + int bookieIndex, + GenericCallback cb) + throws InvalidFragmentException { + long firstStored = fragment.getFirstStoredEntryId(bookieIndex); + long lastStored = fragment.getLastStoredEntryId(bookieIndex); + + BookieSocketAddress bookie = fragment.getAddress(bookieIndex); + if (null == bookie) { + throw new InvalidFragmentException(); + } + if (firstStored == LedgerHandle.INVALID_ENTRY_ID) { + // this fragment is not on this bookie if (lastStored != LedgerHandle.INVALID_ENTRY_ID) { throw new InvalidFragmentException(); } cb.operationComplete(BKException.Code.OK, fragment); - return; - } - if (firstStored == lastStored) { + } else if (firstStored == lastStored) { ReadManyEntriesCallback manycb = new ReadManyEntriesCallback(1, fragment, cb); - bookieClient.readEntry(fragment.getAddress(), fragment + bookieClient.readEntry(bookie, fragment .getLedgerId(), firstStored, manycb, null); } else { ReadManyEntriesCallback manycb = new ReadManyEntriesCallback(2, fragment, cb); - bookieClient.readEntry(fragment.getAddress(), fragment - .getLedgerId(), firstStored, manycb, null); - bookieClient.readEntry(fragment.getAddress(), fragment - .getLedgerId(), lastStored, manycb, null); + bookieClient.readEntry(bookie, fragment.getLedgerId(), firstStored, manycb, null); + bookieClient.readEntry(bookie, fragment.getLedgerId(), lastStored, manycb, null); } } @@ -191,44 +276,49 @@ public void checkLedger(LedgerHandle lh, for (Map.Entry> e : lh .getLedgerMetadata().getEnsembles().entrySet()) { if (curEntryId != null) { + Set bookieIndexes = new HashSet(); for (int i = 0; i < curEnsemble.size(); i++) { - fragments.add(new LedgerFragment(lh, curEntryId, - e.getKey() - 1, i)); + bookieIndexes.add(i); } + fragments.add(new LedgerFragment(lh, curEntryId, + e.getKey() - 1, bookieIndexes)); } curEntryId = e.getKey(); curEnsemble = e.getValue(); } - /* Checking the last fragment of the ledger can be complicated in some cases. + + + + /* Checking the last segment of the ledger can be complicated in some cases. * In the case that the ledger is closed, we can just check the fragments of - * the ledger as normal, except in the case that no entry was ever written, - * to the ledger, in which case we check no fragments. + * the segment as normal even if no data has ever been written to. * In the case that the ledger is open, but enough entries have been written, - * for lastAddConfirmed to be set above the start entry of the fragment, we + * for lastAddConfirmed to be set above the start entry of the segment, we * can also check as normal. - * However, if lastAddConfirmed cannot be trusted, such as when it's lower than - * the first entry id, or not set at all, we cannot be sure if there has been - * data written to the fragment. For this reason, we have to send a read request + * However, if ledger is open, sometimes lastAddConfirmed cannot be trusted, + * such as when it's lower than the first entry id, or not set at all, + * we cannot be sure if there has been data written to the segment. + * For this reason, we have to send a read request * to the bookies which should have the first entry. If they respond with * NoSuchEntry we can assume it was never written. If they respond with anything * else, we must assume the entry has been written, so we run the check. */ - if (curEntryId != null && !(lh.getLedgerMetadata().isClosed() && lh.getLastAddConfirmed() < curEntryId)) { + if (curEntryId != null) { long lastEntry = lh.getLastAddConfirmed(); - if (lastEntry < curEntryId) { + if (!lh.isClosed() && lastEntry < curEntryId) { lastEntry = curEntryId; } - final Set finalFragments = new HashSet(); + Set bookieIndexes = new HashSet(); for (int i = 0; i < curEnsemble.size(); i++) { - finalFragments.add(new LedgerFragment(lh, curEntryId, - lastEntry, i)); + bookieIndexes.add(i); } + final LedgerFragment lastLedgerFragment = new LedgerFragment(lh, curEntryId, + lastEntry, bookieIndexes); - // Check for the case that no last confirmed entry has - // been set. + // Check for the case that no last confirmed entry has been set if (curEntryId == lastEntry) { final long entryToRead = curEntryId; @@ -237,7 +327,7 @@ public void checkLedger(LedgerHandle lh, new GenericCallback() { public void operationComplete(int rc, Boolean result) { if (result) { - fragments.addAll(finalFragments); + fragments.add(lastLedgerFragment); } checkFragments(fragments, cb); } @@ -250,7 +340,7 @@ public void operationComplete(int rc, Boolean result) { } return; } else { - fragments.addAll(finalFragments); + fragments.add(lastLedgerFragment); } } @@ -275,7 +365,10 @@ private void checkFragments(Set fragments, LOG.error("Invalid fragment found : {}", r); allFragmentsCb.operationComplete( BKException.Code.IncorrectParameterException, r); + } catch (BKException e) { + LOG.error("BKException when checking fragment : {}", r, e); } } } + } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerFragment.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerFragment.java index e12f77f583f..3f02355a66c 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerFragment.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerFragment.java @@ -20,19 +20,20 @@ package org.apache.bookkeeper.client; import java.util.ArrayList; +import java.util.HashSet; import java.util.List; +import java.util.Set; import java.util.SortedMap; - import org.apache.bookkeeper.net.BookieSocketAddress; /** - * Represents the entries of a segment of a ledger which are stored on a single - * bookie in the segments bookie ensemble. - * + * Represents the entries of a segment of a ledger which are stored on subset of + * bookies in the segments bookie ensemble. + * * Used for checking and recovery */ public class LedgerFragment { - private final int bookieIndex; + private final Set bookieIndexes; private final List ensemble; private final long firstEntryId; private final long lastKnownEntryId; @@ -40,12 +41,14 @@ public class LedgerFragment { private final DistributionSchedule schedule; private final boolean isLedgerClosed; - LedgerFragment(LedgerHandle lh, long firstEntryId, - long lastKnownEntryId, int bookieIndex) { + LedgerFragment(LedgerHandle lh, + long firstEntryId, + long lastKnownEntryId, + Set bookieIndexes) { this.ledgerId = lh.getId(); this.firstEntryId = firstEntryId; this.lastKnownEntryId = lastKnownEntryId; - this.bookieIndex = bookieIndex; + this.bookieIndexes = bookieIndexes; this.ensemble = lh.getLedgerMetadata().getEnsemble(firstEntryId); this.schedule = lh.getDistributionSchedule(); SortedMap> ensembles = lh @@ -54,6 +57,27 @@ public class LedgerFragment { || !ensemble.equals(ensembles.get(ensembles.lastKey())); } + LedgerFragment(LedgerFragment lf, Set subset) { + this.ledgerId = lf.ledgerId; + this.firstEntryId = lf.firstEntryId; + this.lastKnownEntryId = lf.lastKnownEntryId; + this.bookieIndexes = subset; + this.ensemble = lf.ensemble; + this.schedule = lf.schedule; + this.isLedgerClosed = lf.isLedgerClosed; + } + + /** + * Return a ledger fragment contains subset of bookies. + * + * @param subset + * subset of bookies. + * @return ledger fragment contains subset of bookies + */ + public LedgerFragment subset(Set subset) { + return new LedgerFragment(this, subset); + } + /** * Returns true, if and only if the ledger fragment will never be modified * by any of the clients in future, otherwise false. i.e, @@ -83,23 +107,51 @@ long getLastKnownEntryId() { /** * Gets the failedBookie address */ - public BookieSocketAddress getAddress() { + public BookieSocketAddress getAddress(int bookieIndex) { return ensemble.get(bookieIndex); } - + + public Set getAddresses() { + Set addresses = new HashSet(); + for (int bookieIndex : bookieIndexes) { + addresses.add(ensemble.get(bookieIndex)); + } + return addresses; + } + /** * Gets the failedBookie index */ - public int getBookiesIndex() { - return bookieIndex; + public Set getBookiesIndexes() { + return bookieIndexes; } /** - * Gets the first stored entry id of the fragment in failed bookie. - * + * Gets the first stored entry id of the fragment in failed bookies. + * * @return entryId */ public long getFirstStoredEntryId() { + Long firstEntry = null; + for (int bookieIndex : bookieIndexes) { + Long firstStoredEntryForBookie = getFirstStoredEntryId(bookieIndex); + if (null == firstEntry) { + firstEntry = firstStoredEntryForBookie; + } else if (null != firstStoredEntryForBookie) { + firstEntry = Math.min(firstEntry, firstStoredEntryForBookie); + } + } + return null == firstEntry ? LedgerHandle.INVALID_ENTRY_ID : firstEntry; + } + + /** + * Get the first stored entry id of the fragment in the given failed bookies. + * + * @param bookieIndex + * the bookie index in the ensemble. + * @return first stored entry id on the bookie. + */ + public Long getFirstStoredEntryId(int bookieIndex) { long firstEntry = firstEntryId; for (int i = 0; i < ensemble.size() && firstEntry <= lastKnownEntryId; i++) { @@ -114,10 +166,30 @@ public long getFirstStoredEntryId() { /** * Gets the last stored entry id of the fragment in failed bookie. - * + * * @return entryId */ public long getLastStoredEntryId() { + Long lastEntry = null; + for (int bookieIndex : bookieIndexes) { + Long lastStoredEntryIdForBookie = getLastStoredEntryId(bookieIndex); + if (null == lastEntry) { + lastEntry = lastStoredEntryIdForBookie; + } else if (null != lastStoredEntryIdForBookie) { + lastEntry = Math.max(lastEntry, lastStoredEntryIdForBookie); + } + } + return null == lastEntry ? LedgerHandle.INVALID_ENTRY_ID : lastEntry; + } + + /** + * Get the last stored entry id of the fragment in the given failed bookie. + * + * @param bookieIndex + * the bookie index in the ensemble. + * @return first stored entry id on the bookie. + */ + public Long getLastStoredEntryId(int bookieIndex) { long lastEntry = lastKnownEntryId; for (int i = 0; i < ensemble.size() && lastEntry >= firstEntryId; i++) { if (schedule.hasEntry(lastEntry, bookieIndex)) { @@ -131,7 +203,7 @@ public long getLastStoredEntryId() { /** * Gets the ensemble of fragment - * + * * @return the ensemble for the segment which this fragment is a part of */ public List getEnsemble() { @@ -143,6 +215,6 @@ public String toString() { return String.format("Fragment(LedgerID: %d, FirstEntryID: %d[%d], " + "LastKnownEntryID: %d[%d], Host: %s, Closed: %s)", ledgerId, firstEntryId, getFirstStoredEntryId(), lastKnownEntryId, getLastStoredEntryId(), - getAddress(), isLedgerClosed); + getAddresses(), isLedgerClosed); } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerFragmentReplicator.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerFragmentReplicator.java index 172f9ec5cd7..95c48d5981b 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerFragmentReplicator.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerFragmentReplicator.java @@ -19,17 +19,20 @@ */ package org.apache.bookkeeper.client; +import static org.apache.bookkeeper.client.LedgerHandle.INVALID_ENTRY_ID; + import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; - import java.util.ArrayList; import java.util.Enumeration; import java.util.HashSet; import java.util.Iterator; import java.util.LinkedList; import java.util.List; +import java.util.Map; import java.util.Set; - +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; import org.apache.bookkeeper.client.AsyncCallback.ReadCallback; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookieProtocol; @@ -79,7 +82,7 @@ public LedgerFragmentReplicator(BookKeeper bkc) { private void replicateFragmentInternal(final LedgerHandle lh, final LedgerFragment lf, final AsyncCallback.VoidCallback ledgerFragmentMcb, - final BookieSocketAddress newBookie) throws InterruptedException { + final Set newBookies) throws InterruptedException { if (!lf.isClosed()) { LOG.error("Trying to replicate an unclosed fragment;" + " This is not safe {}", lf); @@ -94,13 +97,13 @@ private void replicateFragmentInternal(final LedgerHandle lh, * Ideally this should never happen if bookie failure is taken care * of properly. Nothing we can do though in this case. */ - LOG.warn("Dead bookie (" + lf.getAddress() + LOG.warn("Dead bookie (" + lf.getAddresses() + ") is still part of the current" + " active ensemble for ledgerId: " + lh.getId()); ledgerFragmentMcb.processResult(BKException.Code.OK, null, null); return; } - if (startEntryId > endEntryId) { + if (startEntryId > endEntryId || endEntryId <= INVALID_ENTRY_ID) { // for open ledger which there is no entry, the start entry id is 0, // the end entry id is -1. // we can return immediately to trigger forward read @@ -126,7 +129,7 @@ private void replicateFragmentInternal(final LedgerHandle lh, BKException.Code.LedgerRecoveryException); for (final Long entryId : entriesToReplicate) { recoverLedgerFragmentEntry(entryId, lh, ledgerFragmentEntryMcb, - newBookie); + newBookies); } } @@ -146,27 +149,27 @@ private void replicateFragmentInternal(final LedgerHandle lh, * @param ledgerFragmentMcb * MultiCallback to invoke once we've recovered the current * ledger fragment. - * @param targetBookieAddress - * New bookie we want to use to recover and replicate the ledger + * @param targetBookieAddresses + * New bookies we want to use to recover and replicate the ledger * entries that were stored on the failed bookie. */ void replicate(final LedgerHandle lh, final LedgerFragment lf, final AsyncCallback.VoidCallback ledgerFragmentMcb, - final BookieSocketAddress targetBookieAddress) + final Set targetBookieAddresses) throws InterruptedException { Set partionedFragments = splitIntoSubFragments(lh, lf, bkc.getConf().getRereplicationEntryBatchSize()); - LOG.info("Fragment :" + lf + " is split into sub fragments :" - + partionedFragments); + LOG.info("Replicating fragment {} in {} sub fragments.", + lf, partionedFragments.size()); replicateNextBatch(lh, partionedFragments.iterator(), - ledgerFragmentMcb, targetBookieAddress); + ledgerFragmentMcb, targetBookieAddresses); } /** Replicate the batched entry fragments one after other */ private void replicateNextBatch(final LedgerHandle lh, final Iterator fragments, final AsyncCallback.VoidCallback ledgerFragmentMcb, - final BookieSocketAddress targetBookieAddress) { + final Set targetBookieAddresses) { if (fragments.hasNext()) { try { replicateFragmentInternal(lh, fragments.next(), @@ -179,11 +182,11 @@ public void processResult(int rc, String v, Object ctx) { } else { replicateNextBatch(lh, fragments, ledgerFragmentMcb, - targetBookieAddress); + targetBookieAddresses); } } - }, targetBookieAddress); + }, targetBookieAddresses); } catch (InterruptedException e) { ledgerFragmentMcb.processResult( BKException.Code.InterruptedException, null, null); @@ -224,7 +227,7 @@ static Set splitIntoSubFragments(LedgerHandle lh, for (int i = 0; i < splitsWithFullEntries; i++) { fragmentSplitLastEntry = (firstEntryId + rereplicationEntryBatchSize) - 1; fragments.add(new LedgerFragment(lh, firstEntryId, - fragmentSplitLastEntry, ledgerFragment.getBookiesIndex())); + fragmentSplitLastEntry, ledgerFragment.getBookiesIndexes())); firstEntryId = fragmentSplitLastEntry + 1; } @@ -233,7 +236,7 @@ static Set splitIntoSubFragments(LedgerHandle lh, if (lastSplitWithPartialEntries > 0) { fragments.add(new LedgerFragment(lh, firstEntryId, firstEntryId + lastSplitWithPartialEntries - 1, ledgerFragment - .getBookiesIndex())); + .getBookiesIndexes())); } return fragments; } @@ -242,7 +245,7 @@ static Set splitIntoSubFragments(LedgerHandle lh, * This method asynchronously recovers a specific ledger entry by reading * the values via the BookKeeper Client (which would read it from the other * replicas) and then writing it to the chosen new bookie. - * + * * @param entryId * Ledger Entry ID to recover. * @param lh @@ -250,14 +253,41 @@ static Set splitIntoSubFragments(LedgerHandle lh, * @param ledgerFragmentEntryMcb * MultiCallback to invoke once we've recovered the current * ledger entry. - * @param newBookie - * New bookie we want to use to recover and replicate the ledger + * @param newBookies + * New bookies we want to use to recover and replicate the ledger * entries that were stored on the failed bookie. */ private void recoverLedgerFragmentEntry(final Long entryId, final LedgerHandle lh, final AsyncCallback.VoidCallback ledgerFragmentEntryMcb, - final BookieSocketAddress newBookie) throws InterruptedException { + final Set newBookies) throws InterruptedException { + final AtomicInteger numCompleted = new AtomicInteger(0); + final AtomicBoolean completed = new AtomicBoolean(false); + final WriteCallback multiWriteCallback = new WriteCallback() { + @Override + public void writeComplete(int rc, long ledgerId, long entryId, BookieSocketAddress addr, Object ctx) { + if (rc != BKException.Code.OK) { + LOG.error("BK error writing entry for ledgerId: {}, entryId: {}, bookie: {}", + new Object[] { ledgerId, entryId, addr, BKException.create(rc) }); + if (completed.compareAndSet(false, true)) { + ledgerFragmentEntryMcb.processResult(rc, null, null); + } + } else { + numEntriesWritten.inc(); + if (ctx instanceof Long) { + numBytesWritten.registerSuccessfulValue((Long) ctx); + } + if (LOG.isDebugEnabled()) { + LOG.debug("Success writing ledger id {}, entry id {} to a new bookie {}!", + new Object[] { ledgerId, entryId, addr }); + } + if (numCompleted.incrementAndGet() == newBookies.size() && + completed.compareAndSet(false, true)) { + ledgerFragmentEntryMcb.processResult(rc, null, null); + } + } + } + }; /* * Read the ledger entry using the LedgerHandle. This will allow us to * read the entry from one of the other replicated bookies other than @@ -286,42 +316,16 @@ public void readComplete(int rc, LedgerHandle lh, .computeDigestAndPackageForSending(entryId, lh.getLastAddConfirmed(), entry.getLength(), Unpooled.wrappedBuffer(data, 0, data.length)); - bkc.getBookieClient().addEntry(newBookie, lh.getId(), - lh.getLedgerKey(), entryId, toSend, - new WriteCallback() { - @Override - public void writeComplete(int rc, long ledgerId, - long entryId, BookieSocketAddress addr, - Object ctx) { - if (rc != BKException.Code.OK) { - LOG.error( - "BK error writing entry for ledgerId: " - + ledgerId + ", entryId: " - + entryId + ", bookie: " - + addr, BKException - .create(rc)); - } else { - numEntriesWritten.inc(); - numBytesWritten.registerSuccessfulValue(dataLength); - if (LOG.isDebugEnabled()) { - LOG.debug("Success writing ledger id " - + ledgerId + ", entry id " - + entryId + " to a new bookie " - + addr + "!"); - } - } - /* - * Pass the return code result up the chain with - * the parent callback. - */ - ledgerFragmentEntryMcb.processResult(rc, null, - null); - } - }, null, BookieProtocol.FLAG_RECOVERY_ADD); + for (BookieSocketAddress newBookie : newBookies) { + bkc.getBookieClient().addEntry(newBookie, lh.getId(), + lh.getLedgerKey(), entryId, toSend.retainedSlice(), + multiWriteCallback, dataLength, BookieProtocol.FLAG_RECOVERY_ADD); + } + toSend.release(); } }, null); } - + /** * Callback for recovery of a single ledger fragment. Once the fragment has * had all entries replicated, update the ensemble in zookeeper. Once @@ -332,17 +336,15 @@ static class SingleFragmentCallback implements AsyncCallback.VoidCallback { final AsyncCallback.VoidCallback ledgerFragmentsMcb; final LedgerHandle lh; final long fragmentStartId; - final BookieSocketAddress oldBookie; - final BookieSocketAddress newBookie; + final Map oldBookie2NewBookie; SingleFragmentCallback(AsyncCallback.VoidCallback ledgerFragmentsMcb, LedgerHandle lh, long fragmentStartId, - BookieSocketAddress oldBookie, BookieSocketAddress newBookie) { + Map oldBookie2NewBookie) { this.ledgerFragmentsMcb = ledgerFragmentsMcb; this.lh = lh; this.fragmentStartId = fragmentStartId; - this.newBookie = newBookie; - this.oldBookie = oldBookie; + this.oldBookie2NewBookie = oldBookie2NewBookie; } @Override @@ -353,39 +355,32 @@ public void processResult(int rc, String path, Object ctx) { ledgerFragmentsMcb.processResult(rc, null, null); return; } - updateEnsembleInfo(ledgerFragmentsMcb, fragmentStartId, lh, - oldBookie, newBookie); + updateEnsembleInfo(ledgerFragmentsMcb, fragmentStartId, lh, oldBookie2NewBookie); } } /** Updates the ensemble with newBookie and notify the ensembleUpdatedCb */ private static void updateEnsembleInfo( AsyncCallback.VoidCallback ensembleUpdatedCb, long fragmentStartId, - LedgerHandle lh, BookieSocketAddress oldBookie, - BookieSocketAddress newBookie) { + LedgerHandle lh, Map oldBookie2NewBookie) { /* * Update the ledger metadata's ensemble info to point to the new * bookie. */ ArrayList ensemble = lh.getLedgerMetadata() .getEnsembles().get(fragmentStartId); - int deadBookieIndex = ensemble.indexOf(oldBookie); - - /* - * An update to the ensemble info might happen after re-reading ledger metadata. - * Such an update might reflect a change to the ensemble membership such that - * it might not be necessary to replace the bookie. - */ - if (deadBookieIndex >= 0) { - ensemble.remove(deadBookieIndex); - ensemble.add(deadBookieIndex, newBookie); - lh.writeLedgerConfig(new UpdateEnsembleCb(ensembleUpdatedCb, - fragmentStartId, lh, oldBookie, newBookie)); - } - else { - LOG.warn("Bookie {} doesn't exist in ensemble {} anymore.", oldBookie, ensemble); - ensembleUpdatedCb.processResult(BKException.Code.UnexpectedConditionException, null, null); + for (Map.Entry entry : oldBookie2NewBookie.entrySet()) { + int deadBookieIndex = ensemble.indexOf(entry.getKey()); + // update ensemble info might happen after re-read ledger metadata, so the ensemble might already + // change. if ensemble is already changed, skip replacing the bookie doesn't exist. + if (deadBookieIndex >= 0) { + ensemble.set(deadBookieIndex, entry.getValue()); + } else { + LOG.info("Bookie {} doesn't exist in ensemble {} anymore.", entry.getKey(), ensemble); + } } + lh.writeLedgerConfig(new UpdateEnsembleCb(ensembleUpdatedCb, + fragmentStartId, lh, oldBookie2NewBookie)); } /** @@ -397,17 +392,15 @@ private static class UpdateEnsembleCb implements GenericCallback { final AsyncCallback.VoidCallback ensembleUpdatedCb; final LedgerHandle lh; final long fragmentStartId; - final BookieSocketAddress oldBookie; - final BookieSocketAddress newBookie; + final Map oldBookie2NewBookie; public UpdateEnsembleCb(AsyncCallback.VoidCallback ledgerFragmentsMcb, long fragmentStartId, LedgerHandle lh, - BookieSocketAddress oldBookie, BookieSocketAddress newBookie) { + Map oldBookie2NewBookie) { this.ensembleUpdatedCb = ledgerFragmentsMcb; this.lh = lh; this.fragmentStartId = fragmentStartId; - this.newBookie = newBookie; - this.oldBookie = oldBookie; + this.oldBookie2NewBookie = oldBookie2NewBookie; } @Override @@ -418,9 +411,8 @@ public void operationComplete(int rc, Void result) { // try again, the previous success (with which this has // conflicted) will have updated the stat other operations // such as (addEnsemble) would update it too. - lh - .rereadMetadata(new OrderedSafeGenericCallback( - lh.bk.getMainWorkerPool(), lh.getId()) { + lh.rereadMetadata(new OrderedSafeGenericCallback( + lh.bk.mainWorkerPool, lh.getId()) { @Override public void safeOperationComplete(int rc, LedgerMetadata newMeta) { @@ -433,8 +425,7 @@ public void safeOperationComplete(int rc, } else { lh.metadata = newMeta; updateEnsembleInfo(ensembleUpdatedCb, - fragmentStartId, lh, oldBookie, - newBookie); + fragmentStartId, lh, oldBookie2NewBookie); } } @Override @@ -449,8 +440,8 @@ public String toString() { } else { LOG.info("Updated ZK for ledgerId: (" + lh.getId() + " : " + fragmentStartId - + ") to point ledger fragments from old dead bookie: (" - + oldBookie + ") to new bookie: (" + newBookie + ")"); + + ") to point ledger fragments from old bookies to new bookies: " + + oldBookie2NewBookie); } /* * Pass the return code result up the chain with the parent diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ServerConfiguration.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ServerConfiguration.java index 10fb329b43c..787540e1638 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ServerConfiguration.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ServerConfiguration.java @@ -115,10 +115,13 @@ public class ServerConfiguration extends AbstractConfiguration { protected final static String DISK_USAGE_WARN_THRESHOLD = "diskUsageWarnThreshold"; protected final static String DISK_USAGE_LWM_THRESHOLD = "diskUsageLwmThreshold"; protected final static String DISK_CHECK_INTERVAL = "diskCheckInterval"; + + // Replication parameters protected final static String AUDITOR_PERIODIC_CHECK_INTERVAL = "auditorPeriodicCheckInterval"; protected final static String AUDITOR_PERIODIC_BOOKIE_CHECK_INTERVAL = "auditorPeriodicBookieCheckInterval"; protected final static String AUTO_RECOVERY_DAEMON_ENABLED = "autoRecoveryDaemonEnabled"; protected final static String LOST_BOOKIE_RECOVERY_DELAY = "lostBookieRecoveryDelay"; + protected final static String RW_REREPLICATE_BACKOFF_MS = "rwRereplicateBackoffMs"; // Worker Thread parameters. protected final static String NUM_ADD_WORKER_THREADS = "numAddWorkerThreads"; @@ -1759,6 +1762,24 @@ public void setLostBookieRecoveryDelay(int interval) { setProperty(LOST_BOOKIE_RECOVERY_DELAY, interval); } + /** + * Get how long to backoff when encountering exception on rereplicating a ledger. + * + * @return backoff time in milliseconds + */ + public int getRwRereplicateBackoffMs() { + return getInt(RW_REREPLICATE_BACKOFF_MS, 5000); + } + + /** + * Set how long to backoff when encountering exception on rereplicating a ledger. + * + * @param backoffMs backoff time in milliseconds + */ + public void setRwRereplicateBackoffMs(int backoffMs) { + setProperty(RW_REREPLICATE_BACKOFF_MS, backoffMs); + } + /** * Sets that whether force start a bookie in readonly mode * diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/Auditor.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/Auditor.java index ab35b416453..4f84d03a289 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/Auditor.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/Auditor.java @@ -20,6 +20,10 @@ */ package org.apache.bookkeeper.replication; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Stopwatch; +import com.google.common.collect.Sets; +import com.google.common.util.concurrent.SettableFuture; import java.io.IOException; import java.util.ArrayList; import java.util.Collection; @@ -33,7 +37,6 @@ import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; - import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.BookKeeperAdmin; @@ -63,11 +66,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Stopwatch; -import com.google.common.collect.Sets; -import com.google.common.util.concurrent.SettableFuture; - /** * Auditor is a single entity in the entire Bookie cluster and will be watching * all the bookies under 'ledgerrootpath/available' zkpath. When any of the @@ -570,7 +568,7 @@ public void operationComplete(int rc, Set fragments) { if (rc == BKException.Code.OK) { Set bookies = Sets.newHashSet(); for (LedgerFragment f : fragments) { - bookies.add(f.getAddress()); + bookies.addAll(f.getAddresses()); } for (BookieSocketAddress bookie : bookies) { publishSuspectedLedgers(bookie.toString(), Sets.newHashSet(lh.getId())); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AutoRecoveryMain.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AutoRecoveryMain.java index 9c3c019ebff..5caac6f0618 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AutoRecoveryMain.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AutoRecoveryMain.java @@ -20,14 +20,12 @@ */ package org.apache.bookkeeper.replication; +import com.google.common.annotations.VisibleForTesting; import java.io.File; import java.io.IOException; import java.net.MalformedURLException; import java.util.HashSet; import java.util.Set; - -import com.google.common.annotations.VisibleForTesting; - import org.apache.bookkeeper.bookie.Bookie; import org.apache.bookkeeper.bookie.BookieCriticalThread; import org.apache.bookkeeper.bookie.ExitCode; @@ -105,8 +103,7 @@ public void process(WatchedEvent event) { .build(); auditorElector = new AuditorElector(Bookie.getBookieAddress(conf).toString(), conf, zk, statsLogger.scope(AUDITOR_SCOPE)); - replicationWorker = new ReplicationWorker(zk, conf, - Bookie.getBookieAddress(conf), statsLogger.scope(REPLICATION_WORKER_SCOPE)); + replicationWorker = new ReplicationWorker(zk, conf, statsLogger.scope(REPLICATION_WORKER_SCOPE)); deathWatcher = new AutoRecoveryDeathWatcher(this); } @@ -115,7 +112,7 @@ public AutoRecoveryMain(ServerConfiguration conf, ZooKeeper zk) throws IOExcepti this.conf = conf; this.zk = zk; auditorElector = new AuditorElector(Bookie.getBookieAddress(conf).toString(), conf, zk); - replicationWorker = new ReplicationWorker(zk, conf, Bookie.getBookieAddress(conf)); + replicationWorker = new ReplicationWorker(zk, conf); deathWatcher = new AutoRecoveryDeathWatcher(this); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/ReplicationStats.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/ReplicationStats.java index fb7de209344..0b48e33e3b8 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/ReplicationStats.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/ReplicationStats.java @@ -43,6 +43,7 @@ public interface ReplicationStats { public final static String NUM_BYTES_READ = "NUM_BYTES_READ"; public final static String NUM_ENTRIES_WRITTEN = "NUM_ENTRIES_WRITTEN"; public final static String NUM_BYTES_WRITTEN = "NUM_BYTES_WRITTEN"; + public final static String REPLICATE_EXCEPTION = "exceptions"; public final static String BK_CLIENT_SCOPE = "bk_client"; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/ReplicationWorker.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/ReplicationWorker.java index 98960b02023..61205f764ff 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/ReplicationWorker.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/ReplicationWorker.java @@ -19,8 +19,10 @@ */ package org.apache.bookkeeper.replication; +import com.google.common.base.Stopwatch; import java.io.IOException; -import java.util.List; +import java.util.HashMap; +import java.util.Map; import java.util.Set; import java.util.Timer; import java.util.TimerTask; @@ -29,13 +31,12 @@ import java.util.Collection; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; - -import com.google.common.base.Stopwatch; - import org.apache.bookkeeper.bookie.BookieThread; import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.BKException.BKBookieHandleNotAvailableException; +import org.apache.bookkeeper.client.BKException.BKLedgerRecoveryException; import org.apache.bookkeeper.client.BKException.BKNoSuchLedgerExistsException; +import org.apache.bookkeeper.client.BKException.BKNotEnoughBookiesException; import org.apache.bookkeeper.client.BKException.BKReadException; import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.BookKeeperAdmin; @@ -61,6 +62,8 @@ import org.slf4j.LoggerFactory; import static org.apache.bookkeeper.replication.ReplicationStats.BK_CLIENT_SCOPE; +import static org.apache.bookkeeper.replication.ReplicationStats.NUM_FULL_OR_PARTIAL_LEDGERS_REPLICATED; +import static org.apache.bookkeeper.replication.ReplicationStats.REPLICATE_EXCEPTION; import static org.apache.bookkeeper.replication.ReplicationStats.REREPLICATE_OP; /** @@ -68,24 +71,25 @@ * ZKLedgerUnderreplicationManager and replicates to it. */ public class ReplicationWorker implements Runnable { - private final static Logger LOG = LoggerFactory + private static final Logger LOG = LoggerFactory .getLogger(ReplicationWorker.class); - final private LedgerUnderreplicationManager underreplicationManager; + private final LedgerUnderreplicationManager underreplicationManager; private final ServerConfiguration conf; private final ZooKeeper zkc; private volatile boolean workerRunning = false; - private volatile boolean isInReadOnlyMode = false; - final private BookKeeperAdmin admin; + private final BookKeeperAdmin admin; private final LedgerChecker ledgerChecker; - private final BookieSocketAddress targetBookie; private final BookKeeper bkc; private final Thread workerThread; + private final long rwRereplicateBackoffMs; private final long openLedgerRereplicationGracePeriod; private final Timer pendingReplicationTimer; // Expose Stats + private final StatsLogger statsLogger; private final OpStatsLogger rereplicateOpStats; private final Counter numLedgersReplicated; + private final Map exceptionCounters; /** * Replication worker for replicating the ledger fragments from @@ -96,15 +100,12 @@ public class ReplicationWorker implements Runnable { * - ZK instance * @param conf * - configurations - * @param targetBKAddr - * - to where replication should happen. Ideally this will be - * local Bookie address. */ public ReplicationWorker(final ZooKeeper zkc, - final ServerConfiguration conf, BookieSocketAddress targetBKAddr) + final ServerConfiguration conf) throws CompatibilityException, KeeperException, InterruptedException, IOException { - this(zkc, conf, targetBKAddr, NullStatsLogger.INSTANCE); + this(zkc, conf, NullStatsLogger.INSTANCE); } /** @@ -116,18 +117,16 @@ public ReplicationWorker(final ZooKeeper zkc, * - ZK instance * @param conf * - configurations - * @param targetBKAddr - * - to where replication should happen. Ideally this will be - * local Bookie address. + * @param statsLogger + * - stats logger */ public ReplicationWorker(final ZooKeeper zkc, - final ServerConfiguration conf, BookieSocketAddress targetBKAddr, + final ServerConfiguration conf, StatsLogger statsLogger) throws CompatibilityException, KeeperException, InterruptedException, IOException { this.zkc = zkc; this.conf = conf; - this.targetBookie = targetBKAddr; LedgerManagerFactory mFactory = LedgerManagerFactory .newLedgerManagerFactory(this.conf, this.zkc); this.underreplicationManager = mFactory @@ -141,11 +140,14 @@ public ReplicationWorker(final ZooKeeper zkc, this.workerThread = new BookieThread(this, "ReplicationWorker"); this.openLedgerRereplicationGracePeriod = conf .getOpenLedgerRereplicationGracePeriod(); + this.rwRereplicateBackoffMs = conf.getRwRereplicateBackoffMs(); this.pendingReplicationTimer = new Timer("PendingReplicationTimer"); // Expose Stats - this.rereplicateOpStats = statsLogger.getOpStatsLogger(REREPLICATE_OP); - this.numLedgersReplicated = statsLogger.getCounter(ReplicationStats.NUM_FULL_OR_PARTIAL_LEDGERS_REPLICATED); + this.statsLogger = statsLogger; + this.rereplicateOpStats = this.statsLogger.getOpStatsLogger(REREPLICATE_OP); + this.numLedgersReplicated = this.statsLogger.getCounter(NUM_FULL_OR_PARTIAL_LEDGERS_REPLICATED); + this.exceptionCounters = new HashMap(); } /** Start the replication worker */ @@ -167,38 +169,23 @@ public void run() { return; } catch (BKException e) { LOG.error("BKException while replicating fragments", e); - if (e instanceof BKException.BKWriteOnReadOnlyBookieException) { - waitTillTargetBookieIsWritable(); - } else { - waitBackOffTime(); - } + waitBackOffTime(rwRereplicateBackoffMs); } catch (UnavailableException e) { LOG.error("UnavailableException " + "while replicating fragments", e); - waitBackOffTime(); + waitBackOffTime(rwRereplicateBackoffMs); } } LOG.info("ReplicationWorker exited loop!"); } - private static void waitBackOffTime() { + private static void waitBackOffTime(long backoffMs) { try { - Thread.sleep(5000); + Thread.sleep(backoffMs); } catch (InterruptedException e) { } } - private void waitTillTargetBookieIsWritable() { - LOG.info("Waiting for target bookie {} to be back in read/write mode", targetBookie); - while (workerRunning && admin.getReadOnlyBookiesAsync().contains(targetBookie)) { - isInReadOnlyMode = true; - waitBackOffTime(); - } - - isInReadOnlyMode = false; - LOG.info("Target bookie {} is back in read/write mode", targetBookie); - } - /** * Replicates the under replicated fragments from failed bookie ledger to * targetBookie @@ -229,39 +216,22 @@ private boolean rereplicate(long ledgerIdToReplicate) throws InterruptedExceptio LOG.debug("Going to replicate the fragments of the ledger: {}", ledgerIdToReplicate); } try (LedgerHandle lh = admin.openLedgerNoRecovery(ledgerIdToReplicate)) { - Set fragments = getUnderreplicatedFragments(lh); + Set fragmentsBeforeReplicate = getUnderreplicatedFragments(lh); if (LOG.isDebugEnabled()) { - LOG.debug("Founds fragments {} for replication from ledger: {}", fragments, ledgerIdToReplicate); + LOG.debug("Founds fragments {} for replication from ledger: {}", + fragmentsBeforeReplicate, ledgerIdToReplicate); } boolean foundOpenFragments = false; long numFragsReplicated = 0; - for (LedgerFragment ledgerFragment : fragments) { + for (LedgerFragment ledgerFragment : fragmentsBeforeReplicate) { if (!ledgerFragment.isClosed()) { foundOpenFragments = true; continue; - } else if (isTargetBookieExistsInFragmentEnsemble(lh, - ledgerFragment)) { - if (LOG.isDebugEnabled()) { - LOG.debug("Target Bookie[{}] found in the fragment ensemble: {}", targetBookie, - ledgerFragment.getEnsemble()); - } - continue; - } - try { - admin.replicateLedgerFragment(lh, ledgerFragment, targetBookie); - numFragsReplicated++; - } catch (BKException.BKBookieHandleNotAvailableException e) { - LOG.warn("BKBookieHandleNotAvailableException " - + "while replicating the fragment", e); - } catch (BKException.BKLedgerRecoveryException e) { - LOG.warn("BKLedgerRecoveryException " - + "while replicating the fragment", e); - if (admin.getReadOnlyBookiesAsync().contains(targetBookie)) { - underreplicationManager.releaseUnderreplicatedLedger(ledgerIdToReplicate); - throw new BKException.BKWriteOnReadOnlyBookieException(); - } } + LOG.info("Going to replicate the fragments of the ledger: {}", ledgerIdToReplicate); + admin.replicateLedgerFragment(lh, ledgerFragment); + numFragsReplicated++; } if (numFragsReplicated > 0) { @@ -273,13 +243,13 @@ private boolean rereplicate(long ledgerIdToReplicate) throws InterruptedExceptio return false; } - fragments = getUnderreplicatedFragments(lh); - if (fragments.size() == 0) { - LOG.info("Ledger replicated successfully. ledger id is: " - + ledgerIdToReplicate); + Set fragmentsAfterReplicate = getUnderreplicatedFragments(lh); + if (fragmentsAfterReplicate.size() == 0) { + LOG.info("Ledger {} is replicated successfully.", ledgerIdToReplicate); underreplicationManager.markLedgerReplicated(ledgerIdToReplicate); return true; } else { + LOG.info("Fail to replicate ledger {}.", ledgerIdToReplicate); // Releasing the underReplication ledger lock and compete // for the replication again for the pending fragments underreplicationManager @@ -289,30 +259,34 @@ private boolean rereplicate(long ledgerIdToReplicate) throws InterruptedExceptio } catch (BKNoSuchLedgerExistsException e) { // Ledger might have been deleted by user LOG.info("BKNoSuchLedgerExistsException while opening " - + "ledger for replication. Other clients " + + "ledger {} for replication. Other clients " + "might have deleted the ledger. " - + "So, no harm to continue"); + + "So, no harm to continue", ledgerIdToReplicate); underreplicationManager.markLedgerReplicated(ledgerIdToReplicate); + getExceptionCounter("BKNoSuchLedgerExistsException").inc(); return false; - } catch (BKReadException e) { - LOG.info("BKReadException while" - + " opening ledger for replication." - + " Enough Bookies might not have available" - + "So, no harm to continue"); - underreplicationManager - .releaseUnderreplicatedLedger(ledgerIdToReplicate); - return false; - } catch (BKBookieHandleNotAvailableException e) { - LOG.info("BKBookieHandleNotAvailableException while" - + " opening ledger for replication." - + " Enough Bookies might not have available" - + "So, no harm to continue"); - underreplicationManager - .releaseUnderreplicatedLedger(ledgerIdToReplicate); + } catch (BKNotEnoughBookiesException e) { + logBKExceptionAndReleaseLedger(e, ledgerIdToReplicate); + throw e; + } catch (BKException e) { + logBKExceptionAndReleaseLedger(e, ledgerIdToReplicate); return false; } } + private void logBKExceptionAndReleaseLedger(BKException e, long ledgerIdToReplicate) + throws UnavailableException { + LOG.info("{} while" + + " rereplicating ledger {}." + + " Enough Bookies might not have available" + + " So, no harm to continue", + e.getClass().getSimpleName(), + ledgerIdToReplicate); + underreplicationManager + .releaseUnderreplicatedLedger(ledgerIdToReplicate); + getExceptionCounter(e.getClass().getSimpleName()).inc(); + } + /** * When checking the fragments of a ledger, there is a corner case * where if the last segment/ensemble is open, but nothing has been written to @@ -471,21 +445,6 @@ boolean isRunning() { return workerRunning && workerThread.isAlive(); } - boolean isInReadOnlyMode() { - return isInReadOnlyMode; - } - - private boolean isTargetBookieExistsInFragmentEnsemble(LedgerHandle lh, - LedgerFragment ledgerFragment) { - List ensemble = ledgerFragment.getEnsemble(); - for (BookieSocketAddress bkAddr : ensemble) { - if (targetBookie.equals(bkAddr)) { - return true; - } - } - return false; - } - /** Ledger checker call back */ private static class CheckerCallback implements GenericCallback> { @@ -508,4 +467,13 @@ Set waitAndGetResult() throws InterruptedException { } } + private Counter getExceptionCounter(String name) { + Counter counter = this.exceptionCounters.get(name); + if (counter == null) { + counter = this.statsLogger.scope(REPLICATE_EXCEPTION).getCounter(name); + this.exceptionCounters.put(name, counter); + } + return counter; + } + } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperCloseTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperCloseTest.java index 183af5bb037..f415152c462 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperCloseTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperCloseTest.java @@ -20,6 +20,14 @@ */ package org.apache.bookkeeper.client; +import com.google.common.util.concurrent.SettableFuture; +import io.netty.buffer.ByteBuf; +import java.io.IOException; +import java.util.Enumeration; +import java.util.Set; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import org.apache.bookkeeper.bookie.Bookie; import org.apache.bookkeeper.bookie.BookieException; import org.apache.bookkeeper.client.AsyncCallback.AddCallback; @@ -37,17 +45,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.util.concurrent.SettableFuture; - -import io.netty.buffer.ByteBuf; - -import java.io.IOException; -import java.util.Enumeration; -import java.util.Set; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; - import static org.junit.Assert.*; /** @@ -518,7 +515,7 @@ public void testBookKeeperAdmin() throws Exception { try { bkadmin.replicateLedgerFragment(lh3, - checkercb.getResult(10, TimeUnit.SECONDS).iterator().next(), newBookie); + checkercb.getResult(10, TimeUnit.SECONDS).iterator().next()); fail("Shouldn't be able to replicate with a closed client"); } catch (BKException.BKClientClosedException cce) { // correct behaviour diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieRecoveryTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieRecoveryTest.java index fc78d8f16ca..a9ccf89d128 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieRecoveryTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieRecoveryTest.java @@ -322,13 +322,9 @@ public void testAsyncBookieRecoveryToSpecificBookie() throws Exception { // Call the async recover bookie method. BookieSocketAddress bookieSrc = new BookieSocketAddress(InetAddress.getLocalHost().getHostAddress(), initialPort); - BookieSocketAddress bookieDest = new BookieSocketAddress(InetAddress.getLocalHost().getHostAddress(), - newBookiePort); - LOG.info("Now recover the data on the killed bookie (" + bookieSrc + ") and replicate it to the new one (" - + bookieDest + ")"); // Initiate the sync object sync.value = false; - bkAdmin.asyncRecoverBookieData(bookieSrc, bookieDest, bookieRecoverCb, sync); + bkAdmin.asyncRecoverBookieData(bookieSrc, bookieRecoverCb, sync); // Wait for the async method to complete. synchronized (sync) { @@ -380,12 +376,11 @@ public void testAsyncBookieRecoveryToRandomBookies() throws Exception { // Call the async recover bookie method. BookieSocketAddress bookieSrc = new BookieSocketAddress(InetAddress.getLocalHost().getHostAddress(), initialPort); - BookieSocketAddress bookieDest = null; LOG.info("Now recover the data on the killed bookie (" + bookieSrc + ") and replicate it to a random available one"); // Initiate the sync object sync.value = false; - bkAdmin.asyncRecoverBookieData(bookieSrc, bookieDest, bookieRecoverCb, sync); + bkAdmin.asyncRecoverBookieData(bookieSrc, bookieRecoverCb, sync); // Wait for the async method to complete. synchronized (sync) { diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestLedgerChecker.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestLedgerChecker.java index 83ec5938b80..3bcac7d2f2e 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestLedgerChecker.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestLedgerChecker.java @@ -91,8 +91,13 @@ public void testChecker() throws Exception { LOG.info("unreplicated fragment: {}", r); } assertEquals("Should have one missing fragment", 1, result.size()); - assertEquals("Fragment should be missing from first replica", result - .iterator().next().getAddress(), replicaToKill); + assertEquals("There should be 1 fragments. But returned fragments are " + + result, 1, result.size()); + LedgerFragment lf = result.iterator().next(); + assertEquals("There should be 1 failed bookies in first fragment " + lf, + 1, lf.getBookiesIndexes().size()); + assertEquals("Fragment should be missing from first replica", + lf.getAddress(0), replicaToKill); BookieSocketAddress replicaToKill2 = lh.getLedgerMetadata() .getEnsembles().get(0L).get(1); @@ -104,7 +109,16 @@ public void testChecker() throws Exception { for (LedgerFragment r : result) { LOG.info("unreplicated fragment: {}", r); } - assertEquals("Should have three missing fragments", 3, result.size()); + assertEquals("Should have two missing fragments", 2, result.size()); + for (LedgerFragment fragment : result) { + if (fragment.getFirstEntryId() == 0L) { + assertEquals("There should be 2 failed bookies in first fragment", + 2, fragment.getBookiesIndexes().size()); + } else { + assertEquals("There should be 1 failed bookies in second fragment", + 1, fragment.getBookiesIndexes().size()); + } + } } /** @@ -191,7 +205,9 @@ public void testShouldGetTwoFrgamentsIfTwoBookiesFailedInSameEnsemble() LOG.info("unreplicated fragment: {}", r); } - assertEquals("There should be 2 fragments", 2, result.size()); + assertEquals("There should be 1 fragments", 1, result.size()); + assertEquals("There should be 2 failed bookies in the fragment", + 2, result.iterator().next().getBookiesIndexes().size()); } /** @@ -264,7 +280,9 @@ public void testShouldGetFailedEnsembleNumberOfFgmntsIfEnsembleBookiesFailedOnNe LOG.info("unreplicated fragment: {}", r); } - assertEquals("There should be 3 fragments", 3, result.size()); + assertEquals("There should be 1 fragments", 1, result.size()); + assertEquals("There should be 3 failed bookies in the fragment", + 3, result.iterator().next().getBookiesIndexes().size()); } /** @@ -297,7 +315,9 @@ public void testShouldGet2FragmentsWithEmptyLedgerButBookiesDead() throws Except } Set result = getUnderReplicatedFragments(lh); assertNotNull("Result shouldn't be null", result); - assertEquals("There should be 2 fragments.", 2, result.size()); + assertEquals("There should be 1 fragments.", 1, result.size()); + assertEquals("There should be 2 failed bookies in the fragment", + 2, result.iterator().next().getBookiesIndexes().size()); } /** @@ -326,6 +346,8 @@ public void testShouldGetOneFragmentWithSingleEntryOpenedLedger() throws Excepti assertNotNull("Result shouldn't be null", result); assertEquals("There should be 1 fragment. But returned fragments are " + result, 1, result.size()); + assertEquals("There should be 1 failed bookies in the fragment", + 1, result.iterator().next().getBookiesIndexes().size()); } /** @@ -364,8 +386,17 @@ public void testSingleEntryAfterEnsembleChange() throws Exception { Set result = getUnderReplicatedFragments(lh1); assertNotNull("Result shouldn't be null", result); - assertEquals("There should be 3 fragment. But returned fragments are " - + result, 3, result.size()); + assertEquals("There should be 2 fragments. But returned fragments are " + + result, 2, result.size()); + for (LedgerFragment lf : result) { + if (lf.getFirstEntryId() == 0L) { + assertEquals("There should be 2 failed bookies in first fragment", + 2, lf.getBookiesIndexes().size()); + } else { + assertEquals("There should be 1 failed bookie in second fragment", + 1, lf.getBookiesIndexes().size()); + } + } } /** @@ -439,6 +470,8 @@ public void testClosedSingleEntryLedger() throws Exception { assertNotNull("Result shouldn't be null", result); assertEquals("There should be 1 fragment. But returned fragments are " + result, 1, result.size()); + assertEquals("There should be 1 failed bookies in the fragment", + 1, result.iterator().next().getBookiesIndexes().size()); lh1.close(); // kill bookie 0 @@ -454,8 +487,10 @@ public void testClosedSingleEntryLedger() throws Exception { result = getUnderReplicatedFragments(lh1); assertNotNull("Result shouldn't be null", result); - assertEquals("There should be 2 fragment. But returned fragments are " - + result, 2, result.size()); + assertEquals("There should be 1 fragment. But returned fragments are " + + result, 1, result.size()); + assertEquals("There should be 2 failed bookies in the fragment", + 2, result.iterator().next().getBookiesIndexes().size()); lh1.close(); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestLedgerFragmentReplication.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestLedgerFragmentReplication.java index c0ff8d7743e..e2d85e02187 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestLedgerFragmentReplication.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestLedgerFragmentReplication.java @@ -19,6 +19,7 @@ */ package org.apache.bookkeeper.client; +import com.google.common.collect.Sets; import java.net.InetAddress; import java.util.ArrayList; import java.util.Enumeration; @@ -26,7 +27,6 @@ import java.util.Set; import java.util.SortedMap; import java.util.concurrent.CountDownLatch; - import org.apache.bookkeeper.client.BookKeeper.DigestType; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; @@ -104,7 +104,7 @@ public void testReplicateLFShouldCopyFailedBookieFragmentsToTargetBookie() // 0-9 entries should be copy to new bookie for (LedgerFragment lf : result) { - admin.replicateLedgerFragment(lh, lf, newBkAddr); + admin.replicateLedgerFragment(lh, lf); } // Killing all bookies except newly replicated bookie @@ -170,11 +170,11 @@ public void testReplicateLFFailsOnlyOnLastUnClosedFragments() int unclosedCount = 0; for (LedgerFragment lf : result) { if (lf.isClosed()) { - admin.replicateLedgerFragment(lh, lf, newBkAddr); + admin.replicateLedgerFragment(lh, lf); } else { unclosedCount++; try { - admin.replicateLedgerFragment(lh, lf, newBkAddr); + admin.replicateLedgerFragment(lh, lf); fail("Shouldn't be able to rereplicate unclosed ledger"); } catch (BKException bke) { // correct behaviour @@ -216,12 +216,9 @@ public void testReplicateLFShouldReturnFalseIfTheReplicationFails() Set fragments = getFragmentsToReplicate(lh); BookKeeperAdmin admin = new BookKeeperAdmin(baseClientConf); - int startNewBookie = startNewBookie(); - BookieSocketAddress additionalBK = new BookieSocketAddress(InetAddress - .getLocalHost().getHostAddress(), startNewBookie); for (LedgerFragment lf : fragments) { try { - admin.replicateLedgerFragment(lh, lf, additionalBK); + admin.replicateLedgerFragment(lh, lf); } catch (BKException.BKLedgerRecoveryException e) { // expected } @@ -265,7 +262,7 @@ void testSplitIntoSubFragments(final long oriFragmentFirstEntry, final long oriFragmentLastEntry, long entriesPerSubFragment, long expectedSubFragments, LedgerHandle lh) { LedgerFragment fr = new LedgerFragment(lh, oriFragmentFirstEntry, - oriFragmentLastEntry, 0) { + oriFragmentLastEntry, Sets.newHashSet(0)) { @Override public long getLastStoredEntryId() { return oriFragmentLastEntry; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/BookieAutoRecoveryTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/BookieAutoRecoveryTest.java index d273dabb994..2ce591b46a6 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/BookieAutoRecoveryTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/BookieAutoRecoveryTest.java @@ -79,6 +79,7 @@ public BookieAutoRecoveryTest() throws IOException, KeeperException, baseConf.setLedgerManagerFactoryClassName( "org.apache.bookkeeper.meta.HierarchicalLedgerManagerFactory"); baseConf.setOpenLedgerRereplicationGracePeriod(openLedgerRereplicationGracePeriod); + baseConf.setRwRereplicateBackoffMs(500); baseClientConf.setLedgerManagerFactoryClassName( "org.apache.bookkeeper.meta.HierarchicalLedgerManagerFactory"); this.digestType = DigestType.MAC; @@ -550,8 +551,7 @@ public void testLedgerMetadataContainsHostNameAsBookieID() } - private int getReplicaIndexInLedger(LedgerHandle lh, - BookieSocketAddress replicaToKill) { + private int getReplicaIndexInLedger(LedgerHandle lh, BookieSocketAddress replicaToKill) { SortedMap> ensembles = LedgerHandleAdapter .getLedgerMetadata(lh).getEnsembles(); int ledgerReplicaIndex = -1; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestReplicationWorker.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestReplicationWorker.java index 0ec3bb0f8d1..e873f157e66 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestReplicationWorker.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestReplicationWorker.java @@ -22,7 +22,6 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; -import static org.junit.Assert.assertNull; import java.net.InetAddress; import java.util.ArrayList; @@ -39,7 +38,6 @@ import org.apache.bookkeeper.meta.LedgerManagerFactory; import org.apache.bookkeeper.meta.LedgerUnderreplicationManager; import org.apache.bookkeeper.net.BookieSocketAddress; -import org.apache.bookkeeper.proto.BookieServer; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.apache.bookkeeper.util.BookKeeperConstants; import org.apache.bookkeeper.zookeeper.ZooKeeperClient; @@ -132,7 +130,7 @@ public void testRWShouldReplicateFragmentsToTargetBookie() throws Exception { .getLocalHost().getHostAddress(), startNewBookie); LOG.info("New Bookie addr :" + newBkAddr); - ReplicationWorker rw = new ReplicationWorker(zkc, baseConf, newBkAddr); + ReplicationWorker rw = new ReplicationWorker(zkc, baseConf); rw.start(); try { @@ -179,7 +177,7 @@ public void testRWShouldRetryUntilThereAreEnoughBksAvailableForReplication() LOG.info("New Bookie addr :" + newBkAddr); killAllBookies(lh, newBkAddr); - ReplicationWorker rw = new ReplicationWorker(zkc, baseConf, newBkAddr); + ReplicationWorker rw = new ReplicationWorker(zkc, baseConf); rw.start(); try { @@ -231,7 +229,7 @@ public void test2RWsShouldCompeteForReplicationOf2FragmentsAndCompleteReplicatio BookieSocketAddress newBkAddr1 = new BookieSocketAddress(InetAddress .getLocalHost().getHostAddress(), startNewBookie1); LOG.info("New Bookie addr :" + newBkAddr1); - ReplicationWorker rw1 = new ReplicationWorker(zkc, baseConf, newBkAddr1); + ReplicationWorker rw1 = new ReplicationWorker(zkc, baseConf); // Starte RW2 int startNewBookie2 = startNewBookie(); @@ -242,8 +240,7 @@ public void test2RWsShouldCompeteForReplicationOf2FragmentsAndCompleteReplicatio .connectString(zkUtil.getZooKeeperConnectString()) .sessionTimeoutMs(10000) .build(); - ReplicationWorker rw2 = new ReplicationWorker(zkc1, baseConf, - newBkAddr2); + ReplicationWorker rw2 = new ReplicationWorker(zkc1, baseConf); rw1.start(); rw2.start(); @@ -296,7 +293,7 @@ public void testRWShouldCleanTheLedgerFromUnderReplicationIfLedgerAlreadyDeleted BookieSocketAddress newBkAddr = new BookieSocketAddress(InetAddress .getLocalHost().getHostAddress(), startNewBookie); LOG.info("New Bookie addr :" + newBkAddr); - ReplicationWorker rw = new ReplicationWorker(zkc, baseConf, newBkAddr); + ReplicationWorker rw = new ReplicationWorker(zkc, baseConf); rw.start(); try { @@ -354,7 +351,7 @@ public void testMultipleLedgerReplicationWithReplicationWorker() .getLocalHost().getHostAddress(), startNewBookie); LOG.info("New Bookie addr :" + newBkAddr); - ReplicationWorker rw = new ReplicationWorker(zkc, baseConf, newBkAddr); + ReplicationWorker rw = new ReplicationWorker(zkc, baseConf); rw.start(); try { @@ -413,7 +410,7 @@ public void testRWShouldReplicateTheLedgersAfterTimeoutIfLastFragmentIsUR() // set to 3s instead of default 30s baseConf.setOpenLedgerRereplicationGracePeriod("3000"); - ReplicationWorker rw = new ReplicationWorker(zkc, baseConf, newBkAddr); + ReplicationWorker rw = new ReplicationWorker(zkc, baseConf); LedgerManagerFactory mFactory = LedgerManagerFactory .newLedgerManagerFactory(baseClientConf, zkc); @@ -474,7 +471,7 @@ public void testRWShouldReplicateTheLedgersAfterTimeoutIfLastFragmentIsNotUR() .getLocalHost().getHostAddress(), startNewBookie); LOG.info("New Bookie addr :" + newBkAddr); - ReplicationWorker rw = new ReplicationWorker(zkc, baseConf, newBkAddr); + ReplicationWorker rw = new ReplicationWorker(zkc, baseConf); LedgerManagerFactory mFactory = LedgerManagerFactory .newLedgerManagerFactory(baseClientConf, zkc); @@ -508,49 +505,6 @@ public void testRWShouldReplicateTheLedgersAfterTimeoutIfLastFragmentIsNotUR() } - /** - * Test that if the local bookie turns out to be read-only, then the replicator will pause but not shutdown. - */ - @Test - public void testRWOnLocalBookieReadonlyTransition() throws Exception { - LedgerHandle lh = bkc.createLedger(3, 3, BookKeeper.DigestType.CRC32, TESTPASSWD); - - for (int i = 0; i < 10; i++) { - lh.addEntry(data); - } - BookieSocketAddress replicaToKill = - LedgerHandleAdapter.getLedgerMetadata(lh).getEnsembles().get(0L).get(0); - - LOG.info("Killing Bookie", replicaToKill); - killBookie(replicaToKill); - - int newBkPort = startNewBookie(); - for (int i = 0; i < 10; i++) { - lh.addEntry(data); - } - - BookieSocketAddress newBkAddr = new BookieSocketAddress(InetAddress.getLocalHost().getHostAddress(), newBkPort); - LOG.info("New Bookie addr :" + newBkAddr); - - ReplicationWorker rw = new ReplicationWorker(zkc, baseConf, newBkAddr); - - rw.start(); - try { - BookieServer newBk = bs.get(bs.size() - 1); - bsConfs.get(bsConfs.size() - 1).setReadOnlyModeEnabled(true); - newBk.getBookie().doTransitionToReadOnlyMode(); - underReplicationManager.markLedgerUnderreplicated(lh.getId(), replicaToKill.toString()); - while (ReplicationTestUtil.isLedgerInUnderReplication(zkc, lh.getId(), basePath) && rw.isRunning() - && !rw.isInReadOnlyMode()) { - Thread.sleep(100); - } - assertNull(zkc.exists(String.format("%s/urL%010d", baseLockPath, lh.getId()), false)); - assertTrue("RW should continue even if the bookie is readonly", rw.isRunning()); - } finally { - rw.shutdown(); - } - } - /** * Test that the replication worker will not shutdown on a simple ZK disconnection */ @@ -562,7 +516,7 @@ public void testRWZKConnectionLost() throws Exception { .build(); try { - ReplicationWorker rw = new ReplicationWorker(zk, baseConf, getBookie(0)); + ReplicationWorker rw = new ReplicationWorker(zk, baseConf); rw.start(); for (int i = 0; i < 10; i++) { if (rw.isRunning()) { From 3c493223ea2ebd43b1a871c66ca36d91bfb79fdf Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Fri, 27 Oct 2017 16:13:10 -0700 Subject: [PATCH 06/15] ISSUE #265: Add persistable bookie status Descriptions of the changes in this PR: - Add support for persisting bookie status - Add configuration to enable/disable this feature - Add test cases - Improve bookie status stat It also includes changes for BOOKKEEPER-754 (flush ledger storage after replaying journal) Author: Sijie Guo Author: Yiming Zang Reviewers: Yiming Zang , Enrico Olivelli This closes #266 from sijie/server_side_crc32, closes #265 --- bookkeeper-server/conf/bk_server.conf | 4 + .../org/apache/bookkeeper/bookie/Bookie.java | 60 ++++- .../bookkeeper/bookie/BookieStatus.java | 242 ++++++++++++++++++ .../bookkeeper/bookie/ReadOnlyBookie.java | 4 +- .../apache/bookkeeper/bookie/SyncThread.java | 73 ++++-- .../bookkeeper/conf/ServerConfiguration.java | 25 ++ .../apache/bookkeeper/proto/BookieServer.java | 21 -- .../org/apache/bookkeeper/server/Main.java | 4 +- .../bookkeeper/util/BookKeeperConstants.java | 1 + .../bookie/BookieInitializationTest.java | 187 +++++++++++++- site/_data/config/bk_server.yaml | 3 +- 11 files changed, 561 insertions(+), 63 deletions(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieStatus.java diff --git a/bookkeeper-server/conf/bk_server.conf b/bookkeeper-server/conf/bk_server.conf index 98dcbcded06..11cbff27108 100755 --- a/bookkeeper-server/conf/bk_server.conf +++ b/bookkeeper-server/conf/bk_server.conf @@ -485,6 +485,10 @@ ledgerManagerFactoryClass=org.apache.bookkeeper.meta.HierarchicalLedgerManagerFa # Whether the bookie is force started in read only mode or not # forceReadOnlyBookie=false +# Persiste the bookie status locally on the disks. So the bookies can keep their status upon restarts +# @Since 4.6 +# persistBookieStatusEnabled=false + ############################################################################# ## Disk utilization ############################################################################# diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Bookie.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Bookie.java index 953fb17d301..83888c173bd 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Bookie.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Bookie.java @@ -58,6 +58,7 @@ import java.util.Observer; import java.util.Set; import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; @@ -134,6 +135,8 @@ public class Bookie extends BookieCriticalThread { private volatile boolean running = false; // Flag identify whether it is in shutting down progress private volatile boolean shuttingdown = false; + // Bookie status + private final BookieStatus bookieStatus = new BookieStatus(); private int exitCode = ExitCode.OK; @@ -142,7 +145,7 @@ public class Bookie extends BookieCriticalThread { protected final String bookieId; private final AtomicBoolean rmRegistered = new AtomicBoolean(false); - protected final AtomicBoolean readOnly = new AtomicBoolean(false); + protected final AtomicBoolean forceReadOnly = new AtomicBoolean(false); // executor to manage the state changes for a bookie. final ExecutorService stateService = Executors.newSingleThreadExecutor( new ThreadFactoryBuilder().setNameFormat("BookieStateService-%d").build()); @@ -728,7 +731,13 @@ public Number getDefaultValue() { @Override public Number getSample() { - return rmRegistered.get() ? (readOnly.get() ? 0 : 1) : -1; + if (!rmRegistered.get()){ + return -1; + } else if (forceReadOnly.get() || bookieStatus.isInReadOnlyMode()) { + return 0; + } else { + return 1; + } } }); } @@ -812,6 +821,12 @@ public synchronized void start() { if (indexDirsManager != ledgerDirsManager) { idxMonitor.start(); } + + // start sync thread first, so during replaying journals, we could do checkpoint + // which reduce the chance that we need to replay journals again if bookie restarted + // again before finished journal replays. + syncThread.start(); + // replay journals try { readJournal(); @@ -824,7 +839,19 @@ public synchronized void start() { shutdown(ExitCode.BOOKIE_EXCEPTION); return; } + + // Do a fully flush after journal replay + try { + syncThread.requestFlush().get(); + } catch (InterruptedException e) { + LOG.warn("Interrupting the fully flush after replaying journals : ", e); + Thread.currentThread().interrupt(); + } catch (ExecutionException e) { + LOG.error("Error on executing a fully flush after replaying journals."); + shutdown(ExitCode.BOOKIE_EXCEPTION); + } LOG.info("Finished reading journal, starting bookie"); + // start bookie thread super.start(); @@ -837,7 +864,13 @@ public synchronized void start() { ledgerStorage.start(); - syncThread.start(); + // check the bookie status to start with + if (forceReadOnly.get()) { + this.bookieStatus.setToReadOnlyMode(); + } else if (conf.isPersistBookieStatusEnabled()) { + this.bookieStatus.readFromDirectories(ledgerDirsManager.getAllLedgerDirs()); + } + // set running here. // since bookie server use running as a flag to tell bookie server whether it is alive // if setting it in bookie thread, the watcher might run before bookie thread. @@ -942,7 +975,7 @@ public Void call() throws IOException { } protected void doRegisterBookie() throws IOException { - doRegisterBookie(readOnly.get()); + doRegisterBookie(forceReadOnly.get() || bookieStatus.isInReadOnlyMode()); } private void doRegisterBookie(boolean isReadOnly) throws IOException { @@ -979,13 +1012,18 @@ public Void call() throws Exception { @VisibleForTesting public void doTransitionToWritableMode() { - if (shuttingdown) { + if (shuttingdown || forceReadOnly.get()) { return; } - if (!readOnly.compareAndSet(true, false)) { + + if (!bookieStatus.setToWritableMode()) { + // do nothing if already in writable mode return; } LOG.info("Transitioning Bookie to Writable mode and will serve read/write requests."); + if (conf.isPersistBookieStatusEnabled()) { + bookieStatus.writeToDirectories(ledgerDirsManager.getAllLedgerDirs()); + } // change zookeeper state only when using zookeeper if (null == registrationManager) { return; @@ -1026,7 +1064,7 @@ public void doTransitionToReadOnlyMode() { if (shuttingdown) { return; } - if (!readOnly.compareAndSet(false, true)) { + if (!bookieStatus.setToReadOnlyMode()) { return; } if (!conf.isReadOnlyModeEnabled()) { @@ -1039,6 +1077,10 @@ public void doTransitionToReadOnlyMode() { } LOG.info("Transitioning Bookie to ReadOnly mode," + " and will serve only read requests from clients!"); + // persist the bookie status if we enable this + if (conf.isPersistBookieStatusEnabled()) { + this.bookieStatus.writeToDirectories(ledgerDirsManager.getAllLedgerDirs()); + } // change zookeeper state only when using zookeeper if (null == registrationManager) { return; @@ -1057,7 +1099,7 @@ public void doTransitionToReadOnlyMode() { * Check whether Bookie is writable */ public boolean isReadOnly() { - return readOnly.get(); + return forceReadOnly.get() || bookieStatus.isInReadOnlyMode(); } public boolean isRunning() { @@ -1133,7 +1175,7 @@ synchronized int shutdown(int exitCode) { // turn bookie to read only during shutting down process LOG.info("Turning bookie to read only during shut down"); - this.readOnly.set(true); + this.forceReadOnly.set(true); // Shutdown Sync thread syncThread.shutdown(); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieStatus.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieStatus.java new file mode 100644 index 00000000000..560868aa349 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieStatus.java @@ -0,0 +1,242 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.bookkeeper.bookie; + +import static com.google.common.base.Charsets.UTF_8; +import static org.apache.bookkeeper.util.BookKeeperConstants.BOOKIE_STATUS_FILENAME; + +import java.io.BufferedReader; +import java.io.BufferedWriter; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.InputStreamReader; +import java.io.OutputStreamWriter; +import java.util.List; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * The status object represents the current status of a bookie instance. + */ +public class BookieStatus { + + private static final Logger LOG = LoggerFactory.getLogger(BookieStatus.class); + + static final int CURRENT_STATUS_LAYOUT_VERSION = 1; + + enum BookieMode { + READ_ONLY, + READ_WRITE; + } + + private final static long INVALID_UPDATE_TIME = -1; + + private int layoutVersion; + private long lastUpdateTime; + private BookieMode bookieMode; + + + BookieStatus() { + this.bookieMode = BookieMode.READ_WRITE; + this.layoutVersion = CURRENT_STATUS_LAYOUT_VERSION; + this.lastUpdateTime = INVALID_UPDATE_TIME; + } + + private synchronized BookieMode getBookieMode() { + return bookieMode; + } + + public synchronized boolean isInWritable() { + return bookieMode.equals(BookieMode.READ_WRITE); + } + + synchronized boolean setToWritableMode() { + if (!bookieMode.equals(BookieMode.READ_WRITE)) { + bookieMode = BookieMode.READ_WRITE; + this.lastUpdateTime = System.currentTimeMillis(); + return true; + } + return false; + } + + synchronized boolean isInReadOnlyMode() { + return bookieMode.equals(BookieMode.READ_ONLY); + } + + synchronized boolean setToReadOnlyMode() { + if (!bookieMode.equals(BookieMode.READ_ONLY)) { + bookieMode = BookieMode.READ_ONLY; + this.lastUpdateTime = System.currentTimeMillis(); + return true; + } + return false; + } + + /** + * Write bookie status to multiple directories in best effort + * + * @param directories list of directories to write to + * + */ + synchronized void writeToDirectories(List directories) { + boolean success = false; + for (File dir : directories) { + try { + File statusFile = new File(dir, BOOKIE_STATUS_FILENAME); + writeToFile(statusFile, toString()); + success = true; + } catch (IOException e) { + LOG.warn("IOException while trying to write bookie status to directory {}." + + " This is fine if not all directories are failed.", dir); + } + } + if(success){ + LOG.info("Successfully persist bookie status {}", this.bookieMode); + } else { + LOG.warn("Failed to persist bookie status {}", this.bookieMode); + } + } + + /** + * Write content to the file. If file does not exist, it will create one. + * + * @param file file that you want to write to + * @param body content to write + * @throws IOException + */ + private static void writeToFile(File file, String body) throws IOException { + FileOutputStream fos = new FileOutputStream(file); + BufferedWriter bw = null; + try { + bw = new BufferedWriter(new OutputStreamWriter(fos, UTF_8)); + bw.write(body); + } finally { + if (bw != null) { + bw.close(); + } + fos.close(); + } + } + + /** + * Read bookie status from the status files, and update the bookie status if read succeed. + * If a status file is not readable or not found, it will skip and try to read from the next file. + * + * @param directories list of directories that store the status file + */ + void readFromDirectories(List directories) { + boolean success = false; + for (File dir : directories) { + File statusFile = new File(dir, BOOKIE_STATUS_FILENAME); + try { + BookieStatus status = readFromFile(statusFile); + if (null != status) { + synchronized (status) { + if (status.lastUpdateTime > this.lastUpdateTime) { + this.lastUpdateTime = status.lastUpdateTime; + this.layoutVersion = status.layoutVersion; + this.bookieMode = status.bookieMode; + success = true; + } + } + } + } catch (IOException e) { + LOG.warn("IOException while trying to read bookie status from directory {}." + + " This is fine if not all directories failed.", dir); + } catch (IllegalArgumentException e ){ + LOG.warn("IllegalArgumentException while trying to read bookie status from directory {}." + + " This is fine if not all directories failed.", dir); + } + } + if (success) { + LOG.info("Successfully retrieve bookie status {} from disks.", getBookieMode()); + } else { + LOG.warn("Failed to retrieve bookie status from disks." + + " Fall back to current or default bookie status: {}", getBookieMode()); + } + } + + + /** + * Function to read the bookie status from a single file + * + * @param file file to read from + * @return BookieStatus if not error, null if file not exist or any exception happens + * @throws IOException + */ + private BookieStatus readFromFile(File file) + throws IOException, IllegalArgumentException { + if (!file.exists()) { + return null; + } + + try (BufferedReader reader = new BufferedReader( + new InputStreamReader(new FileInputStream(file), UTF_8))) { + return parse(reader); + } + } + + /** + * Parse the bookie status object using appropriate layout version + * + * @param reader + * @return BookieStatus if parse succeed, otherwise return null + * @throws IOException + */ + public BookieStatus parse(BufferedReader reader) + throws IOException, IllegalArgumentException { + BookieStatus status = new BookieStatus(); + String line = reader.readLine(); + if (line == null || line.trim().isEmpty()) { + LOG.debug("Empty line when parsing bookie status"); + return null; + } + String[] parts = line.split(","); + if (parts.length == 0) { + LOG.debug("Error in parsing bookie status: {}", line); + return null; + } + synchronized (status) { + status.layoutVersion = Integer.parseInt(parts[0].trim()); + if (status.layoutVersion == 1 && parts.length == 3) { + status.bookieMode = BookieMode.valueOf(parts[1]); + status.lastUpdateTime = Long.parseLong(parts[2].trim()); + return status; + } + } + return null; + + } + + @Override + public String toString() { + StringBuilder builder = new StringBuilder(); + builder.append(CURRENT_STATUS_LAYOUT_VERSION); + builder.append(","); + builder.append(getBookieMode()); + builder.append(","); + builder.append(System.currentTimeMillis()); + builder.append("\n"); + return builder.toString(); + } + +} diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ReadOnlyBookie.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ReadOnlyBookie.java index 75361209af6..3970a1d6c99 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ReadOnlyBookie.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ReadOnlyBookie.java @@ -43,13 +43,13 @@ public ReadOnlyBookie(ServerConfiguration conf, StatsLogger statsLogger) throws IOException, KeeperException, InterruptedException, BookieException { super(conf, statsLogger); if (conf.isReadOnlyModeEnabled()) { - readOnly.set(true); + forceReadOnly.set(true); } else { String err = "Try to init ReadOnly Bookie, while ReadOnly mode is not enabled"; LOG.error(err); throw new IOException(err); } - LOG.info("Running bookie in readonly mode."); + LOG.info("Running bookie in force readonly mode."); } @Override diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/SyncThread.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/SyncThread.java index 413c4874546..ca001a462f1 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/SyncThread.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/SyncThread.java @@ -24,7 +24,9 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.util.concurrent.ThreadFactoryBuilder; import java.io.IOException; +import java.util.concurrent.Callable; import java.util.concurrent.Executors; +import java.util.concurrent.Future; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import org.apache.bookkeeper.bookie.CheckpointSource.Checkpoint; @@ -61,6 +63,10 @@ class SyncThread { final LedgerDirsListener dirsListener; final CheckpointSource checkpointSource; + private final Object suspensionLock = new Object(); + private boolean suspended = false; + private boolean disableCheckpoint = false; + public SyncThread(ServerConfiguration conf, LedgerDirsListener dirsListener, LedgerStorage ledgerStorage, @@ -79,25 +85,41 @@ public SyncThread(ServerConfiguration conf, void start() { executor.scheduleAtFixedRate(new Runnable() { - public void run() { - try { - synchronized (suspensionLock) { - while (suspended) { - try { - suspensionLock.wait(); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - continue; - } + public void run() { + try { + synchronized (suspensionLock) { + while (suspended) { + try { + suspensionLock.wait(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + continue; } } + } + if (!disableCheckpoint) { checkpoint(checkpointSource.newCheckpoint()); - } catch (Throwable t) { - LOG.error("Exception in SyncThread", t); - dirsListener.fatalError(); } + } catch (Throwable t) { + LOG.error("Exception in SyncThread", t); + dirsListener.fatalError(); + } + } + }, flushInterval, flushInterval, TimeUnit.MILLISECONDS); + } + + public Future requestFlush() { + return executor.submit(new Callable() { + @Override + public Void call() throws Exception { + try { + flush(); + } catch (Throwable t) { + LOG.error("Exception flushing ledgers ", t); } - }, flushInterval, flushInterval, TimeUnit.MILLISECONDS); + return null; + } + }); } private void flush() { @@ -113,6 +135,11 @@ private void flush() { return; } + if (disableCheckpoint) { + return; + } + + LOG.info("Flush ledger storage at checkpoint {}.", checkpoint); try { checkpointSource.checkpointComplete(checkpoint, false); } catch (IOException e) { @@ -142,9 +169,6 @@ public void checkpoint(Checkpoint checkpoint) { } } - private Object suspensionLock = new Object(); - private boolean suspended = false; - /** * Suspend sync thread. (for testing) */ @@ -166,18 +190,15 @@ public void resumeSync() { } } + @VisibleForTesting + public void disableCheckpoint() { + disableCheckpoint = true; + } + // shutdown sync thread void shutdown() throws InterruptedException { LOG.info("Shutting down SyncThread"); - executor.submit(new Runnable() { - public void run() { - try { - flush(); - } catch (Throwable t) { - LOG.error("Exception flushing ledgers at shutdown", t); - } - } - }); + requestFlush(); executor.shutdown(); long start = MathUtils.now(); while (!executor.awaitTermination(5, TimeUnit.MINUTES)) { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ServerConfiguration.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ServerConfiguration.java index 787540e1638..cbf88ad58f1 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ServerConfiguration.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ServerConfiguration.java @@ -110,6 +110,8 @@ public class ServerConfiguration extends AbstractConfiguration { protected final static String READ_ONLY_MODE_ENABLED = "readOnlyModeEnabled"; //Whether the bookie is force started in ReadOnly mode protected final static String FORCE_READ_ONLY_BOOKIE = "forceReadOnlyBookie"; + //Whether to persist the bookie status + protected final static String PERSIST_BOOKIE_STATUS_ENABLED = "persistBookieStatusEnabled"; //Disk utilization protected final static String DISK_USAGE_THRESHOLD = "diskUsageThreshold"; protected final static String DISK_USAGE_WARN_THRESHOLD = "diskUsageWarnThreshold"; @@ -1607,6 +1609,29 @@ public float getDiskUsageWarnThreshold() { return getFloat(DISK_USAGE_WARN_THRESHOLD, 0.90f); } + /** + * Whether to persist the bookie status so that when bookie server restarts, + * it will continue using the previous status + * + * @param enabled + * - true if persist the bookie status. Otherwise false. + * @return ServerConfiguration + */ + public ServerConfiguration setPersistBookieStatusEnabled(boolean enabled) { + setProperty(PERSIST_BOOKIE_STATUS_ENABLED, enabled); + return this; + } + + /** + * Get whether to persist the bookie status so that when bookie server restarts, + * it will continue using the previous status. + * + * @return true - if need to start a bookie in read only mode. Otherwise false. + */ + public boolean isPersistBookieStatusEnabled() { + return getBoolean(PERSIST_BOOKIE_STATUS_ENABLED, false); + } + /** * Set the Disk free space threshold as a fraction of the total * after which disk will be considered as full during disk check. diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieServer.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieServer.java index 5e80f15b3dc..00c41bcc9fb 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieServer.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieServer.java @@ -32,9 +32,6 @@ import org.apache.bookkeeper.bookie.ExitCode; import org.apache.bookkeeper.bookie.ReadOnlyBookie; import org.apache.bookkeeper.conf.ServerConfiguration; -import org.apache.bookkeeper.http.BKHttpServiceProvider; -import org.apache.bookkeeper.http.HttpServer; -import org.apache.bookkeeper.http.HttpServerLoader; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.processor.RequestProcessor; import org.apache.bookkeeper.replication.ReplicationException.CompatibilityException; @@ -63,9 +60,6 @@ public class BookieServer { int exitCode = ExitCode.OK; - // operation stats - HttpServer httpServer = null; - // request processor private final RequestProcessor requestProcessor; @@ -114,18 +108,6 @@ public void start() throws IOException, UnavailableException, InterruptedExcepti exitCode = bookie.getExitCode(); return; } - if (conf.isHttpServerEnabled()) { - BKHttpServiceProvider serviceProvider = new BKHttpServiceProvider.Builder() - .setBookieServer(this) - .setServerConfiguration(conf) - .build(); - HttpServerLoader.loadHttpServer(conf); - this.httpServer = HttpServerLoader.get(); - if (this.httpServer != null) { - this.httpServer.initialize(serviceProvider); - this.httpServer.startServer(conf.getHttpServerPort()); - } - } this.nettyServer.start(); running = true; @@ -173,9 +155,6 @@ public synchronized void shutdown() { } exitCode = bookie.shutdown(); this.requestProcessor.close(); - if (this.httpServer != null && this.httpServer.isRunning()) { - this.httpServer.stopServer(); - } running = false; } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/Main.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/Main.java index 99a93aeafed..9ba9d719873 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/Main.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/Main.java @@ -62,7 +62,7 @@ public class Main { BK_OPTS.addOption("c", "conf", true, "Configuration for Bookie Server"); BK_OPTS.addOption("withAutoRecovery", false, "Start Autorecovery service Bookie server"); - BK_OPTS.addOption("readOnly", false, + BK_OPTS.addOption("r", "readOnly", false, "Force Start a ReadOnly Bookie server"); BK_OPTS.addOption("z", "zkserver", true, "Zookeeper Server"); BK_OPTS.addOption("m", "zkledgerpath", true, "Zookeeper ledgers root path"); @@ -129,7 +129,7 @@ private static ServerConfiguration parseArgs(String[] args) conf.setAutoRecoveryDaemonEnabled(true); } - if (cmdLine.hasOption("readOnly")) { + if (cmdLine.hasOption("r")) { conf.setForceReadOnlyBookie(true); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/BookKeeperConstants.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/BookKeeperConstants.java index 362e1e6dd70..3bbda1c6e5c 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/BookKeeperConstants.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/BookKeeperConstants.java @@ -28,6 +28,7 @@ public class BookKeeperConstants { public static final String LEDGER_NODE_PREFIX = "L"; public static final String COLON = ":"; public static final String VERSION_FILENAME = "VERSION"; + public static final String BOOKIE_STATUS_FILENAME = "BOOKIE_STATUS"; public final static String PASSWD = "passwd"; public static final String CURRENT_DIR = "current"; public static final String READONLY = "readonly"; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieInitializationTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieInitializationTest.java index 3783c57ccf2..1d1edaee337 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieInitializationTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieInitializationTest.java @@ -20,15 +20,23 @@ */ package org.apache.bookkeeper.bookie; +import static com.google.common.base.Charsets.UTF_8; +import static org.apache.bookkeeper.util.BookKeeperConstants.BOOKIE_STATUS_FILENAME; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import java.io.BufferedWriter; import java.io.File; +import java.io.FileOutputStream; import java.io.IOException; +import java.io.OutputStreamWriter; import java.net.BindException; import java.net.InetAddress; +import java.util.ArrayList; +import java.util.List; +import java.util.Random; import org.apache.bookkeeper.bookie.BookieException.DiskPartitionDuplicationException; import org.apache.bookkeeper.bookie.LedgerDirsManager.NoWritableLedgerDirException; import org.apache.bookkeeper.client.BookKeeper; @@ -319,10 +327,10 @@ public void testDuplicateBookieServerStartup() throws Exception { rm.initialize(conf, () -> {}, NullStatsLogger.INSTANCE); bs1.getBookie().setRegistrationManager(rm); bs1.start(); - + BookieServer bs2 = null; // starting bk server with same conf try { - BookieServer bs2 = new BookieServer(conf); + bs2 = new BookieServer(conf); RegistrationManager newRm = new ZKRegistrationManager(); newRm.initialize(conf, () -> {}, NullStatsLogger.INSTANCE); bs2.getBookie().registrationManager = newRm; @@ -333,6 +341,11 @@ public void testDuplicateBookieServerStartup() throws Exception { } catch (IOException e) { Assert.assertTrue("BKServer allowed duplicate Startups!", e.getMessage().contains("bind")); + } finally { + bs1.shutdown(); + if (bs2 != null) { + bs2.shutdown(); + } } } @@ -727,4 +740,174 @@ private ZooKeeperClient createNewZKClient() throws Exception { .connectString(zkUtil.getZooKeeperConnectString()) .build(); } + + /** + * Check bookie status should be able to persist on disk and retrieve when restart the bookie. + */ + @Test(timeout = 10000) + public void testPersistBookieStatus() throws Exception { + // enable persistent bookie status + File tmpDir = createTempDir("bookie", "test"); + final ServerConfiguration conf = TestBKConfiguration.newServerConfiguration() + .setZkServers(zkUtil.getZooKeeperConnectString()) + .setJournalDirName(tmpDir.getPath()) + .setLedgerDirNames(new String[] { tmpDir.getPath() }) + .setReadOnlyModeEnabled(true) + .setPersistBookieStatusEnabled(true); + BookieServer bookieServer = new BookieServer(conf); + bookieServer.start(); + Bookie bookie = bookieServer.getBookie(); + assertFalse(bookie.isReadOnly()); + // transition to readonly mode, bookie status should be persisted in ledger disks + bookie.doTransitionToReadOnlyMode(); + assertTrue(bookie.isReadOnly()); + + // restart bookie should start in read only mode + bookieServer.shutdown(); + bookieServer = new BookieServer(conf); + bookieServer.start(); + bookie = bookieServer.getBookie(); + assertTrue(bookie.isReadOnly()); + // transition to writable mode + bookie.doTransitionToWritableMode(); + // restart bookie should start in writable mode + bookieServer.shutdown(); + bookieServer = new BookieServer(conf); + bookieServer.start(); + bookie = bookieServer.getBookie(); + assertFalse(bookie.isReadOnly()); + bookieServer.shutdown(); + } + + /** + * Check when we start a ReadOnlyBookie, we should ignore bookie status + */ + @Test(timeout = 10000) + public void testReadOnlyBookieShouldIgnoreBookieStatus() throws Exception { + File tmpDir = createTempDir("bookie", "test"); + final ServerConfiguration conf = TestBKConfiguration.newServerConfiguration() + .setZkServers(zkUtil.getZooKeeperConnectString()) + .setJournalDirName(tmpDir.getPath()) + .setLedgerDirNames(new String[] { tmpDir.getPath() }) + .setReadOnlyModeEnabled(true) + .setPersistBookieStatusEnabled(true); + // start new bookie + BookieServer bookieServer = new BookieServer(conf); + bookieServer.start(); + Bookie bookie = bookieServer.getBookie(); + // persist bookie status + bookie.doTransitionToReadOnlyMode(); + bookie.doTransitionToWritableMode(); + assertFalse(bookie.isReadOnly()); + bookieServer.shutdown(); + // start read only bookie + final ServerConfiguration readOnlyConf = TestBKConfiguration.newServerConfiguration(); + readOnlyConf.loadConf(conf); + readOnlyConf.setForceReadOnlyBookie(true); + bookieServer = new BookieServer(readOnlyConf); + bookieServer.start(); + bookie = bookieServer.getBookie(); + assertTrue(bookie.isReadOnly()); + // transition to writable should fail + bookie.doTransitionToWritableMode(); + assertTrue(bookie.isReadOnly()); + bookieServer.shutdown(); + } + + /** + * Check that if there's multiple bookie status copies, as long as not all of them are corrupted, + * the bookie status should be retrievable. + */ + @Test(timeout = 10000) + public void testRetrieveBookieStatusWhenStatusFileIsCorrupted() throws Exception { + File[] tmpLedgerDirs = new File[3]; + String[] filePath = new String[tmpLedgerDirs.length]; + for (int i = 0; i < tmpLedgerDirs.length; i++) { + tmpLedgerDirs[i] = createTempDir("bookie", "test" + i); + filePath[i] = tmpLedgerDirs[i].getPath(); + } + final ServerConfiguration conf = TestBKConfiguration.newServerConfiguration() + .setZkServers(zkUtil.getZooKeeperConnectString()) + .setJournalDirName(filePath[0]) + .setLedgerDirNames(filePath) + .setReadOnlyModeEnabled(true) + .setPersistBookieStatusEnabled(true); + // start a new bookie + BookieServer bookieServer = new BookieServer(conf); + bookieServer.start(); + // transition in to read only and persist the status on disk + Bookie bookie = bookieServer.getBookie(); + assertFalse(bookie.isReadOnly()); + bookie.doTransitionToReadOnlyMode(); + assertTrue(bookie.isReadOnly()); + // corrupt status file + List ledgerDirs = bookie.getLedgerDirsManager().getAllLedgerDirs(); + corruptFile(new File(ledgerDirs.get(0), BOOKIE_STATUS_FILENAME)); + corruptFile(new File(ledgerDirs.get(1), BOOKIE_STATUS_FILENAME)); + // restart the bookie should be in read only mode + bookieServer.shutdown(); + bookieServer = new BookieServer(conf); + bookieServer.start(); + bookie = bookieServer.getBookie(); + assertTrue(bookie.isReadOnly()); + bookieServer.shutdown(); + } + + /** + * Check if the bookie would read the latest status if the status files are not consistent. + * @throws Exception + */ + @Test(timeout = 10000) + public void testReadLatestBookieStatus() throws Exception { + File[] tmpLedgerDirs = new File[3]; + String[] filePath = new String[tmpLedgerDirs.length]; + for (int i = 0; i < tmpLedgerDirs.length; i++) { + tmpLedgerDirs[i] = createTempDir("bookie", "test" + i); + filePath[i] = tmpLedgerDirs[i].getPath(); + } + final ServerConfiguration conf = TestBKConfiguration.newServerConfiguration() + .setZkServers(zkUtil.getZooKeeperConnectString()) + .setJournalDirName(filePath[0]) + .setLedgerDirNames(filePath) + .setReadOnlyModeEnabled(true) + .setPersistBookieStatusEnabled(true); + // start a new bookie + BookieServer bookieServer = new BookieServer(conf); + bookieServer.start(); + // transition in to read only and persist the status on disk + Bookie bookie = bookieServer.getBookie(); + assertFalse(bookie.isReadOnly()); + bookie.doTransitionToReadOnlyMode(); + assertTrue(bookie.isReadOnly()); + // Manually update a status file, so it becomes the latest + Thread.sleep(1); + BookieStatus status = new BookieStatus(); + List dirs = new ArrayList(); + dirs.add(bookie.getLedgerDirsManager().getAllLedgerDirs().get(0)); + status.writeToDirectories(dirs); + // restart the bookie should start in writable state + bookieServer.shutdown(); + bookieServer = new BookieServer(conf); + bookieServer.start(); + bookie = bookieServer.getBookie(); + assertFalse(bookie.isReadOnly()); + bookieServer.shutdown(); + } + + private void corruptFile(File file) throws IOException { + FileOutputStream fos = new FileOutputStream(file); + BufferedWriter bw = null; + try { + bw = new BufferedWriter(new OutputStreamWriter(fos, UTF_8)); + byte[] bytes = new byte[64]; + new Random().nextBytes(bytes); + bw.write(new String(bytes)); + } finally { + if (bw != null) { + bw.close(); + } + fos.close(); + } + } + } diff --git a/site/_data/config/bk_server.yaml b/site/_data/config/bk_server.yaml index af74b5a8db3..54a26c4be63 100644 --- a/site/_data/config/bk_server.yaml +++ b/site/_data/config/bk_server.yaml @@ -342,7 +342,8 @@ groups: - param: forceReadOnlyBookie description: Whether the bookie is force started in read only mode or not. default: 'false' - + - param: persistBookieStatusEnabled + description: Persist the bookie status locally on the disks. So the bookies can keep their status upon restarts. - name: Disk utilization params: From 6fb5b5f668a49589c0736d606392269ca8355870 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Sun, 29 Oct 2017 10:28:10 +0800 Subject: [PATCH 07/15] ISSUE #675: Enable checkstyle in a few packages These packages include: - feature - processor - shims - stats - streaming - versioning - zookeeper not fully enabled in conf package. Author: Sijie Guo Reviewers: Jia Zhai This closes #676 from sijie/fix_checkstyles, closes #675 --- .../conf/AbstractConfiguration.java | 5 +- .../bookkeeper/conf/ClientConfiguration.java | 5 +- .../apache/bookkeeper/conf/Configurable.java | 11 +-- .../bookkeeper/conf/ServerConfiguration.java | 4 +- .../apache/bookkeeper/conf/package-info.java | 23 ++++++ .../feature/CacheableFeatureProvider.java | 8 +- .../apache/bookkeeper/feature/Feature.java | 3 +- .../bookkeeper/feature/FeatureProvider.java | 3 +- .../bookkeeper/feature/FixedValueFeature.java | 6 +- .../bookkeeper/feature/SettableFeature.java | 6 +- .../feature/SettableFeatureProvider.java | 5 +- .../bookkeeper/feature/package-info.java | 33 ++++++++ .../processor/RequestProcessor.java | 7 +- .../bookkeeper/processor/package-info.java | 23 ++++++ .../shims/zk/ZooKeeperServerShimFactory.java | 3 + .../shims/zk/ZooKeeperServerShimImpl.java | 5 +- .../bookkeeper/shims/zk/package-info.java | 23 ++++++ .../bookkeeper/stats/AlertStatsLogger.java | 4 +- .../apache/bookkeeper/stats/package-info.java | 23 ++++++ .../streaming/LedgerInputStream.java | 20 ++--- .../streaming/LedgerOutputStream.java | 9 ++- .../bookkeeper/streaming/package-info.java | 23 ++++++ .../bookkeeper/versioning/LongVersion.java | 2 +- .../apache/bookkeeper/versioning/Version.java | 20 ++--- .../bookkeeper/versioning/Versioned.java | 31 +++----- .../bookkeeper/versioning/package-info.java | 23 ++++++ .../ExponentialBackoffRetryPolicy.java | 3 + .../bookkeeper/zookeeper/RetryPolicy.java | 8 +- .../bookkeeper/zookeeper/ZooKeeperClient.java | 77 +++++++++++-------- .../zookeeper/ZooKeeperWatcherBase.java | 8 +- .../bookkeeper/zookeeper/ZooWorker.java | 21 +++-- .../bookkeeper/zookeeper/package-info.java | 23 ++++++ .../bookkeeper/server-suppressions.xml | 8 -- 33 files changed, 334 insertions(+), 142 deletions(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/package-info.java create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/feature/package-info.java create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/processor/package-info.java create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/shims/zk/package-info.java create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/stats/package-info.java create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/streaming/package-info.java create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/versioning/package-info.java create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/zookeeper/package-info.java diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/AbstractConfiguration.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/AbstractConfiguration.java index 1858c67dc58..594973d05f0 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/AbstractConfiguration.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/AbstractConfiguration.java @@ -29,15 +29,12 @@ import org.apache.commons.configuration.ConfigurationException; import org.apache.commons.configuration.PropertiesConfiguration; import org.apache.commons.configuration.SystemConfiguration; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** - * Abstract configuration + * Abstract configuration. */ public abstract class AbstractConfiguration extends CompositeConfiguration { - static final Logger LOG = LoggerFactory.getLogger(AbstractConfiguration.class); public static final String READ_SYSTEM_PROPERTIES_PROPERTY = "org.apache.bookkeeper.conf.readsystemproperties"; /** diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ClientConfiguration.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ClientConfiguration.java index 6e11f94f394..5a2175346f4 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ClientConfiguration.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ClientConfiguration.java @@ -18,14 +18,13 @@ package org.apache.bookkeeper.conf; import static com.google.common.base.Charsets.UTF_8; -import io.netty.buffer.ByteBuf; import static org.apache.bookkeeper.util.BookKeeperConstants.FEATURE_DISABLE_ENSEMBLE_CHANGE; +import io.netty.buffer.ByteBuf; import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; - import org.apache.bookkeeper.client.BookKeeper.DigestType; import org.apache.bookkeeper.client.EnsemblePlacementPolicy; import org.apache.bookkeeper.client.LedgerHandle; @@ -37,7 +36,7 @@ /** - * Configuration settings for client side + * Configuration settings for client side. */ public class ClientConfiguration extends AbstractConfiguration { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/Configurable.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/Configurable.java index 36166904369..6b4adb3a7fb 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/Configurable.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/Configurable.java @@ -20,14 +20,15 @@ */ package org.apache.bookkeeper.conf; +import org.apache.bookkeeper.common.annotation.InterfaceAudience.Public; +import org.apache.bookkeeper.common.annotation.InterfaceStability.Stable; import org.apache.commons.configuration.Configuration; -import com.google.common.annotations.Beta; - /** * Class that may be configured with a {@link Configuration}. */ -@Beta +@Public +@Stable public interface Configurable { /** @@ -36,12 +37,12 @@ public interface Configurable { * @param conf * Configuration object to use */ - public void setConf(Configuration conf); + void setConf(Configuration conf); /** * Return the configuration used by this object. * * @return configuration used by this object. */ - public Configuration getConf(); + Configuration getConf(); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ServerConfiguration.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ServerConfiguration.java index cbf88ad58f1..fea051c5f7a 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ServerConfiguration.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ServerConfiguration.java @@ -22,13 +22,11 @@ import java.util.concurrent.TimeUnit; import com.google.common.annotations.Beta; - import org.apache.bookkeeper.bookie.InterleavedLedgerStorage; import org.apache.bookkeeper.bookie.LedgerStorage; import org.apache.bookkeeper.bookie.SortedLedgerStorage; import org.apache.bookkeeper.discover.RegistrationManager; import org.apache.bookkeeper.discover.ZKRegistrationManager; -import org.apache.bookkeeper.server.component.ServerLifecycleComponent; import org.apache.bookkeeper.stats.NullStatsProvider; import org.apache.bookkeeper.stats.StatsProvider; import org.apache.bookkeeper.util.BookKeeperConstants; @@ -37,7 +35,7 @@ import org.apache.commons.lang.StringUtils; /** - * Configuration manages server-side settings + * Configuration manages server-side settings. */ public class ServerConfiguration extends AbstractConfiguration { // Entry Log Parameters diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/package-info.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/package-info.java new file mode 100644 index 00000000000..bf4aaa52895 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/package-info.java @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/** + * Configurations used by bookkeeper. + */ +package org.apache.bookkeeper.conf; \ No newline at end of file diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/feature/CacheableFeatureProvider.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/feature/CacheableFeatureProvider.java index 95c9981db23..b80ea516250 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/feature/CacheableFeatureProvider.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/feature/CacheableFeatureProvider.java @@ -1,5 +1,3 @@ -package org.apache.bookkeeper.feature; - /* * * Licensed to the Apache Software Foundation (ASF) under one @@ -20,14 +18,14 @@ * under the License. * */ - -import org.apache.commons.lang.StringUtils; +package org.apache.bookkeeper.feature; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import org.apache.commons.lang.StringUtils; /** - * Cacheable Feature Provider + * Cacheable Feature Provider. */ public abstract class CacheableFeatureProvider implements FeatureProvider { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/feature/Feature.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/feature/Feature.java index ab9fae97b7a..4578e7f541e 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/feature/Feature.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/feature/Feature.java @@ -25,7 +25,8 @@ * This interface represents a feature. */ public interface Feature { - public static int FEATURE_AVAILABILITY_MAX_VALUE = 100; + + int FEATURE_AVAILABILITY_MAX_VALUE = 100; /** * Returns a textual representation of the feature. diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/feature/FeatureProvider.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/feature/FeatureProvider.java index 04686b7f8a5..0710cea22c9 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/feature/FeatureProvider.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/feature/FeatureProvider.java @@ -1,5 +1,3 @@ -package org.apache.bookkeeper.feature; - /* * * Licensed to the Apache Software Foundation (ASF) under one @@ -20,6 +18,7 @@ * under the License. * */ +package org.apache.bookkeeper.feature; /** * Provider to provide features. diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/feature/FixedValueFeature.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/feature/FixedValueFeature.java index 825276bd3a8..729ac6b3cc9 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/feature/FixedValueFeature.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/feature/FixedValueFeature.java @@ -1,5 +1,3 @@ -package org.apache.bookkeeper.feature; - /* * * Licensed to the Apache Software Foundation (ASF) under one @@ -20,7 +18,11 @@ * under the License. * */ +package org.apache.bookkeeper.feature; +/** + * A feature implementation that has a fixed value of availability. + */ public class FixedValueFeature implements Feature { protected final String name; protected int availability; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/feature/SettableFeature.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/feature/SettableFeature.java index f175bf01c19..bb6d1db091e 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/feature/SettableFeature.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/feature/SettableFeature.java @@ -1,5 +1,3 @@ -package org.apache.bookkeeper.feature; - /* * * Licensed to the Apache Software Foundation (ASF) under one @@ -20,7 +18,11 @@ * under the License. * */ +package org.apache.bookkeeper.feature; +/** + * A feature implementation that allow changing availability programmatically. + */ public class SettableFeature extends FixedValueFeature { public SettableFeature(String name, int initialAvailability) { super(name, initialAvailability); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/feature/SettableFeatureProvider.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/feature/SettableFeatureProvider.java index e21ad9db8e6..9a6cc841fdd 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/feature/SettableFeatureProvider.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/feature/SettableFeatureProvider.java @@ -1,5 +1,3 @@ -package org.apache.bookkeeper.feature; - /* * * Licensed to the Apache Software Foundation (ASF) under one @@ -20,13 +18,14 @@ * under the License. * */ +package org.apache.bookkeeper.feature; /** * A provider will provide settable features. */ public class SettableFeatureProvider extends CacheableFeatureProvider { - public final static FeatureProvider DISABLE_ALL = new SettableFeatureProvider("", 0); + public static final FeatureProvider DISABLE_ALL = new SettableFeatureProvider("", 0); protected final int availability; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/feature/package-info.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/feature/package-info.java new file mode 100644 index 00000000000..cd911cc12f0 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/feature/package-info.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/** + * A feature-flag system + * that is used to proportionally control what features are enabled for the system. + * + *

In other words, it is a way of altering the control in a system without restarting it. + * It can be used during all stages of developement, its most visible use case is on production. + * For instance, during a production release, you can enable or disable individual features, + * control the data flow through the system, thereby minimizing risk of system failures + * in real time. + * + *

The feature provider interface is pluggable and easy to integrate with + * any configuration management system. + */ +package org.apache.bookkeeper.feature; \ No newline at end of file diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/processor/RequestProcessor.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/processor/RequestProcessor.java index 658753c9d37..19f095c07d8 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/processor/RequestProcessor.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/processor/RequestProcessor.java @@ -22,12 +22,15 @@ import io.netty.channel.Channel; +/** + * A request processor that is used for processing requests at bookie side. + */ public interface RequestProcessor { /** * Close the request processor. */ - public void close(); + void close(); /** * Process request. @@ -37,6 +40,6 @@ public interface RequestProcessor { * @param channel * channel received the given request r */ - public void processRequest(Object r, Channel channel); + void processRequest(Object r, Channel channel); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/processor/package-info.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/processor/package-info.java new file mode 100644 index 00000000000..bba0c7b5614 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/processor/package-info.java @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/** + * Define the processors used by bookie from processing requests. + */ +package org.apache.bookkeeper.processor; \ No newline at end of file diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/shims/zk/ZooKeeperServerShimFactory.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/shims/zk/ZooKeeperServerShimFactory.java index 59ff88a3a4a..6f82e46cfc4 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/shims/zk/ZooKeeperServerShimFactory.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/shims/zk/ZooKeeperServerShimFactory.java @@ -23,6 +23,9 @@ import java.io.File; import java.io.IOException; +/** + * A factory to create zookeeper server. + */ public class ZooKeeperServerShimFactory { public static ZooKeeperServerShim createServer(File snapDir, File logDir, int zkPort, int maxCC) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/shims/zk/ZooKeeperServerShimImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/shims/zk/ZooKeeperServerShimImpl.java index b74df4b6c6f..cc6e2e2871a 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/shims/zk/ZooKeeperServerShimImpl.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/shims/zk/ZooKeeperServerShimImpl.java @@ -20,12 +20,11 @@ */ package org.apache.bookkeeper.shims.zk; -import org.apache.zookeeper.server.NIOServerCnxnFactory; -import org.apache.zookeeper.server.ZooKeeperServer; - import java.io.File; import java.io.IOException; import java.net.InetSocketAddress; +import org.apache.zookeeper.server.NIOServerCnxnFactory; +import org.apache.zookeeper.server.ZooKeeperServer; class ZooKeeperServerShimImpl implements ZooKeeperServerShim { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/shims/zk/package-info.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/shims/zk/package-info.java new file mode 100644 index 00000000000..5f944612e08 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/shims/zk/package-info.java @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/** + * A shim layer to adopt different zookeeper versions. + */ +package org.apache.bookkeeper.shims.zk; \ No newline at end of file diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/stats/AlertStatsLogger.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/stats/AlertStatsLogger.java index 63ba3b593f9..f9ffe3578f3 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/stats/AlertStatsLogger.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/stats/AlertStatsLogger.java @@ -20,7 +20,7 @@ import org.slf4j.LoggerFactory; /** - * This class is used to raise alert when we detect an event that should never happen in production + * This class is used to raise alert when we detect an event that should never happen in production. */ public class AlertStatsLogger { private static final Logger logger = LoggerFactory.getLogger(AlertStatsLogger.class); @@ -49,7 +49,7 @@ public AlertStatsLogger(StatsLogger globalStatsLogger, String alertStatName) { private String format(String msg) { return msg.startsWith("ALERT!: ") ? msg : - ("ALERT!: " + (scope != null ? "(" + scope + "):" : "" ) + msg); + ("ALERT!: " + (scope != null ? "(" + scope + "):" : "") + msg); } private void initializeCountersIfNeeded() { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/stats/package-info.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/stats/package-info.java new file mode 100644 index 00000000000..df77c581c35 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/stats/package-info.java @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/** + * The bookkeeper stats related classes. + */ +package org.apache.bookkeeper.stats; \ No newline at end of file diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/streaming/LedgerInputStream.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/streaming/LedgerInputStream.java index a56003a78cb..20f26dc4d8f 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/streaming/LedgerInputStream.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/streaming/LedgerInputStream.java @@ -24,15 +24,15 @@ import java.io.InputStream; import java.nio.ByteBuffer; import java.util.Enumeration; - import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.LedgerEntry; import org.apache.bookkeeper.client.LedgerHandle; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; +/** + * An input stream on reading data from a ledger. + */ public class LedgerInputStream extends InputStream { - private final static Logger LOG = LoggerFactory.getLogger(LedgerInputStream.class); + private final LedgerHandle lh; private ByteBuffer bytebuff; byte[] bbytes; @@ -42,11 +42,12 @@ public class LedgerInputStream extends InputStream { Enumeration ledgerSeq = null; /** - * construct a outputstream from a ledger handle + * construct a outputstream from a ledger handle. * * @param lh * ledger handle - * @throws {@link BKException}, {@link InterruptedException} + * @throws BKException when encountered bookkeeper exceptions + * @throws InterruptedException when opening a ledger input stream is interrupted. */ public LedgerInputStream(LedgerHandle lh) throws BKException, InterruptedException { this.lh = lh; @@ -58,13 +59,14 @@ public LedgerInputStream(LedgerHandle lh) throws BKException, InterruptedExcepti } /** - * construct a outputstream from a ledger handle + * construct a outputstream from a ledger handle. * * @param lh * the ledger handle * @param size * the size of the buffer - * @throws {@link BKException}, {@link InterruptedException} + * @throws BKException when encountered bookkeeper exceptions + * @throws InterruptedException when opening a ledger input stream is interrupted. */ public LedgerInputStream(LedgerHandle lh, int size) throws BKException, InterruptedException { this.lh = lh; @@ -88,7 +90,7 @@ public void close() { } /** - * refill the buffer, we need to read more bytes + * refill the buffer, we need to read more bytes. * * @return if we can refill or not */ diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/streaming/LedgerOutputStream.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/streaming/LedgerOutputStream.java index ac7e04fcc81..0d2720428a4 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/streaming/LedgerOutputStream.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/streaming/LedgerOutputStream.java @@ -23,7 +23,6 @@ import java.io.IOException; import java.io.OutputStream; import java.nio.ByteBuffer; - import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.LedgerHandle; import org.slf4j.Logger; @@ -36,14 +35,16 @@ * out the entry to the ledger. */ public class LedgerOutputStream extends OutputStream { - private final static Logger LOG = LoggerFactory.getLogger(LedgerOutputStream.class); + + private static final Logger LOG = LoggerFactory.getLogger(LedgerOutputStream.class); + private final LedgerHandle lh; private ByteBuffer bytebuff; byte[] bbytes; int defaultSize = 1024 * 1024; // 1MB default size /** - * construct a outputstream from a ledger handle + * construct a outputstream from a ledger handle. * * @param lh * ledger handle @@ -55,7 +56,7 @@ public LedgerOutputStream(LedgerHandle lh) { } /** - * construct a outputstream from a ledger handle + * construct a outputstream from a ledger handle. * * @param lh * the ledger handle diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/streaming/package-info.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/streaming/package-info.java new file mode 100644 index 00000000000..9da615ff3c8 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/streaming/package-info.java @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/** + * A java io stream wrapper over bookkeeper ledgers. + */ +package org.apache.bookkeeper.streaming; \ No newline at end of file diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/versioning/LongVersion.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/versioning/LongVersion.java index 11f9b1900f0..4ec43882677 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/versioning/LongVersion.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/versioning/LongVersion.java @@ -39,7 +39,7 @@ public Occurred compare(Version v) { } else if (!(v instanceof LongVersion)) { throw new IllegalArgumentException("Invalid version type"); } - LongVersion zv = (LongVersion)v; + LongVersion zv = (LongVersion) v; int res = Long.compare(version, zv.version); if (res == 0) { return Occurred.CONCURRENTLY; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/versioning/Version.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/versioning/Version.java index 39ad3e5396d..10f4415f416 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/versioning/Version.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/versioning/Version.java @@ -28,7 +28,7 @@ public interface Version { /** * Initial version. */ - public static final Version NEW = new Version() { + Version NEW = new Version() { @Override public Occurred compare(Version v) { if (null == v) { @@ -44,19 +44,19 @@ public Occurred compare(Version v) { /** * Match any version. */ - public static final Version ANY = new Version() { - @Override - public Occurred compare(Version v) { - if (null == v) { - throw new NullPointerException("Version is not allowed to be null."); - } - return Occurred.CONCURRENTLY; + Version ANY = v -> { + if (null == v) { + throw new NullPointerException("Version is not allowed to be null."); } + return Occurred.CONCURRENTLY; }; - public static enum Occurred { + /** + * Define the sequence of versions. + */ + enum Occurred { BEFORE, AFTER, CONCURRENTLY } - public Occurred compare(Version v); + Occurred compare(Version v); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/versioning/Versioned.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/versioning/Versioned.java index 7e8f1c96f4e..f6926248d02 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/versioning/Versioned.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/versioning/Versioned.java @@ -18,28 +18,17 @@ package org.apache.bookkeeper.versioning; +import lombok.AllArgsConstructor; +import lombok.Data; + +/** + * A Versioned value represents a value associated with a version. + * + * @param value type. + */ +@Data +@AllArgsConstructor public class Versioned { T value; Version version; - - public Versioned(T value, Version version) { - this.value = value; - this.version = version; - } - - public void setValue(T value) { - this.value = value; - } - - public T getValue() { - return value; - } - - public void setVersion(Version version) { - this.version = version; - } - - public Version getVersion() { - return version; - } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/versioning/package-info.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/versioning/package-info.java new file mode 100644 index 00000000000..86fb4b5588c --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/versioning/package-info.java @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/** + * a versioning library provides an abstraction over versioned data. + */ +package org.apache.bookkeeper.versioning; \ No newline at end of file diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/zookeeper/ExponentialBackoffRetryPolicy.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/zookeeper/ExponentialBackoffRetryPolicy.java index 23d86f48a85..bb66ee9e52c 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/zookeeper/ExponentialBackoffRetryPolicy.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/zookeeper/ExponentialBackoffRetryPolicy.java @@ -22,6 +22,9 @@ import java.util.Random; +/** + * A retry policy implementation that backoff retries exponentially. + */ public class ExponentialBackoffRetryPolicy implements RetryPolicy { private final Random random; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/zookeeper/RetryPolicy.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/zookeeper/RetryPolicy.java index a4269fbe719..fba8b1a879c 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/zookeeper/RetryPolicy.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/zookeeper/RetryPolicy.java @@ -28,7 +28,7 @@ public interface RetryPolicy { /** * Called when retrying an operation failed for some reason. Return true if * another attempt is allowed to make. - * + * * @param retryCount * The number of times retried so far (1 for the first time). * @param elapsedRetryTime @@ -36,12 +36,12 @@ public interface RetryPolicy { * milliseconds) * @return true if anther attempt is allowed to make. otherwise, false. */ - public boolean allowRetry(int retryCount, long elapsedRetryTime); + boolean allowRetry(int retryCount, long elapsedRetryTime); /** * Called before making an attempt to retry a failed operation. Return 0 if * an attempt needs to be made immediately. - * + * * @param retryCount * The number of times retried so far (0 for the first time). * @param elapsedRetryTime @@ -50,6 +50,6 @@ public interface RetryPolicy { * @return the elapsed time that the attempt needs to wait before retrying. * (in milliseconds) */ - public long nextRetryWaitTime(int retryCount, long elapsedRetryTime); + long nextRetryWaitTime(int retryCount, long elapsedRetryTime); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/zookeeper/ZooKeeperClient.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/zookeeper/ZooKeeperClient.java index 94df8da3f3c..0c143da0a53 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/zookeeper/ZooKeeperClient.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/zookeeper/ZooKeeperClient.java @@ -20,6 +20,12 @@ */ package org.apache.bookkeeper.zookeeper; +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkNotNull; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.util.concurrent.RateLimiter; +import com.google.common.util.concurrent.ThreadFactoryBuilder; import java.io.IOException; import java.util.List; import java.util.Set; @@ -31,20 +37,14 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; -import com.google.common.util.concurrent.RateLimiter; -import com.google.common.util.concurrent.ThreadFactoryBuilder; - import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.stats.OpStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.zookeeper.ZooWorker.ZooCallable; import org.apache.zookeeper.AsyncCallback.ACLCallback; -import org.apache.zookeeper.AsyncCallback.Create2Callback; import org.apache.zookeeper.AsyncCallback.Children2Callback; import org.apache.zookeeper.AsyncCallback.ChildrenCallback; +import org.apache.zookeeper.AsyncCallback.Create2Callback; import org.apache.zookeeper.AsyncCallback.DataCallback; import org.apache.zookeeper.AsyncCallback.MultiCallback; import org.apache.zookeeper.AsyncCallback.StatCallback; @@ -66,11 +66,11 @@ import org.slf4j.LoggerFactory; /** - * Provide a zookeeper client to handle session expire + * Provide a zookeeper client to handle session expire. */ public class ZooKeeperClient extends ZooKeeper implements Watcher { - final static Logger logger = LoggerFactory.getLogger(ZooKeeperClient.class); + private static final Logger logger = LoggerFactory.getLogger(ZooKeeperClient.class); private static final int DEFAULT_RETRY_EXECUTOR_THREAD_COUNT = 1; @@ -160,6 +160,9 @@ static ZooKeeperClient createConnectedZooKeeperClient( .build(); } + /** + * A builder to build retryable zookeeper client. + */ public static class Builder { String connectString = null; int sessionTimeoutMs = 10000; @@ -213,10 +216,10 @@ public Builder retryThreadCount(int numThreads) { } public ZooKeeperClient build() throws IOException, KeeperException, InterruptedException { - Preconditions.checkNotNull(connectString); - Preconditions.checkArgument(sessionTimeoutMs > 0); - Preconditions.checkNotNull(statsLogger); - Preconditions.checkArgument(retryExecThreadCount > 0); + checkNotNull(connectString); + checkArgument(sessionTimeoutMs > 0); + checkNotNull(statsLogger); + checkArgument(retryExecThreadCount > 0); if (null == connectRetryPolicy) { connectRetryPolicy = @@ -326,8 +329,8 @@ protected ZooKeeper createZooKeeper() throws IOException { @Override public void process(WatchedEvent event) { - if (event.getType() == EventType.None && - event.getState() == KeeperState.Expired) { + if (event.getType() == EventType.None + && event.getState() == KeeperState.Expired) { onExpired(); } } @@ -351,7 +354,10 @@ private void onExpired() { } } - static abstract class ZkRetryRunnable implements Runnable { + /** + * A runnable that retries zookeeper operations. + */ + abstract static class ZkRetryRunnable implements Runnable { final ZooWorker worker; final RateLimiter rateLimiter; @@ -465,7 +471,7 @@ public void multi(final Iterable ops, @Override public void processResult(int rc, String path, Object ctx, List results) { - ZooWorker worker = (ZooWorker)ctx; + ZooWorker worker = (ZooWorker) ctx; if (allowRetry(worker, rc)) { backOffAndRetry(that, worker.nextRetryWaitTime()); } else { @@ -536,7 +542,7 @@ public void getACL(final String path, final Stat stat, final ACLCallback cb, fin @Override public void processResult(int rc, String path, Object ctx, List acl, Stat stat) { - ZooWorker worker = (ZooWorker)ctx; + ZooWorker worker = (ZooWorker) ctx; if (allowRetry(worker, rc)) { backOffAndRetry(that, worker.nextRetryWaitTime()); } else { @@ -596,7 +602,7 @@ public void setACL(final String path, final List acl, final int version, @Override public void processResult(int rc, String path, Object ctx, Stat stat) { - ZooWorker worker = (ZooWorker)ctx; + ZooWorker worker = (ZooWorker) ctx; if (allowRetry(worker, rc)) { backOffAndRetry(that, worker.nextRetryWaitTime()); } else { @@ -633,7 +639,7 @@ public void sync(final String path, final VoidCallback cb, final Object context) @Override public void processResult(int rc, String path, Object ctx) { - ZooWorker worker = (ZooWorker)ctx; + ZooWorker worker = (ZooWorker) ctx; if (allowRetry(worker, rc)) { backOffAndRetry(that, worker.nextRetryWaitTime()); } else { @@ -714,7 +720,7 @@ public void create(final String path, final byte[] data, final List acl, @Override public void processResult(int rc, String path, Object ctx, String name) { - ZooWorker worker = (ZooWorker)ctx; + ZooWorker worker = (ZooWorker) ctx; if (allowRetry(worker, rc)) { backOffAndRetry(that, worker.nextRetryWaitTime()); } else { @@ -782,7 +788,7 @@ public void create(final String path, @Override public void processResult(int rc, String path, Object ctx, String name, Stat stat) { - ZooWorker worker = (ZooWorker)ctx; + ZooWorker worker = (ZooWorker) ctx; if (allowRetry(worker, rc)) { backOffAndRetry(that, worker.nextRetryWaitTime()); } else { @@ -842,7 +848,7 @@ public void delete(final String path, final int version, final VoidCallback cb, @Override public void processResult(int rc, String path, Object ctx) { - ZooWorker worker = (ZooWorker)ctx; + ZooWorker worker = (ZooWorker) ctx; if (allowRetry(worker, rc)) { backOffAndRetry(that, worker.nextRetryWaitTime()); } else { @@ -921,7 +927,7 @@ public void exists(final String path, final Watcher watcher, final StatCallback @Override public void processResult(int rc, String path, Object ctx, Stat stat) { - ZooWorker worker = (ZooWorker)ctx; + ZooWorker worker = (ZooWorker) ctx; if (allowRetry(worker, rc)) { backOffAndRetry(that, worker.nextRetryWaitTime()); } else { @@ -958,7 +964,7 @@ public void exists(final String path, final boolean watch, final StatCallback cb @Override public void processResult(int rc, String path, Object ctx, Stat stat) { - ZooWorker worker = (ZooWorker)ctx; + ZooWorker worker = (ZooWorker) ctx; if (allowRetry(worker, rc)) { backOffAndRetry(that, worker.nextRetryWaitTime()); } else { @@ -1039,7 +1045,7 @@ public void getData(final String path, final Watcher watcher, final DataCallback @Override public void processResult(int rc, String path, Object ctx, byte[] data, Stat stat) { - ZooWorker worker = (ZooWorker)ctx; + ZooWorker worker = (ZooWorker) ctx; if (allowRetry(worker, rc)) { backOffAndRetry(that, worker.nextRetryWaitTime()); } else { @@ -1076,7 +1082,7 @@ public void getData(final String path, final boolean watch, final DataCallback c @Override public void processResult(int rc, String path, Object ctx, byte[] data, Stat stat) { - ZooWorker worker = (ZooWorker)ctx; + ZooWorker worker = (ZooWorker) ctx; if (allowRetry(worker, rc)) { backOffAndRetry(that, worker.nextRetryWaitTime()); } else { @@ -1136,7 +1142,7 @@ public void setData(final String path, final byte[] data, final int version, @Override public void processResult(int rc, String path, Object ctx, Stat stat) { - ZooWorker worker = (ZooWorker)ctx; + ZooWorker worker = (ZooWorker) ctx; if (allowRetry(worker, rc)) { backOffAndRetry(that, worker.nextRetryWaitTime()); } else { @@ -1219,7 +1225,7 @@ public void getChildren(final String path, final Watcher watcher, @Override public void processResult(int rc, String path, Object ctx, List children, Stat stat) { - ZooWorker worker = (ZooWorker)ctx; + ZooWorker worker = (ZooWorker) ctx; if (allowRetry(worker, rc)) { backOffAndRetry(that, worker.nextRetryWaitTime()); } else { @@ -1258,7 +1264,7 @@ public void getChildren(final String path, final boolean watch, final Children2C @Override public void processResult(int rc, String path, Object ctx, List children, Stat stat) { - ZooWorker worker = (ZooWorker)ctx; + ZooWorker worker = (ZooWorker) ctx; if (allowRetry(worker, rc)) { backOffAndRetry(that, worker.nextRetryWaitTime()); } else { @@ -1342,7 +1348,7 @@ public void getChildren(final String path, final Watcher watcher, @Override public void processResult(int rc, String path, Object ctx, List children) { - ZooWorker worker = (ZooWorker)ctx; + ZooWorker worker = (ZooWorker) ctx; if (allowRetry(worker, rc)) { backOffAndRetry(that, worker.nextRetryWaitTime()); } else { @@ -1381,7 +1387,7 @@ public void getChildren(final String path, final boolean watch, @Override public void processResult(int rc, String path, Object ctx, List children) { - ZooWorker worker = (ZooWorker)ctx; + ZooWorker worker = (ZooWorker) ctx; if (allowRetry(worker, rc)) { backOffAndRetry(that, worker.nextRetryWaitTime()); } else { @@ -1422,7 +1428,12 @@ public void removeWatches(String path, Watcher watcher, WatcherType watcherType, } @Override - public void removeWatches(String path, Watcher watcher, WatcherType watcherType, boolean local, VoidCallback cb, Object ctx) { + public void removeWatches(String path, + Watcher watcher, + WatcherType watcherType, + boolean local, + VoidCallback cb, + Object ctx) { ZooKeeper zkHandle = zk.get(); if (null == zkHandle) { ZooKeeperClient.super.removeWatches(path, watcher, watcherType, local, cb, ctx); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/zookeeper/ZooKeeperWatcherBase.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/zookeeper/ZooKeeperWatcherBase.java index 7a98668bb80..f5816b0c331 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/zookeeper/ZooKeeperWatcherBase.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/zookeeper/ZooKeeperWatcherBase.java @@ -26,7 +26,6 @@ import java.util.concurrent.CopyOnWriteArraySet; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; - import org.apache.bookkeeper.stats.Counter; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; @@ -132,7 +131,8 @@ public void process(WatchedEvent event) { clientConnectLatch.countDown(); break; case Disconnected: - LOG.info("ZooKeeper client is disconnected from zookeeper now, but it is OK unless we received EXPIRED event."); + LOG.info("ZooKeeper client is disconnected from zookeeper now," + + " but it is OK unless we received EXPIRED event."); break; case Expired: clientConnectLatch = new CountDownLatch(1); @@ -147,7 +147,7 @@ public void process(WatchedEvent event) { } /** - * Waiting for the SyncConnected event from the ZooKeeper server + * Waiting for the SyncConnected event from the ZooKeeper server. * * @throws KeeperException * when there is no connection @@ -161,7 +161,7 @@ public void waitForConnection() throws KeeperException, InterruptedException { } /** - * Return zookeeper session time out + * Return zookeeper session time out. */ public int getZkSessionTimeOut() { return zkSessionTimeOut; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/zookeeper/ZooWorker.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/zookeeper/ZooWorker.java index 117ba8edc4d..c2ca0194785 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/zookeeper/ZooWorker.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/zookeeper/ZooWorker.java @@ -20,10 +20,9 @@ */ package org.apache.bookkeeper.zookeeper; +import com.google.common.util.concurrent.RateLimiter; import java.util.concurrent.Callable; import java.util.concurrent.TimeUnit; - -import com.google.common.util.concurrent.RateLimiter; import org.apache.bookkeeper.stats.OpStatsLogger; import org.apache.bookkeeper.util.MathUtils; import org.apache.zookeeper.KeeperException; @@ -36,7 +35,7 @@ */ class ZooWorker { - static final Logger logger = LoggerFactory.getLogger(ZooWorker.class); + private static final Logger logger = LoggerFactory.getLogger(ZooWorker.class); int attempts = 0; long startTimeNanos; @@ -75,10 +74,10 @@ public long nextRetryWaitTime() { * @return true if given result code is recoverable. */ public static boolean isRecoverableException(int rc) { - return KeeperException.Code.CONNECTIONLOSS.intValue() == rc || - KeeperException.Code.OPERATIONTIMEOUT.intValue() == rc || - KeeperException.Code.SESSIONMOVED.intValue() == rc || - KeeperException.Code.SESSIONEXPIRED.intValue() == rc; + return KeeperException.Code.CONNECTIONLOSS.intValue() == rc + || KeeperException.Code.OPERATIONTIMEOUT.intValue() == rc + || KeeperException.Code.SESSIONMOVED.intValue() == rc + || KeeperException.Code.SESSIONEXPIRED.intValue() == rc; } /** @@ -91,7 +90,7 @@ public static boolean isRecoverableException(KeeperException exception) { return isRecoverableException(exception.code().intValue()); } - static interface ZooCallable { + interface ZooCallable { /** * Be compatible with ZooKeeper interface. * @@ -99,7 +98,7 @@ static interface ZooCallable { * @throws InterruptedException * @throws KeeperException */ - public T call() throws InterruptedException, KeeperException; + T call() throws InterruptedException, KeeperException; } /** @@ -145,8 +144,8 @@ public static T syncCallWithRetries(ZooKeeperClient client, ++attempts; boolean rethrow = true; long elapsedTime = MathUtils.elapsedMSec(startTimeNanos); - if (((null != client && isRecoverableException(e)) || null == client) && - retryPolicy.allowRetry(attempts, elapsedTime)) { + if (((null != client && isRecoverableException(e)) || null == client) + && retryPolicy.allowRetry(attempts, elapsedTime)) { rethrow = false; } if (rethrow) { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/zookeeper/package-info.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/zookeeper/package-info.java new file mode 100644 index 00000000000..aace95b26ae --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/zookeeper/package-info.java @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/** + * A simple retryable zookeeper wrapper provided in bookkeeper. + */ +package org.apache.bookkeeper.zookeeper; \ No newline at end of file diff --git a/buildtools/src/main/resources/bookkeeper/server-suppressions.xml b/buildtools/src/main/resources/bookkeeper/server-suppressions.xml index d5c14b2622e..0d4f896b077 100644 --- a/buildtools/src/main/resources/bookkeeper/server-suppressions.xml +++ b/buildtools/src/main/resources/bookkeeper/server-suppressions.xml @@ -24,24 +24,16 @@ - - - - - - - - From f09835e5675611005f25cb8930bf54d90471f98d Mon Sep 17 00:00:00 2001 From: Ivan Kelly Date: Mon, 30 Oct 2017 20:47:18 +0100 Subject: [PATCH 08/15] DistributionSchedule uses custom wrapped int[] rather than HashSet This avoids the autoboxing on Integers and allocations of cells for the hashset. This also implies using wrapped int[] object for the write set in the DistributionSchedules and PlacementPolicies. This patch was originally submitted as dc7933b on the yahoo-4.3 branch, though this has been modified extensively to remove the dependency on carrotsearch hppc and to allow it to work with placement policies which didn't exist at the time of the original patch. Author: Ivan Kelly Reviewers: Enrico Olivelli , Sijie Guo This closes #657 from ivankelly/yahoo-backports --- .../LocalBookieEnsemblePlacementPolicy.java | 13 +- .../DefaultEnsemblePlacementPolicy.java | 21 +- .../client/DistributionSchedule.java | 106 +++++- .../client/EnsemblePlacementPolicy.java | 30 +- .../bookkeeper/client/LedgerChecker.java | 14 +- .../bookkeeper/client/LedgerHandle.java | 3 +- .../bookkeeper/client/PendingAddOp.java | 32 +- .../bookkeeper/client/PendingReadOp.java | 28 +- .../bookkeeper/client/PendingWriteLacOp.java | 24 +- .../RackawareEnsemblePlacementPolicy.java | 20 +- .../RackawareEnsemblePlacementPolicyImpl.java | 98 ++++-- .../client/ReadLastConfirmedAndEntryOp.java | 19 +- .../RegionAwareEnsemblePlacementPolicy.java | 148 ++++---- .../RoundRobinDistributionSchedule.java | 331 ++++++++++++++++-- .../TopologyAwareEnsemblePlacementPolicy.java | 21 +- .../RoundRobinDistributionScheduleTest.java | 41 ++- .../bookkeeper/client/TestLedgerChecker.java | 11 +- .../TestRackawareEnsemblePlacementPolicy.java | 184 +++++++--- ...estRegionAwareEnsemblePlacementPolicy.java | 153 ++++---- 19 files changed, 965 insertions(+), 332 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LocalBookieEnsemblePlacementPolicy.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LocalBookieEnsemblePlacementPolicy.java index fe18d5ddf4b..6bb8e5db262 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LocalBookieEnsemblePlacementPolicy.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LocalBookieEnsemblePlacementPolicy.java @@ -29,6 +29,7 @@ import java.util.Set; import org.apache.bookkeeper.client.BKException.BKNotEnoughBookiesException; import org.apache.bookkeeper.client.BookieInfoReader.BookieInfo; +import org.apache.bookkeeper.client.DistributionSchedule; import org.apache.bookkeeper.client.EnsemblePlacementPolicy; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.ServerConfiguration; @@ -88,14 +89,18 @@ public BookieSocketAddress replaceBookie(int ensembleSize, int writeQuorumSize, } @Override - public List reorderReadSequence(ArrayList ensemble, List writeSet, - Map bookieFailureHistory) { + public DistributionSchedule.WriteSet reorderReadSequence( + ArrayList ensemble, + Map bookieFailureHistory, + DistributionSchedule.WriteSet writeSet) { return null; } @Override - public List reorderReadLACSequence(ArrayList ensemble, List writeSet, - Map bookieFailureHistory) { + public DistributionSchedule.WriteSet reorderReadLACSequence( + ArrayList ensemble, + Map bookieFailureHistory, + DistributionSchedule.WriteSet writeSet) { return null; } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/DefaultEnsemblePlacementPolicy.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/DefaultEnsemblePlacementPolicy.java index 14a8a0ca6eb..ffb229d6bcb 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/DefaultEnsemblePlacementPolicy.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/DefaultEnsemblePlacementPolicy.java @@ -146,21 +146,20 @@ public Set onClusterChanged(Set writab } @Override - public List reorderReadSequence(ArrayList ensemble, List writeSet, Map bookieFailureHistory) { + public DistributionSchedule.WriteSet reorderReadSequence( + ArrayList ensemble, + Map bookieFailureHistory, + DistributionSchedule.WriteSet writeSet) { return writeSet; } @Override - public List reorderReadLACSequence(ArrayList ensemble, List writeSet, Map bookieFailureHistory) { - List retList = new ArrayList(writeSet); - if (retList.size() < ensemble.size()) { - for (int i = 0; i < ensemble.size(); i++) { - if (!retList.contains(i)) { - retList.add(i); - } - } - } - return retList; + public DistributionSchedule.WriteSet reorderReadLACSequence( + ArrayList ensemble, + Map bookieFailureHistory, + DistributionSchedule.WriteSet writeSet) { + writeSet.addMissingIndices(ensemble.size()); + return writeSet; } @Override diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/DistributionSchedule.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/DistributionSchedule.java index c0d78e9ccc5..6db2b6c2b99 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/DistributionSchedule.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/DistributionSchedule.java @@ -19,7 +19,6 @@ import org.apache.bookkeeper.net.BookieSocketAddress; -import java.util.List; import java.util.Map; /** @@ -32,12 +31,108 @@ * to. */ -interface DistributionSchedule { +public interface DistributionSchedule { + + /** + * A write set represents the set of bookies to which + * a request will be written. + * The set consists of a list of indices which can be + * used to lookup the bookie in the ensemble. + */ + public interface WriteSet { + /** + * The number of indexes in the write set. + */ + public int size(); + + /** + * Whether the set contains the given index. + */ + public boolean contains(int i); + + /** + * Get the index at index i. + */ + public int get(int i); + + /** + * Set the index at index i. + * @return the previous value at that index. + */ + public int set(int i, int index); + + /** + * Sort the indices + */ + public void sort(); + + /** + * Index of a specified bookie index. + * -1 if not found. + */ + public int indexOf(int index); + + /** + * If we want a write set to cover all bookies in an ensemble + * of size X, then all of the index from 0..X must exist in the + * write set. This method appends those which are missing to the + * end of the write set. + */ + public void addMissingIndices(int maxIndex); + + /** + * Move an index from one position to another, + * shifting the other indices accordingly. + */ + public void moveAndShift(int from, int to); + + /** + * Recycle write set object when not in use. + */ + public void recycle(); + + /** + * Make a deep copy of this write set. + */ + public WriteSet copy(); + } + + public static WriteSet NULL_WRITE_SET = new WriteSet() { + @Override + public int size() { return 0; } + @Override + public boolean contains(int i) { return false; } + @Override + public int get(int i) { + throw new ArrayIndexOutOfBoundsException(); + } + @Override + public int set(int i, int index) { + throw new ArrayIndexOutOfBoundsException(); + } + @Override + public void sort() {} + @Override + public int indexOf(int index) { return -1; } + @Override + public void addMissingIndices(int maxIndex) { + throw new ArrayIndexOutOfBoundsException(); + } + @Override + public void moveAndShift(int from, int to) { + throw new ArrayIndexOutOfBoundsException(); + } + @Override + public void recycle() {} + @Override + public WriteSet copy() { return this; } + }; /** * return the set of bookie indices to send the message to */ - public List getWriteSet(long entryId); + public WriteSet getWriteSet(long entryId); + /** * An ack set represents the set of bookies from which @@ -75,6 +170,11 @@ public interface AckSet { * Used for reissuing write requests. */ public boolean removeBookieAndCheck(int bookie); + + /** + * Recycle this ack set when not used anymore + */ + public void recycle(); } /** diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/EnsemblePlacementPolicy.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/EnsemblePlacementPolicy.java index 8a9b86a3653..2301a9544f5 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/EnsemblePlacementPolicy.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/EnsemblePlacementPolicy.java @@ -294,16 +294,19 @@ public BookieSocketAddress replaceBookie(int ensembleSize, * * @param ensemble * Ensemble to read entries. - * @param writeSet - * Write quorum to read entries. * @param bookieFailureHistory * Observed failures on the bookies - * @return read sequence of bookies + * @param writeSet + * Write quorum to read entries. This will be modified, rather than + * allocating a new WriteSet. + * @return The read sequence. This will be the same object as the passed in + * writeSet. * @since 4.5 */ - public List reorderReadSequence(ArrayList ensemble, - List writeSet, - Map bookieFailureHistory); + public DistributionSchedule.WriteSet reorderReadSequence( + ArrayList ensemble, + Map bookieFailureHistory, + DistributionSchedule.WriteSet writeSet); /** @@ -311,16 +314,19 @@ public List reorderReadSequence(ArrayList ensemble * * @param ensemble * Ensemble to read entries. - * @param writeSet - * Write quorum to read entries. * @param bookieFailureHistory * Observed failures on the bookies - * @return read sequence of bookies + * @param writeSet + * Write quorum to read entries. This will be modified, rather than + * allocating a new WriteSet. + * @return The read sequence. This will be the same object as the passed in + * writeSet. * @since 4.5 */ - public List reorderReadLACSequence(ArrayList ensemble, - List writeSet, - Map bookieFailureHistory); + public DistributionSchedule.WriteSet reorderReadLACSequence( + ArrayList ensemble, + Map bookieFailureHistory, + DistributionSchedule.WriteSet writeSet); /** * Send the bookie info details. diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerChecker.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerChecker.java index 2ad80769871..0eebf3ec077 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerChecker.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerChecker.java @@ -266,7 +266,7 @@ public void operationComplete(int rc, LedgerFragment result) { * Check that all the fragments in the passed in ledger, and report those * which are missing. */ - public void checkLedger(LedgerHandle lh, + public void checkLedger(final LedgerHandle lh, final GenericCallback> cb) { // build a set of all fragment replicas final Set fragments = new HashSet(); @@ -322,7 +322,7 @@ public void checkLedger(LedgerHandle lh, if (curEntryId == lastEntry) { final long entryToRead = curEntryId; - EntryExistsCallback eecb + final EntryExistsCallback eecb = new EntryExistsCallback(lh.getLedgerMetadata().getWriteQuorumSize(), new GenericCallback() { public void operationComplete(int rc, Boolean result) { @@ -333,11 +333,13 @@ public void operationComplete(int rc, Boolean result) { } }); - for (int bi : lh.getDistributionSchedule().getWriteSet(entryToRead)) { - BookieSocketAddress addr = curEnsemble.get(bi); - bookieClient.readEntry(addr, lh.getId(), - entryToRead, eecb, null); + DistributionSchedule.WriteSet writeSet + = lh.getDistributionSchedule().getWriteSet(entryToRead); + for (int i = 0; i < writeSet.size(); i++) { + BookieSocketAddress addr = curEnsemble.get(writeSet.get(i)); + bookieClient.readEntry(addr, lh.getId(), entryToRead, eecb, null); } + writeSet.recycle(); return; } else { fragments.add(lastLedgerFragment); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java index bb7c9549f37..c61c85be084 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java @@ -817,8 +817,7 @@ public void asyncAddEntry(final byte[] data, final int offset, final int length, +") or length("+length+")"); } - PendingAddOp op = new PendingAddOp(LedgerHandle.this, cb, ctx); - doAsyncAddEntry(op, Unpooled.wrappedBuffer(data, offset, length), cb, ctx); + asyncAddEntry(Unpooled.wrappedBuffer(data, offset, length), cb, ctx); } public void asyncAddEntry(ByteBuf data, final AddCallback cb, final Object ctx) { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingAddOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingAddOp.java index af4f35e312c..00a36b3168d 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingAddOp.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingAddOp.java @@ -22,9 +22,8 @@ import io.netty.util.Timeout; import io.netty.util.TimerTask; -import java.util.HashSet; +import java.util.Arrays; import java.util.Map; -import java.util.Set; import java.util.concurrent.TimeUnit; import com.google.common.collect.ImmutableMap; @@ -35,6 +34,7 @@ import org.apache.bookkeeper.stats.OpStatsLogger; import org.apache.bookkeeper.util.MathUtils; import org.apache.bookkeeper.util.SafeRunnable; +import org.apache.commons.lang3.ArrayUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.concurrent.RejectedExecutionException; @@ -56,7 +56,6 @@ class PendingAddOp implements WriteCallback, TimerTask { Object ctx; long entryId; int entryLength; - Set writeSet; DistributionSchedule.AckSet ackSet; boolean completed = false; @@ -76,6 +75,7 @@ class PendingAddOp implements WriteCallback, TimerTask { this.cb = cb; this.ctx = ctx; this.entryId = LedgerHandle.INVALID_ENTRY_ID; + this.ackSet = lh.distributionSchedule.getAckSet(); this.addOpLogger = lh.bk.getAddOpLogger(); this.timeoutSec = lh.bk.getConf().getAddEntryQuorumTimeout(); @@ -92,7 +92,6 @@ PendingAddOp enableRecoveryAdd() { void setEntryId(long entryId) { this.entryId = entryId; - writeSet = new HashSet(lh.distributionSchedule.getWriteSet(entryId)); } long getEntryId() { @@ -152,9 +151,15 @@ void unsetSuccessAndSendWriteRequest(int bookieIndex) { // completes. // // We call sendAddSuccessCallback when unsetting t cover this case. - if (!writeSet.contains(bookieIndex)) { - lh.sendAddSuccessCallbacks(); - return; + DistributionSchedule.WriteSet writeSet + = lh.distributionSchedule.getWriteSet(entryId); + try { + if (!writeSet.contains(bookieIndex)) { + lh.sendAddSuccessCallbacks(); + return; + } + } finally { + writeSet.recycle(); } if (callbackTriggered) { @@ -192,8 +197,16 @@ void initiate(ByteBuf toSend, int entryLength) { // Retain the buffer until all writes are complete this.toSend.retain(); this.entryLength = entryLength; - for (int bookieIndex : writeSet) { - sendWriteRequest(bookieIndex); + + // Iterate over set and trigger the sendWriteRequests + DistributionSchedule.WriteSet writeSet + = lh.distributionSchedule.getWriteSet(entryId); + try { + for (int i = 0; i < writeSet.size(); i++) { + sendWriteRequest(writeSet.get(i)); + } + } finally { + writeSet.recycle(); } } @@ -279,6 +292,7 @@ public void writeComplete(int rc, long ledgerId, long entryId, BookieSocketAddre if (ackQuorum && !completed) { completed = true; + ackSet.recycle(); sendAddSuccessCallbacks(); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingReadOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingReadOp.java index 6b62c0f2c5c..40da31c2dc1 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingReadOp.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingReadOp.java @@ -26,7 +26,6 @@ import java.util.BitSet; import java.util.Enumeration; import java.util.HashSet; -import java.util.List; import java.util.NoSuchElementException; import java.util.Queue; import java.util.Set; @@ -87,7 +86,7 @@ abstract class LedgerEntryRequest extends LedgerEntry implements SpeculativeRequ int numMissedEntryReads = 0; final ArrayList ensemble; - final List writeSet; + final DistributionSchedule.WriteSet writeSet; LedgerEntryRequest(ArrayList ensemble, long lId, long eId) { super(lId, eId); @@ -95,10 +94,13 @@ abstract class LedgerEntryRequest extends LedgerEntry implements SpeculativeRequ this.ensemble = ensemble; if (lh.bk.isReorderReadSequence()) { - this.writeSet = lh.bk.getPlacementPolicy().reorderReadSequence(ensemble, - lh.distributionSchedule.getWriteSet(entryId), lh.bookieFailureHistory.asMap()); + writeSet = lh.bk.getPlacementPolicy() + .reorderReadSequence( + ensemble, + lh.bookieFailureHistory.asMap(), + lh.distributionSchedule.getWriteSet(entryId)); } else { - this.writeSet = lh.distributionSchedule.getWriteSet(entryId); + writeSet = lh.distributionSchedule.getWriteSet(entryId); } } @@ -137,6 +139,7 @@ boolean complete(int bookieIndex, BookieSocketAddress host, final ByteBuf buffer */ length = buffer.getLong(DigestManager.METADATA_LENGTH - 8); data = content; + writeSet.recycle(); return true; } else { buffer.release(); @@ -155,6 +158,7 @@ boolean fail(int rc) { if (complete.compareAndSet(false, true)) { this.rc = rc; submitCallback(rc); + writeSet.recycle(); return true; } else { return false; @@ -269,10 +273,10 @@ class ParallelReadRequest extends LedgerEntryRequest { @Override void read() { - for (int bookieIndex : writeSet) { - BookieSocketAddress to = ensemble.get(bookieIndex); + for (int i = 0; i < writeSet.size(); i++) { + BookieSocketAddress to = ensemble.get(writeSet.get(i)); try { - sendReadTo(bookieIndex, to, this); + sendReadTo(writeSet.get(i), to, this); } catch (InterruptedException ie) { LOG.error("Interrupted reading entry {} : ", this, ie); Thread.currentThread().interrupt(); @@ -322,10 +326,6 @@ private synchronized int getNextReplicaIndexToReadFrom() { return nextReplicaIndexToReadFrom; } - private int getReplicaIndex(int bookieIndex) { - return writeSet.indexOf(bookieIndex); - } - private BitSet getSentToBitSet() { BitSet b = new BitSet(ensemble.size()); @@ -386,7 +386,7 @@ synchronized BookieSocketAddress sendNextRead() { } int replica = nextReplicaIndexToReadFrom; - int bookieIndex = lh.distributionSchedule.getWriteSet(entryId).get(nextReplicaIndexToReadFrom); + int bookieIndex = writeSet.get(nextReplicaIndexToReadFrom); nextReplicaIndexToReadFrom++; try { @@ -406,7 +406,7 @@ synchronized BookieSocketAddress sendNextRead() { synchronized void logErrorAndReattemptRead(int bookieIndex, BookieSocketAddress host, String errMsg, int rc) { super.logErrorAndReattemptRead(bookieIndex, host, errMsg, rc); - int replica = getReplicaIndex(bookieIndex); + int replica = writeSet.indexOf(bookieIndex); if (replica == NOT_FOUND) { LOG.error("Received error from a host which is not in the ensemble {} {}.", host, ensemble); return; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingWriteLacOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingWriteLacOp.java index 755f93dbf02..3ab5c7b11e7 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingWriteLacOp.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingWriteLacOp.java @@ -17,7 +17,7 @@ */ package org.apache.bookkeeper.client; - +import java.util.BitSet; import java.util.HashSet; import java.util.Set; @@ -45,8 +45,7 @@ class PendingWriteLacOp implements WriteLacCallback { AddLacCallback cb; long lac; Object ctx; - Set writeSet; - Set receivedResponseSet; + BitSet receivedResponseSet; DistributionSchedule.AckSet ackSet; boolean completed = false; @@ -66,8 +65,11 @@ class PendingWriteLacOp implements WriteLacCallback { void setLac(long lac) { this.lac = lac; - this.writeSet = new HashSet(lh.distributionSchedule.getWriteSet(lac)); - this.receivedResponseSet = new HashSet(writeSet); + + this.receivedResponseSet = new BitSet( + lh.getLedgerMetadata().getWriteQuorumSize()); + this.receivedResponseSet.set(0, + lh.getLedgerMetadata().getWriteQuorumSize()); } void sendWriteLacRequest(int bookieIndex) { @@ -77,8 +79,14 @@ void sendWriteLacRequest(int bookieIndex) { void initiate(ByteBuf toSend) { this.toSend = toSend; - for (int bookieIndex: writeSet) { - sendWriteLacRequest(bookieIndex); + DistributionSchedule.WriteSet writeSet + = lh.distributionSchedule.getWriteSet(lac); + try { + for (int i = 0; i < writeSet.size(); i++) { + sendWriteLacRequest(writeSet.get(i)); + } + } finally { + writeSet.recycle(); } } @@ -95,7 +103,7 @@ public void writeLacComplete(int rc, long ledgerId, BookieSocketAddress addr, Ob } // We got response. - receivedResponseSet.remove(bookieIndex); + receivedResponseSet.clear(bookieIndex); if (rc == BKException.Code.OK) { if (ackSet.completeBookieAndCheck(bookieIndex) && !completed) { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RackawareEnsemblePlacementPolicy.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RackawareEnsemblePlacementPolicy.java index 8126b9656d2..b3b633fc1f2 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RackawareEnsemblePlacementPolicy.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RackawareEnsemblePlacementPolicy.java @@ -118,17 +118,21 @@ public BookieSocketAddress replaceBookie( } @Override - public List reorderReadSequence(ArrayList ensemble, - List writeSet, - Map bookieFailureHistory) { - return super.reorderReadSequence(ensemble, writeSet, bookieFailureHistory); + public DistributionSchedule.WriteSet reorderReadSequence( + ArrayList ensemble, + Map bookieFailureHistory, + DistributionSchedule.WriteSet writeSet) { + return super.reorderReadSequence(ensemble, bookieFailureHistory, + writeSet); } @Override - public List reorderReadLACSequence(ArrayList ensemble, - List writeSet, - Map bookieFailureHistory) { - return super.reorderReadLACSequence(ensemble, writeSet, bookieFailureHistory); + public DistributionSchedule.WriteSet reorderReadLACSequence( + ArrayList ensemble, + Map bookieFailureHistory, + DistributionSchedule.WriteSet writeSet) { + return super.reorderReadLACSequence(ensemble, bookieFailureHistory, + writeSet); } @Override diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RackawareEnsemblePlacementPolicyImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RackawareEnsemblePlacementPolicyImpl.java index 527f180ca2c..b08ce4340a7 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RackawareEnsemblePlacementPolicyImpl.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RackawareEnsemblePlacementPolicyImpl.java @@ -19,6 +19,7 @@ import java.net.InetAddress; import java.net.UnknownHostException; +import java.util.Arrays; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -28,6 +29,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.locks.ReentrantReadWriteLock; import com.google.common.base.Preconditions; @@ -71,6 +73,15 @@ class RackawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsemblePlacemen static final int RACKNAME_DISTANCE_FROM_LEAVES = 1; + // masks for reordering + static final int LOCAL_MASK = 0x01 << 24; + static final int LOCAL_FAIL_MASK = 0x02 << 24; + static final int REMOTE_MASK = 0x04 << 24; + static final int REMOTE_FAIL_MASK = 0x08 << 24; + static final int READ_ONLY_MASK = 0x10 << 24; + static final int UNAVAIL_MASK = 0x20 << 24; + static final int MASK_BITS = 0xFFF << 20; + static class DefaultResolver implements DNSToSwitchMapping { final Supplier defaultRackSupplier; @@ -769,47 +780,86 @@ protected List selectRandomInternal(List bookiesToSelect } @Override - public List reorderReadSequence(ArrayList ensemble, List writeSet, Map bookieFailureHistory) { + public DistributionSchedule.WriteSet reorderReadSequence( + ArrayList ensemble, + Map bookieFailureHistory, + DistributionSchedule.WriteSet writeSet) { int ensembleSize = ensemble.size(); - List finalList = new ArrayList(writeSet.size()); - List observedFailuresList = new ArrayList(writeSet.size()); - List readOnlyList = new ArrayList(writeSet.size()); - List unAvailableList = new ArrayList(writeSet.size()); - for (Integer idx : writeSet) { + + for (int i = 0; i < writeSet.size(); i++) { + int idx = writeSet.get(i); BookieSocketAddress address = ensemble.get(idx); Long lastFailedEntryOnBookie = bookieFailureHistory.get(address); if (null == knownBookies.get(address)) { - // there isn't too much differences between readonly bookies from unavailable bookies. since there - // is no write requests to them, so we shouldn't try reading from readonly bookie in prior to writable + // there isn't too much differences between readonly bookies + // from unavailable bookies. since there + // is no write requests to them, so we shouldn't try reading + // from readonly bookie in prior to writable // bookies. - if ((null == readOnlyBookies) || !readOnlyBookies.contains(address)) { - unAvailableList.add(idx); + if ((null == readOnlyBookies) + || !readOnlyBookies.contains(address)) { + writeSet.set(i, idx | UNAVAIL_MASK); } else { - readOnlyList.add(idx); + writeSet.set(i, idx | READ_ONLY_MASK); } } else { - if ((lastFailedEntryOnBookie == null) || (lastFailedEntryOnBookie < 0)) { - finalList.add(idx); + if ((lastFailedEntryOnBookie == null) + || (lastFailedEntryOnBookie < 0)) { + writeSet.set(i, idx | LOCAL_MASK); } else { - observedFailuresList.add(lastFailedEntryOnBookie * ensembleSize + idx); + long failIdx = lastFailedEntryOnBookie * ensembleSize + idx; + writeSet.set(i, + (int)(failIdx & ~MASK_BITS) | LOCAL_FAIL_MASK); } } } - if (reorderReadsRandom) { - Collections.shuffle(finalList); - Collections.shuffle(readOnlyList); - Collections.shuffle(unAvailableList); + // Add a mask to ensure the sort is stable, sort, + // and then remove mask. This maintains stability as + // long as there are fewer than 16 bookies in the write set. + for (int i = 0; i < writeSet.size(); i++) { + writeSet.set(i, writeSet.get(i) | ((i & 0xF) << 20)); + } + writeSet.sort(); + for (int i = 0; i < writeSet.size(); i++) { + writeSet.set(i, writeSet.get(i) & ~((0xF) << 20)); } - Collections.sort(observedFailuresList); + if (reorderReadsRandom) { + shuffleWithMask(writeSet, LOCAL_MASK, MASK_BITS); + shuffleWithMask(writeSet, READ_ONLY_MASK, MASK_BITS); + shuffleWithMask(writeSet, UNAVAIL_MASK, MASK_BITS); + } - for(long value: observedFailuresList) { - finalList.add((int)(value % ensembleSize)); + // remove all masks + for (int i = 0; i < writeSet.size(); i++) { + writeSet.set(i, (writeSet.get(i) & ~MASK_BITS) % ensembleSize); } - finalList.addAll(readOnlyList); - finalList.addAll(unAvailableList); - return finalList; + return writeSet; + } + + /** + * Shuffle all the entries of an array that matches a mask. + * It assumes all entries with the same mask are contiguous in the array. + */ + static void shuffleWithMask(DistributionSchedule.WriteSet writeSet, + int mask, int bits) { + int first = -1; + int last = -1; + for (int i = 0; i < writeSet.size(); i++) { + if ((writeSet.get(i) & bits) == mask) { + if (first == -1) { + first = i; + } + last = i; + } + } + if (first != -1) { + for (int i = last + 1; i > first; i--) { + int swapWith = ThreadLocalRandom.current().nextInt(i); + writeSet.set(swapWith, writeSet.set(i, writeSet.get(swapWith))); + } + } } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadLastConfirmedAndEntryOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadLastConfirmedAndEntryOp.java index 05434d19538..ae721d502db 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadLastConfirmedAndEntryOp.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadLastConfirmedAndEntryOp.java @@ -34,6 +34,7 @@ import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks; import org.apache.bookkeeper.proto.ReadLastConfirmedAndEntryContext; import org.apache.bookkeeper.util.MathUtils; +import org.apache.commons.lang3.ArrayUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -73,8 +74,8 @@ abstract class ReadLACAndEntryRequest extends LedgerEntry { int numMissedEntryReads = 0; final ArrayList ensemble; - final List writeSet; - final List orderedEnsemble; + final DistributionSchedule.WriteSet writeSet; + final DistributionSchedule.WriteSet orderedEnsemble; ReadLACAndEntryRequest(ArrayList ensemble, long lId, long eId) { super(lId, eId); @@ -83,9 +84,9 @@ abstract class ReadLACAndEntryRequest extends LedgerEntry { this.writeSet = lh.distributionSchedule.getWriteSet(entryId); if (lh.bk.reorderReadSequence) { this.orderedEnsemble = lh.bk.placementPolicy.reorderReadLACSequence(ensemble, - writeSet, lh.bookieFailureHistory.asMap()); + lh.bookieFailureHistory.asMap(), writeSet.copy()); } else { - this.orderedEnsemble = writeSet; + this.orderedEnsemble = writeSet.copy(); } } @@ -121,6 +122,8 @@ boolean complete(int bookieIndex, BookieSocketAddress host, final ByteBuf buffer } if (!complete.getAndSet(true)) { + writeSet.recycle(); + orderedEnsemble.recycle(); rc = BKException.Code.OK; this.entryId = entryId; /* @@ -144,6 +147,8 @@ boolean complete(int bookieIndex, BookieSocketAddress host, final ByteBuf buffer */ boolean fail(int rc) { if (complete.compareAndSet(false, true)) { + writeSet.recycle(); + orderedEnsemble.recycle(); this.rc = rc; translateAndSetFirstError(rc); completeRequest(); @@ -244,10 +249,10 @@ class ParallelReadRequest extends ReadLACAndEntryRequest { @Override void read() { - for (int bookieIndex : orderedEnsemble) { - BookieSocketAddress to = ensemble.get(bookieIndex); + for (int i = 0; i < orderedEnsemble.size(); i++) { + BookieSocketAddress to = ensemble.get(orderedEnsemble.get(i)); try { - sendReadTo(bookieIndex, to, this); + sendReadTo(orderedEnsemble.get(i), to, this); } catch (InterruptedException ie) { LOG.error("Interrupted reading entry {} : ", this, ie); Thread.currentThread().interrupt(); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RegionAwareEnsemblePlacementPolicy.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RegionAwareEnsemblePlacementPolicy.java index 7d4d1604c25..8342a6b1c76 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RegionAwareEnsemblePlacementPolicy.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RegionAwareEnsemblePlacementPolicy.java @@ -17,6 +17,7 @@ */ package org.apache.bookkeeper.client; +import java.util.Arrays; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -500,107 +501,130 @@ protected BookieNode replaceFromRack(BookieNode bookieNodeToReplace, } @Override - public final List reorderReadSequence(ArrayList ensemble, List writeSet, Map bookieFailureHistory) { + public final DistributionSchedule.WriteSet reorderReadSequence( + ArrayList ensemble, + Map bookieFailureHistory, + DistributionSchedule.WriteSet writeSet) { if (UNKNOWN_REGION.equals(myRegion)) { - return super.reorderReadSequence(ensemble, writeSet, bookieFailureHistory); + return super.reorderReadSequence(ensemble, bookieFailureHistory, + writeSet); } else { int ensembleSize = ensemble.size(); - List finalList = new ArrayList(writeSet.size()); - List localList = new ArrayList(writeSet.size()); - List localFailures = new ArrayList(writeSet.size()); - List remoteList = new ArrayList(writeSet.size()); - List remoteFailures = new ArrayList(writeSet.size()); - List readOnlyList = new ArrayList(writeSet.size()); - List unAvailableList = new ArrayList(writeSet.size()); - for (Integer idx : writeSet) { + + for (int i = 0; i < writeSet.size(); i++) { + int idx = writeSet.get(i); BookieSocketAddress address = ensemble.get(idx); String region = getRegion(address); Long lastFailedEntryOnBookie = bookieFailureHistory.get(address); if (null == knownBookies.get(address)) { - // there isn't too much differences between readonly bookies from unavailable bookies. since there - // is no write requests to them, so we shouldn't try reading from readonly bookie in prior to writable - // bookies. - if ((null == readOnlyBookies) || !readOnlyBookies.contains(address)) { - unAvailableList.add(idx); + // there isn't too much differences between readonly bookies + // from unavailable bookies. since there + // is no write requests to them, so we shouldn't try reading + // from readonly bookie in prior to writable bookies. + if ((null == readOnlyBookies) + || !readOnlyBookies.contains(address)) { + writeSet.set(i, idx | UNAVAIL_MASK); } else { - readOnlyList.add(idx); + writeSet.set(i, idx | READ_ONLY_MASK); } } else if (region.equals(myRegion)) { - if ((lastFailedEntryOnBookie == null) || (lastFailedEntryOnBookie < 0)) { - localList.add(idx); + if ((lastFailedEntryOnBookie == null) + || (lastFailedEntryOnBookie < 0)) { + writeSet.set(i, idx | LOCAL_MASK); } else { - localFailures.add(lastFailedEntryOnBookie * ensembleSize + idx); + long failIdx + = lastFailedEntryOnBookie * ensembleSize + idx; + writeSet.set(i, (int)(failIdx & ~MASK_BITS) + | LOCAL_FAIL_MASK); } } else { - if ((lastFailedEntryOnBookie == null) || (lastFailedEntryOnBookie < 0)) { - remoteList.add(idx); + if ((lastFailedEntryOnBookie == null) + || (lastFailedEntryOnBookie < 0)) { + writeSet.set(i, idx | REMOTE_MASK); } else { - remoteFailures.add(lastFailedEntryOnBookie * ensembleSize + idx); + long failIdx + = lastFailedEntryOnBookie * ensembleSize + idx; + writeSet.set(i, (int)(failIdx & ~MASK_BITS) + | REMOTE_FAIL_MASK); } } } - // Given that idx is less than ensemble size the order of the elements in these two lists - // is determined by the lastFailedEntryOnBookie - Collections.sort(localFailures); - Collections.sort(remoteFailures); + // Add a mask to ensure the sort is stable, sort, + // and then remove mask. This maintains stability as + // long as there are fewer than 16 bookies in the write set. + for (int i = 0; i < writeSet.size(); i++) { + writeSet.set(i, writeSet.get(i) | ((i & 0xF) << 20)); + } + writeSet.sort(); + for (int i = 0; i < writeSet.size(); i++) { + writeSet.set(i, writeSet.get(i) & ~((0xF) << 20)); + } if (reorderReadsRandom) { - Collections.shuffle(localList); - Collections.shuffle(remoteList); - Collections.shuffle(readOnlyList); - Collections.shuffle(unAvailableList); + shuffleWithMask(writeSet, LOCAL_MASK, MASK_BITS); + shuffleWithMask(writeSet, REMOTE_MASK, MASK_BITS); + shuffleWithMask(writeSet, READ_ONLY_MASK, MASK_BITS); + shuffleWithMask(writeSet, UNAVAIL_MASK, MASK_BITS); } // nodes within a region are ordered as follows // (Random?) list of nodes that have no history of failure // Nodes with Failure history are ordered in the reverse // order of the most recent entry that generated an error - for(long value: localFailures) { - localList.add((int)(value % ensembleSize)); - } - - for(long value: remoteFailures) { - remoteList.add((int)(value % ensembleSize)); + // The sort will have put them in correct order, + // so remove the bits that sort by age. + for (int i = 0; i < writeSet.size(); i++) { + int mask = writeSet.get(i) & MASK_BITS; + int idx = (writeSet.get(i) & ~MASK_BITS) % ensembleSize; + if (mask == LOCAL_FAIL_MASK) { + writeSet.set(i, LOCAL_MASK | idx); + } else if (mask == REMOTE_FAIL_MASK) { + writeSet.set(i, REMOTE_MASK | idx); + } } - // Insert a node from the remote region at the specified location so we - // try more than one region within the max allowed latency - for (int i = 0; i < REMOTE_NODE_IN_REORDER_SEQUENCE; i++) { - if (localList.size() > 0) { - finalList.add(localList.remove(0)); - } else { + // Insert a node from the remote region at the specified location so + // we try more than one region within the max allowed latency + int firstRemote = -1; + for (int i = 0; i < writeSet.size(); i++) { + if ((writeSet.get(i) & MASK_BITS) == REMOTE_MASK) { + firstRemote = i; break; } } - - if (remoteList.size() > 0) { - finalList.add(remoteList.remove(0)); + if (firstRemote != -1) { + int i = 0; + for (;i < REMOTE_NODE_IN_REORDER_SEQUENCE + && i < writeSet.size(); i++) { + if ((writeSet.get(i) & MASK_BITS) != LOCAL_MASK) { + break; + } + } + writeSet.moveAndShift(firstRemote, i); } - // Add all the local nodes - finalList.addAll(localList); - finalList.addAll(remoteList); - finalList.addAll(readOnlyList); - finalList.addAll(unAvailableList); - return finalList; + + // remove all masks + for (int i = 0; i < writeSet.size(); i++) { + writeSet.set(i, writeSet.get(i) & ~MASK_BITS); + } + return writeSet; } } @Override - public final List reorderReadLACSequence(ArrayList ensemble, List writeSet, Map bookieFailureHistory) { + public final DistributionSchedule.WriteSet reorderReadLACSequence( + ArrayList ensemble, + Map bookieFailureHistory, + DistributionSchedule.WriteSet writeSet) { if (UNKNOWN_REGION.equals(myRegion)) { - return super.reorderReadLACSequence(ensemble, writeSet, bookieFailureHistory); - } - List finalList = reorderReadSequence(ensemble, writeSet, bookieFailureHistory); - - if (finalList.size() < ensemble.size()) { - for (int i = 0; i < ensemble.size(); i++) { - if (!finalList.contains(i)) { - finalList.add(i); - } - } + return super.reorderReadLACSequence(ensemble, bookieFailureHistory, + writeSet); } + DistributionSchedule.WriteSet finalList + = reorderReadSequence(ensemble, bookieFailureHistory, writeSet); + finalList.addMissingIndices(ensemble.size()); return finalList; } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RoundRobinDistributionSchedule.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RoundRobinDistributionSchedule.java index 4eaf37ba73a..ca7ca48854d 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RoundRobinDistributionSchedule.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RoundRobinDistributionSchedule.java @@ -19,12 +19,18 @@ import com.google.common.collect.ImmutableMap; import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.commons.lang3.ArrayUtils; -import java.util.HashMap; -import java.util.List; -import java.util.ArrayList; -import java.util.HashSet; +import java.util.BitSet; import java.util.Map; +import java.util.Arrays; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.MoreObjects; + +import io.netty.util.Recycler; +import io.netty.util.Recycler.Handle; + /** * A specific {@link DistributionSchedule} that places entries in round-robin @@ -34,10 +40,9 @@ * */ class RoundRobinDistributionSchedule implements DistributionSchedule { - private int writeQuorumSize; - private int ackQuorumSize; - private int ensembleSize; - + private final int writeQuorumSize; + private final int ackQuorumSize; + private final int ensembleSize; public RoundRobinDistributionSchedule(int writeQuorumSize, int ackQuorumSize, int ensembleSize) { this.writeQuorumSize = writeQuorumSize; @@ -46,44 +51,295 @@ public RoundRobinDistributionSchedule(int writeQuorumSize, int ackQuorumSize, in } @Override - public List getWriteSet(long entryId) { - List set = new ArrayList(); - for (int i = 0; i < this.writeQuorumSize; i++) { - set.add((int)((entryId + i) % ensembleSize)); + public WriteSet getWriteSet(long entryId) { + return WriteSetImpl.create(ensembleSize, writeQuorumSize, entryId); + } + + @VisibleForTesting + static WriteSet writeSetFromValues(Integer... values) { + WriteSetImpl writeSet = WriteSetImpl.create(0, 0, 0); + writeSet.setSize(values.length); + for (int i = 0; i < values.length; i++) { + writeSet.set(i, values[i]); + } + return writeSet; + } + + private static class WriteSetImpl implements WriteSet { + int[] array = null; + int size; + + private final Handle recyclerHandle; + private static final Recycler RECYCLER + = new Recycler() { + protected WriteSetImpl newObject( + Recycler.Handle handle) { + return new WriteSetImpl(handle); + } + }; + + private WriteSetImpl(Handle recyclerHandle) { + this.recyclerHandle = recyclerHandle; + } + + static WriteSetImpl create(int ensembleSize, + int writeQuorumSize, + long entryId) { + WriteSetImpl writeSet = RECYCLER.get(); + writeSet.reset(ensembleSize, writeQuorumSize, entryId); + return writeSet; + } + + private void reset(int ensembleSize, int writeQuorumSize, + long entryId) { + setSize(writeQuorumSize); + for (int w = 0; w < writeQuorumSize; w++) { + set(w, (int)((entryId + w) % ensembleSize)); + } + } + + private void setSize(int newSize) { + if (array == null) { + array = new int[newSize]; + } else if (newSize > array.length) { + int[] newArray = new int[newSize]; + System.arraycopy(array, 0, + newArray, 0, array.length); + array = newArray; + } + size = newSize; + } + + @Override + public int size() { return size; } + + @Override + public boolean contains(int i) { + return indexOf(i) != -1; + } + + @Override + public int get(int i) { + checkBounds(i); + return array[i]; + } + + @Override + public int set(int i, int index) { + checkBounds(i); + int oldVal = array[i]; + array[i] = index; + return oldVal; + } + + @Override + public void sort() { + Arrays.sort(array, 0, size); + } + + @Override + public int indexOf(int index) { + for (int j = 0; j < size; j++) { + if (array[j] == index) { + return j; + } + } + return -1; + } + + @Override + public void addMissingIndices(int maxIndex) { + if (size < maxIndex) { + int oldSize = size; + setSize(maxIndex); + for (int i = 0, j = oldSize; + i < maxIndex && j < maxIndex; i++) { + if (!contains(i)) { + set(j, i); + j++; + } + } + } + } + + @Override + public void moveAndShift(int from, int to) { + checkBounds(from); + checkBounds(to); + if (from > to) { + int tmp = array[from]; + for (int i = from; i > to; i--) { + array[i] = array[i-1]; + } + array[to] = tmp; + } else if (from < to) { + int tmp = array[from]; + for (int i = from; i < to; i++) { + array[i] = array[i+1]; + } + array[to] = tmp; + } + } + + @Override + public void recycle() { + recyclerHandle.recycle(this); + } + + @Override + public WriteSet copy() { + WriteSetImpl copy = RECYCLER.get(); + copy.setSize(size); + for (int i = 0; i < size; i++) { + copy.set(i, array[i]); + } + return copy; + } + + @Override + public int hashCode() { + int sum = 0; + for (int i = 0; i < size; i++) { + sum += sum * 31 + i; + } + return sum; + } + + @Override + public boolean equals(Object other) { + if (other instanceof WriteSetImpl) { + WriteSetImpl o = (WriteSetImpl)other; + if (o.size() != size()) { + return false; + } + for (int i = 0; i < size(); i++) { + if (o.get(i) != get(i)) { + return false; + } + } + return true; + } + return false; + } + + @Override + public String toString() { + StringBuilder b = new StringBuilder("WriteSet["); + int i = 0; + for (; i < size() - 1; i++) { + b.append(get(i)).append(","); + } + b.append(get(i)).append("]"); + return b.toString(); + } + + private void checkBounds(int i) { + if (i < 0 || i > size) { + throw new IndexOutOfBoundsException( + "Index " + i + " out of bounds, array size = " + size); + } } - return set; } @Override public AckSet getAckSet() { - final HashSet ackSet = new HashSet(); - final HashMap failureMap = - new HashMap(); - return new AckSet() { - public boolean completeBookieAndCheck(int bookieIndexHeardFrom) { - failureMap.remove(bookieIndexHeardFrom); - ackSet.add(bookieIndexHeardFrom); - return ackSet.size() >= ackQuorumSize; + return AckSetImpl.create(ensembleSize, writeQuorumSize, ackQuorumSize); + } + + private static class AckSetImpl implements AckSet { + private int writeQuorumSize; + private int ackQuorumSize; + private final BitSet ackSet = new BitSet(); + // grows on reset() + private BookieSocketAddress[] failureMap = new BookieSocketAddress[0]; + + private final Handle recyclerHandle; + private static final Recycler RECYCLER = new Recycler() { + protected AckSetImpl newObject(Recycler.Handle handle) { + return new AckSetImpl(handle); } + }; - @Override - public boolean failBookieAndCheck(int bookieIndexHeardFrom, BookieSocketAddress address) { - ackSet.remove(bookieIndexHeardFrom); - failureMap.put(bookieIndexHeardFrom, address); - return failureMap.size() > (writeQuorumSize - ackQuorumSize); + private AckSetImpl(Handle recyclerHandle) { + this.recyclerHandle = recyclerHandle; + } + + static AckSetImpl create(int ensembleSize, + int writeQuorumSize, + int ackQuorumSize) { + AckSetImpl ackSet = RECYCLER.get(); + ackSet.reset(ensembleSize, writeQuorumSize, ackQuorumSize); + return ackSet; + } + + private void reset(int ensembleSize, + int writeQuorumSize, + int ackQuorumSize) { + this.ackQuorumSize = ackQuorumSize; + this.writeQuorumSize = writeQuorumSize; + ackSet.clear(); + if (failureMap.length < ensembleSize) { + failureMap = new BookieSocketAddress[ensembleSize]; } + Arrays.fill(failureMap, null); + } + + @Override + public boolean completeBookieAndCheck(int bookieIndexHeardFrom) { + failureMap[bookieIndexHeardFrom] = null; + ackSet.set(bookieIndexHeardFrom); + return ackSet.cardinality() >= ackQuorumSize; + } + + @Override + public boolean failBookieAndCheck(int bookieIndexHeardFrom, + BookieSocketAddress address) { + ackSet.clear(bookieIndexHeardFrom); + failureMap[bookieIndexHeardFrom] = address; + return failed() > (writeQuorumSize - ackQuorumSize); + } - @Override - public Map getFailedBookies() { - return ImmutableMap.copyOf(failureMap); + @Override + public Map getFailedBookies() { + ImmutableMap.Builder builder + = new ImmutableMap.Builder<>(); + for (int i = 0; i < failureMap.length; i++) { + if (failureMap[i] != null) { + builder.put(i, failureMap[i]); + } } + return builder.build(); + } - public boolean removeBookieAndCheck(int bookie) { - ackSet.remove(bookie); - failureMap.remove(bookie); - return ackSet.size() >= ackQuorumSize; + @Override + public boolean removeBookieAndCheck(int bookie) { + ackSet.clear(bookie); + failureMap[bookie] = null; + return ackSet.cardinality() >= ackQuorumSize; + } + + @Override + public void recycle() { + recyclerHandle.recycle(this); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("ackQuorumSize", ackQuorumSize) + .add("ackSet", ackSet) + .add("failureMap", failureMap).toString(); + } + + private int failed() { + int count = 0; + for (int i = 0; i < failureMap.length; i++) { + if (failureMap[i] != null) { + count++; + } } - }; + return count; + } } private class RRQuorumCoverageSet implements QuorumCoverageSet { @@ -136,6 +392,11 @@ public QuorumCoverageSet getCoverageSet() { @Override public boolean hasEntry(long entryId, int bookieIndex) { - return getWriteSet(entryId).contains(bookieIndex); + WriteSet w = getWriteSet(entryId); + try { + return w.contains(bookieIndex); + } finally { + w.recycle(); + } } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/TopologyAwareEnsemblePlacementPolicy.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/TopologyAwareEnsemblePlacementPolicy.java index f72b4561b20..c99b4fc662f 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/TopologyAwareEnsemblePlacementPolicy.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/TopologyAwareEnsemblePlacementPolicy.java @@ -452,20 +452,21 @@ public String toString() { } @Override - public List reorderReadSequence(ArrayList ensemble, List writeSet, Map bookieFailureHistory) { + public DistributionSchedule.WriteSet reorderReadSequence( + ArrayList ensemble, + Map bookieFailureHistory, + DistributionSchedule.WriteSet writeSet) { return writeSet; } @Override - public List reorderReadLACSequence(ArrayList ensemble, List writeSet, Map bookieFailureHistory) { - List retList = new ArrayList(reorderReadSequence(ensemble, writeSet, bookieFailureHistory)); - if (retList.size() < ensemble.size()) { - for (int i = 0; i < ensemble.size(); i++) { - if (!retList.contains(i)) { - retList.add(i); - } - } - } + public DistributionSchedule.WriteSet reorderReadLACSequence( + ArrayList ensemble, + Map bookieFailureHistory, + DistributionSchedule.WriteSet writeSet) { + DistributionSchedule.WriteSet retList = reorderReadSequence( + ensemble, bookieFailureHistory, writeSet); + retList.addMissingIndices(ensemble.size()); return retList; } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/RoundRobinDistributionScheduleTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/RoundRobinDistributionScheduleTest.java index c4cb49aea63..a4ef158fe0f 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/RoundRobinDistributionScheduleTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/RoundRobinDistributionScheduleTest.java @@ -21,12 +21,13 @@ package org.apache.bookkeeper.client; -import java.util.List; import java.util.Set; import java.util.HashSet; + import com.google.common.collect.Sets; import org.junit.Test; +import static org.apache.bookkeeper.client.RoundRobinDistributionSchedule.writeSetFromValues; import static org.junit.Assert.*; import org.slf4j.Logger; @@ -39,14 +40,17 @@ public class RoundRobinDistributionScheduleTest { public void testDistributionSchedule() throws Exception { RoundRobinDistributionSchedule schedule = new RoundRobinDistributionSchedule(3, 2, 5); - List wSet = schedule.getWriteSet(1); + DistributionSchedule.WriteSet wSet = schedule.getWriteSet(1); assertEquals("Write set is wrong size", wSet.size(), 3); - DistributionSchedule.AckSet ackSet = schedule.getAckSet(); - assertFalse("Shouldn't ack yet", ackSet.completeBookieAndCheck(wSet.get(0))); - assertFalse("Shouldn't ack yet", ackSet.completeBookieAndCheck(wSet.get(0))); - assertTrue("Should ack after 2 unique", ackSet.completeBookieAndCheck(wSet.get(2))); - assertTrue("Should still be acking", ackSet.completeBookieAndCheck(wSet.get(1))); + assertFalse("Shouldn't ack yet", + ackSet.completeBookieAndCheck(wSet.get(0))); + assertFalse("Shouldn't ack yet", + ackSet.completeBookieAndCheck(wSet.get(0))); + assertTrue("Should ack after 2 unique", + ackSet.completeBookieAndCheck(wSet.get(2))); + assertTrue("Should still be acking", + ackSet.completeBookieAndCheck(wSet.get(1))); } /** @@ -129,4 +133,27 @@ private int testCoverageForConfiguration(int ensemble, int writeQuorum, int ackQ } return errors; } + + @Test + public void testMoveAndShift() { + DistributionSchedule.WriteSet w = writeSetFromValues(1,2,3,4,5); + w.moveAndShift(3, 1); + assertEquals(w, writeSetFromValues(1,4,2,3,5)); + + w = writeSetFromValues(1,2,3,4,5); + w.moveAndShift(1, 3); + assertEquals(w, writeSetFromValues(1,3,4,2,5)); + + w = writeSetFromValues(1,2,3,4,5); + w.moveAndShift(0, 4); + assertEquals(w, writeSetFromValues(2,3,4,5,1)); + + w = writeSetFromValues(1,2,3,4,5); + w.moveAndShift(0, 0); + assertEquals(w, writeSetFromValues(1,2,3,4,5)); + + w = writeSetFromValues(1,2,3,4,5); + w.moveAndShift(4, 4); + assertEquals(w, writeSetFromValues(1,2,3,4,5)); + } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestLedgerChecker.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestLedgerChecker.java index 3bcac7d2f2e..442fca5379d 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestLedgerChecker.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestLedgerChecker.java @@ -365,8 +365,10 @@ public void testSingleEntryAfterEnsembleChange() throws Exception { } ArrayList firstEnsemble = lh.getLedgerMetadata() .getEnsembles().get(0L); - BookieSocketAddress lastBookieFromEnsemble = firstEnsemble.get( - lh.getDistributionSchedule().getWriteSet(lh.getLastAddPushed()).get(0)); + DistributionSchedule.WriteSet writeSet + = lh.getDistributionSchedule().getWriteSet(lh.getLastAddPushed()); + BookieSocketAddress lastBookieFromEnsemble + = firstEnsemble.get(writeSet.get(0)); LOG.info("Killing " + lastBookieFromEnsemble + " from ensemble=" + firstEnsemble); killBookie(lastBookieFromEnsemble); @@ -374,8 +376,9 @@ public void testSingleEntryAfterEnsembleChange() throws Exception { lh.addEntry(TEST_LEDGER_ENTRY_DATA); - lastBookieFromEnsemble = firstEnsemble.get( - lh.getDistributionSchedule().getWriteSet(lh.getLastAddPushed()).get(1)); + writeSet = lh.getDistributionSchedule().getWriteSet( + lh.getLastAddPushed()); + lastBookieFromEnsemble = firstEnsemble.get(writeSet.get(1)); LOG.info("Killing " + lastBookieFromEnsemble + " from ensemble=" + firstEnsemble); killBookie(lastBookieFromEnsemble); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawareEnsemblePlacementPolicy.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawareEnsemblePlacementPolicy.java index bbf95e96949..c79679a0265 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawareEnsemblePlacementPolicy.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawareEnsemblePlacementPolicy.java @@ -18,6 +18,8 @@ package org.apache.bookkeeper.client; import static org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicy.REPP_DNS_RESOLVER_CLASS; +import static org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicy.shuffleWithMask; +import static org.apache.bookkeeper.client.RoundRobinDistributionSchedule.writeSetFromValues; import static org.apache.bookkeeper.feature.SettableFeatureProvider.DISABLE_ALL; import java.net.InetAddress; @@ -53,7 +55,8 @@ public class TestRackawareEnsemblePlacementPolicy extends TestCase { RackawareEnsemblePlacementPolicy repp; final ArrayList ensemble = new ArrayList(); - final List writeSet = new ArrayList(); + DistributionSchedule.WriteSet writeSet + = DistributionSchedule.NULL_WRITE_SET; ClientConfiguration conf = new ClientConfiguration(); BookieSocketAddress addr1, addr2, addr3, addr4; io.netty.util.HashedWheelTimer timer; @@ -80,9 +83,7 @@ protected void setUp() throws Exception { ensemble.add(addr2); ensemble.add(addr3); ensemble.add(addr4); - for (int i = 0; i < 4; i++) { - writeSet.add(i); - } + writeSet = writeSetFromValues(0,1,2,3); timer = new HashedWheelTimer( new ThreadFactoryBuilder().setNameFormat("TestTimer-%d").build(), @@ -125,15 +126,15 @@ public void testNodeDown() throws Exception { addrs.remove(addr1); repp.onClusterChanged(addrs, new HashSet()); - List reoderSet = repp.reorderReadSequence(ensemble, writeSet, new HashMap()); - List expectedSet = new ArrayList(); - expectedSet.add(1); - expectedSet.add(2); - expectedSet.add(3); - expectedSet.add(0); - LOG.info("reorder set : {}", reoderSet); - assertFalse(reoderSet == writeSet); - assertEquals(expectedSet, reoderSet); + DistributionSchedule.WriteSet origWriteSet = writeSet.copy(); + DistributionSchedule.WriteSet reorderSet = repp.reorderReadSequence( + ensemble, new HashMap(), + writeSet); + DistributionSchedule.WriteSet expectedSet + = writeSetFromValues(1, 2, 3, 0); + LOG.info("reorder set : {}", reorderSet); + assertFalse(reorderSet.equals(origWriteSet)); + assertEquals(expectedSet, reorderSet); } @Test @@ -157,15 +158,14 @@ public void testNodeReadOnly() throws Exception { ro.add(addr1); repp.onClusterChanged(addrs, ro); - List reoderSet = repp.reorderReadSequence(ensemble, writeSet, new HashMap()); - List expectedSet = new ArrayList(); - expectedSet.add(1); - expectedSet.add(2); - expectedSet.add(3); - expectedSet.add(0); - LOG.info("reorder set : {}", reoderSet); - assertFalse(reoderSet == writeSet); - assertEquals(expectedSet, reoderSet); + DistributionSchedule.WriteSet origWriteSet = writeSet.copy(); + DistributionSchedule.WriteSet reorderSet = repp.reorderReadSequence( + ensemble, new HashMap(), writeSet); + DistributionSchedule.WriteSet expectedSet + = writeSetFromValues(1, 2, 3, 0); + LOG.info("reorder set : {}", reorderSet); + assertFalse(reorderSet.equals(origWriteSet)); + assertEquals(expectedSet, reorderSet); } @Test @@ -188,15 +188,14 @@ public void testTwoNodesDown() throws Exception { addrs.remove(addr2); repp.onClusterChanged(addrs, new HashSet()); - List reoderSet = repp.reorderReadSequence(ensemble, writeSet, new HashMap()); - List expectedSet = new ArrayList(); - expectedSet.add(2); - expectedSet.add(3); - expectedSet.add(0); - expectedSet.add(1); - LOG.info("reorder set : {}", reoderSet); - assertFalse(reoderSet == writeSet); - assertEquals(expectedSet, reoderSet); + DistributionSchedule.WriteSet origWriteSet = writeSet.copy(); + DistributionSchedule.WriteSet reorderSet = repp.reorderReadSequence( + ensemble, new HashMap(), writeSet); + DistributionSchedule.WriteSet expectedSet + = writeSetFromValues(2, 3, 0, 1); + LOG.info("reorder set : {}", reorderSet); + assertFalse(reorderSet.equals(origWriteSet)); + assertEquals(expectedSet, reorderSet); } @Test @@ -220,14 +219,13 @@ public void testNodeDownAndReadOnly() throws Exception { Set roAddrs = new HashSet(); roAddrs.add(addr2); repp.onClusterChanged(addrs, roAddrs); - List reoderSet = repp.reorderReadSequence(ensemble, writeSet, new HashMap()); - List expectedSet = new ArrayList(); - expectedSet.add(2); - expectedSet.add(3); - expectedSet.add(1); - expectedSet.add(0); - assertFalse(reoderSet == writeSet); - assertEquals(expectedSet, reoderSet); + DistributionSchedule.WriteSet origWriteSet = writeSet.copy(); + DistributionSchedule.WriteSet reorderSet = repp.reorderReadSequence( + ensemble, new HashMap(), writeSet); + DistributionSchedule.WriteSet expectedSet + = writeSetFromValues(2, 3, 1, 0); + assertFalse(reorderSet.equals(origWriteSet)); + assertEquals(expectedSet, reorderSet); } @Test @@ -716,7 +714,8 @@ public void testNodeWithFailures() throws Exception { bookieFailures.put(addr1, 20L); bookieFailures.put(addr2, 22L); - List reoderSet = repp.reorderReadSequence(ensemble, writeSet, bookieFailures); + DistributionSchedule.WriteSet reoderSet = repp.reorderReadSequence( + ensemble, bookieFailures, writeSet); LOG.info("reorder set : {}", reoderSet); assertEquals(ensemble.get(reoderSet.get(2)), addr1); assertEquals(ensemble.get(reoderSet.get(3)), addr2); @@ -758,4 +757,109 @@ public void testPlacementOnStabilizeNetworkTopology() throws Exception { ArrayList ensemble = repp.newEnsemble(4, 4, 4, null, new HashSet()); assertTrue(ensemble.contains(addr4)); } + + @Test + public void testShuffleWithMask() { + int mask = 0xE1 << 16; + int maskBits = 0xFF << 16; + boolean shuffleOccurred = false; + + for (int i = 0; i < 100; i++) { + DistributionSchedule.WriteSet w = writeSetFromValues( + 1, 2, 3 & mask, 4 & mask, 5 & mask, 6); + shuffleWithMask(w, mask, maskBits); + assertEquals(w.get(0), 1); + assertEquals(w.get(1), 2); + assertEquals(w.get(5), 6); + + if (w.get(3) == (3 & mask) + || w.get(4) == (3 & mask)) { + shuffleOccurred = true; + } else if (w.get(2) != (3 & mask)) { + fail("3 not found"); + } + + if (w.get(2) == (4 & mask) + || w.get(4) == (4 & mask)) { + shuffleOccurred = true; + } else if (w.get(3) != (4 & mask)) { + fail("4 not found"); + } + + if (w.get(2) == (5 & mask) + || w.get(3) == (5 & mask)) { + shuffleOccurred = true; + } else if (w.get(4) != (5 & mask)) { + fail("5 not found"); + } + } + assertTrue(shuffleOccurred); + + // at start of array + shuffleOccurred = false; + for (int i = 0; i < 100; i++) { + DistributionSchedule.WriteSet w = writeSetFromValues( + 1 & mask, 2 & mask, 3 & mask, 4, 5, 6); + shuffleWithMask(w, mask, maskBits); + assertEquals(w.get(3), 4); + assertEquals(w.get(4), 5); + assertEquals(w.get(5), 6); + + if (w.get(1) == (1 & mask) + || w.get(2) == (1 & mask)) { + shuffleOccurred = true; + } else if (w.get(0) != (1 & mask)) { + fail("1 not found"); + } + + if (w.get(0) == (2 & mask) + || w.get(2) == (2 & mask)) { + shuffleOccurred = true; + } else if (w.get(1) != (2 & mask)) { + fail("2 not found"); + } + + if (w.get(0) == (3 & mask) + || w.get(1) == (3 & mask)) { + shuffleOccurred = true; + } else if (w.get(2) != (3 & mask)) { + fail("3 not found"); + } + } + assertTrue(shuffleOccurred); + + // at end of array + shuffleOccurred = false; + for (int i = 0; i < 100; i++) { + DistributionSchedule.WriteSet w = writeSetFromValues( + 1, 2, 3, 4 & mask, 5 & mask, 6 & mask); + shuffleWithMask(w, mask, maskBits); + assertEquals(w.get(0), 1); + assertEquals(w.get(1), 2); + assertEquals(w.get(2), 3); + + if (w.get(4) == (4 & mask) + || w.get(5) == (4 & mask)) { + shuffleOccurred = true; + } else if (w.get(3) != (4 & mask)) { + fail("4 not found"); + } + + if (w.get(3) == (5 & mask) + || w.get(5) == (5 & mask)) { + shuffleOccurred = true; + } else if (w.get(4) != (5 & mask)) { + fail("5 not found"); + } + + if (w.get(3) == (6 & mask) + || w.get(4) == (6 & mask)) { + shuffleOccurred = true; + } else if (w.get(5) != (6 & mask)) { + fail("6 not found"); + } + } + assertTrue(shuffleOccurred); + } + } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRegionAwareEnsemblePlacementPolicy.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRegionAwareEnsemblePlacementPolicy.java index da11e140dd9..d2cfcc4022a 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRegionAwareEnsemblePlacementPolicy.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRegionAwareEnsemblePlacementPolicy.java @@ -19,12 +19,14 @@ import java.net.InetAddress; import java.util.ArrayList; +import java.util.Arrays; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Set; import java.util.concurrent.TimeUnit; +import com.google.common.primitives.Ints; import com.google.common.util.concurrent.ThreadFactoryBuilder; import io.netty.util.HashedWheelTimer; @@ -47,7 +49,9 @@ import junit.framework.TestCase; + import static org.apache.bookkeeper.client.RegionAwareEnsemblePlacementPolicy.*; +import static org.apache.bookkeeper.client.RoundRobinDistributionSchedule.writeSetFromValues; import static org.apache.bookkeeper.feature.SettableFeatureProvider.DISABLE_ALL; public class TestRegionAwareEnsemblePlacementPolicy extends TestCase { @@ -57,7 +61,7 @@ public class TestRegionAwareEnsemblePlacementPolicy extends TestCase { RegionAwareEnsemblePlacementPolicy repp; final ClientConfiguration conf = new ClientConfiguration(); final ArrayList ensemble = new ArrayList(); - final List writeSet = new ArrayList(); + DistributionSchedule.WriteSet writeSet = DistributionSchedule.NULL_WRITE_SET; BookieSocketAddress addr1, addr2, addr3, addr4; HashedWheelTimer timer; @@ -89,9 +93,8 @@ protected void setUp() throws Exception { ensemble.add(addr2); ensemble.add(addr3); ensemble.add(addr4); - for (int i = 0; i < 4; i++) { - writeSet.add(i); - } + + writeSet = writeSetFromValues(0,1,2,3); timer = new HashedWheelTimer( new ThreadFactoryBuilder().setNameFormat("TestTimer-%d").build(), @@ -116,9 +119,10 @@ public void testNotReorderReadIfInDefaultRack() throws Exception { repp = new RegionAwareEnsemblePlacementPolicy(); repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); - List reorderSet = repp.reorderReadSequence(ensemble, writeSet, new HashMap()); - assertFalse(reorderSet == writeSet); - assertEquals(writeSet, reorderSet); + DistributionSchedule.WriteSet origWriteSet = writeSet.copy(); + DistributionSchedule.WriteSet reorderSet = repp.reorderReadSequence( + ensemble, new HashMap(), writeSet); + assertEquals(origWriteSet, reorderSet); } @Test @@ -139,14 +143,14 @@ public void testNodeInSameRegion() throws Exception { addrs.add(addr4); repp.onClusterChanged(addrs, new HashSet()); - List reorderSet = repp.reorderReadSequence(ensemble, writeSet, new HashMap()); - List expectedSet = new ArrayList(); - expectedSet.add(0); - expectedSet.add(3); - expectedSet.add(1); - expectedSet.add(2); + DistributionSchedule.WriteSet reorderSet = repp.reorderReadSequence( + ensemble, new HashMap(), writeSet.copy()); + DistributionSchedule.WriteSet expectedSet = writeSetFromValues(0, 3, 1, 2); + LOG.info("write set : {}", writeSet); LOG.info("reorder set : {}", reorderSet); - assertFalse(reorderSet == writeSet); + LOG.info("expected set : {}", expectedSet); + LOG.info("reorder equals {}", reorderSet.equals(writeSet)); + assertFalse(reorderSet.equals(writeSet)); assertEquals(expectedSet, reorderSet); } @@ -158,10 +162,11 @@ public void testNodeNotInSameRegions() throws Exception { repp = new RegionAwareEnsemblePlacementPolicy(); repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL, NullStatsLogger.INSTANCE); - List reoderSet = repp.reorderReadSequence(ensemble, writeSet, new HashMap()); - LOG.info("reorder set : {}", reoderSet); - assertFalse(reoderSet == writeSet); - assertEquals(writeSet, reoderSet); + DistributionSchedule.WriteSet origWriteSet = writeSet.copy(); + DistributionSchedule.WriteSet reorderSet = repp.reorderReadSequence( + ensemble, new HashMap(), writeSet); + LOG.info("reorder set : {}", reorderSet); + assertEquals(origWriteSet, reorderSet); } @Test @@ -182,15 +187,14 @@ public void testNodeDown() throws Exception { addrs.remove(addr1); repp.onClusterChanged(addrs, new HashSet()); - List reoderSet = repp.reorderReadSequence(ensemble, writeSet, new HashMap()); - List expectedSet = new ArrayList(); - expectedSet.add(3); - expectedSet.add(1); - expectedSet.add(2); - expectedSet.add(0); - LOG.info("reorder set : {}", reoderSet); - assertFalse(reoderSet == writeSet); - assertEquals(expectedSet, reoderSet); + DistributionSchedule.WriteSet origWriteSet = writeSet.copy(); + DistributionSchedule.WriteSet reorderSet = repp.reorderReadSequence( + ensemble, new HashMap(), writeSet); + DistributionSchedule.WriteSet expectedSet + = writeSetFromValues(3, 1, 2, 0); + LOG.info("reorder set : {}", reorderSet); + assertFalse(reorderSet.equals(origWriteSet)); + assertEquals(expectedSet, reorderSet); } @Test @@ -213,15 +217,14 @@ public void testNodeReadOnly() throws Exception { ro.add(addr1); repp.onClusterChanged(addrs, ro); - List reoderSet = repp.reorderReadSequence(ensemble, writeSet, new HashMap()); - List expectedSet = new ArrayList(); - expectedSet.add(3); - expectedSet.add(1); - expectedSet.add(2); - expectedSet.add(0); - LOG.info("reorder set : {}", reoderSet); - assertFalse(reoderSet == writeSet); - assertEquals(expectedSet, reoderSet); + DistributionSchedule.WriteSet origWriteSet = writeSet.copy(); + DistributionSchedule.WriteSet reorderSet = repp.reorderReadSequence( + ensemble, new HashMap(), writeSet); + DistributionSchedule.WriteSet expectedSet + = writeSetFromValues(3, 1, 2, 0); + LOG.info("reorder set : {}", reorderSet); + assertFalse(reorderSet.equals(origWriteSet)); + assertEquals(expectedSet, reorderSet); } @Test @@ -243,15 +246,14 @@ public void testTwoNodesDown() throws Exception { addrs.remove(addr2); repp.onClusterChanged(addrs, new HashSet()); - List reoderSet = repp.reorderReadSequence(ensemble, writeSet, new HashMap()); - List expectedSet = new ArrayList(); - expectedSet.add(3); - expectedSet.add(2); - expectedSet.add(0); - expectedSet.add(1); - LOG.info("reorder set : {}", reoderSet); - assertFalse(reoderSet == writeSet); - assertEquals(expectedSet, reoderSet); + DistributionSchedule.WriteSet origWriteSet = writeSet.copy(); + DistributionSchedule.WriteSet reorderSet = repp.reorderReadSequence( + ensemble, new HashMap(), writeSet); + DistributionSchedule.WriteSet expectedSet + = writeSetFromValues(3, 2, 0, 1); + LOG.info("reorder set : {}", reorderSet); + assertFalse(reorderSet.equals(origWriteSet)); + assertEquals(expectedSet, reorderSet); } @Test @@ -1038,15 +1040,20 @@ private void basicReorderReadSequenceWithLocalRegionTest(String myRegion, boolea int ensembleSize = ensemble.size(); for (int i = 0; i < ensembleSize; i++) { - List writeSet = ds.getWriteSet(i); - List readSet; + DistributionSchedule.WriteSet writeSet = ds.getWriteSet(i); + DistributionSchedule.WriteSet origWriteSet = writeSet.copy(); + DistributionSchedule.WriteSet readSet; if (isReadLAC) { - readSet = repp.reorderReadLACSequence(ensemble, writeSet, new HashMap()); + readSet = repp.reorderReadLACSequence( + ensemble, + new HashMap(), writeSet); } else { - readSet = repp.reorderReadSequence(ensemble, writeSet, new HashMap()); + readSet = repp.reorderReadSequence( + ensemble, + new HashMap(), writeSet); } - LOG.info("Reorder {} => {}.", writeSet, readSet); + LOG.info("Reorder {} => {}.", origWriteSet, readSet); // first few nodes less than REMOTE_NODE_IN_REORDER_SEQUENCE should be local region int k = 0; @@ -1089,13 +1096,19 @@ private void basicReorderReadSequenceWithRemoteRegionTest(String myRegion, boole int ensembleSize = ensemble.size(); for (int i = 0; i < ensembleSize; i++) { - List writeSet = ds.getWriteSet(i); - List readSet; + DistributionSchedule.WriteSet writeSet = ds.getWriteSet(i); + DistributionSchedule.WriteSet readSet; if (isReadLAC) { - readSet = repp.reorderReadLACSequence(ensemble, writeSet, new HashMap()); + readSet = repp.reorderReadLACSequence( + ensemble, + new HashMap(), + writeSet.copy()); } else { - readSet = repp.reorderReadSequence(ensemble, writeSet, new HashMap()); + readSet = repp.reorderReadSequence( + ensemble, + new HashMap(), + writeSet.copy()); } assertEquals(writeSet, readSet); @@ -1124,10 +1137,11 @@ static Set getBookiesForRegion(ArrayList ensemble, - List writeSet, + DistributionSchedule.WriteSet writeSet, String region, List finalSet) { - for (int bi : writeSet) { + for (int i = 0; i < writeSet.size(); i++) { + int bi = writeSet.get(i); String r = StaticDNSResolver.getRegion(ensemble.get(bi).getHostName()); if (r.equals(region)) { finalSet.add(bi); @@ -1159,12 +1173,16 @@ private void reorderReadSequenceWithUnavailableOrReadOnlyBookiesTest(boolean isR int ensembleSize = ensemble.size(); for (int i = 0; i < ensembleSize; i++) { - List writeSet = ds.getWriteSet(i); - List readSet; + DistributionSchedule.WriteSet writeSet = ds.getWriteSet(i); + DistributionSchedule.WriteSet readSet; if (isReadLAC) { - readSet = repp.reorderReadLACSequence(ensemble, writeSet, new HashMap()); + readSet = repp.reorderReadLACSequence( + ensemble, new HashMap(), + writeSet.copy()); } else { - readSet = repp.reorderReadSequence(ensemble, writeSet, new HashMap()); + readSet = repp.reorderReadSequence( + ensemble, new HashMap(), + writeSet.copy()); } LOG.info("Reorder {} => {}.", writeSet, readSet); @@ -1176,8 +1194,10 @@ private void reorderReadSequenceWithUnavailableOrReadOnlyBookiesTest(boolean isR appendBookieIndexByRegion(ensemble, writeSet, readOnlyRegion, expectedReadSet); // unavailable bookies appendBookieIndexByRegion(ensemble, writeSet, unavailableRegion, expectedReadSet); - - assertEquals(expectedReadSet, readSet); + assertEquals(expectedReadSet.size(), readSet.size()); + for (int j = 0; j < expectedReadSet.size(); j++) { + assertEquals(expectedReadSet.get(j).intValue(), readSet.get(j)); + } } } @@ -1229,9 +1249,8 @@ public void testNodeWithFailures() throws Exception { ensemble.add(addr7); ensemble.add(addr8); - for (int i = 4; i < 8; i++) { - writeSet.add(i); - } + DistributionSchedule.WriteSet writeSet2 + = writeSetFromValues(0,1,2,3,4,5,6,7); Set addrs = new HashSet(); addrs.add(addr1); @@ -1251,7 +1270,9 @@ public void testNodeWithFailures() throws Exception { bookieFailures.put(addr3, 24L); bookieFailures.put(addr4, 25L); - List reoderSet = repp.reorderReadSequence(ensemble, writeSet, bookieFailures); + LOG.info("write set : {}", writeSet2); + DistributionSchedule.WriteSet reoderSet = repp.reorderReadSequence( + ensemble, bookieFailures, writeSet2); LOG.info("reorder set : {}", reoderSet); assertEquals(ensemble.get(reoderSet.get(0)), addr6); assertEquals(ensemble.get(reoderSet.get(1)), addr7); From 19825a63f21566927da153498f4680f4d89d45cd Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Mon, 30 Oct 2017 20:49:34 +0100 Subject: [PATCH 09/15] Recycle instances of callback to obtain a PerChannelBookie client This change was originally commit e2e77863 in the yahoo-4.3 branch. Author: Matteo Merli Reviewers: Sijie Guo This closes #679 from ivankelly/yahoo-bp-3 --- .../apache/bookkeeper/proto/BookieClient.java | 86 ++++++++++++++++--- .../proto/PerChannelBookieClient.java | 2 +- 2 files changed, 73 insertions(+), 15 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClient.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClient.java index c604b16844a..ae7f806ab4f 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClient.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClient.java @@ -60,6 +60,8 @@ import io.netty.channel.EventLoopGroup; import io.netty.channel.nio.NioEventLoopGroup; import io.netty.util.HashedWheelTimer; +import io.netty.util.Recycler; +import io.netty.util.Recycler.Handle; import io.netty.util.Timeout; import io.netty.util.TimerTask; @@ -253,26 +255,19 @@ public void addEntry(final BookieSocketAddress addr, return; } - // Retain the buffer, since the connection could be obtained after the PendingApp might have already - // failed + // Retain the buffer, since the connection could be obtained after + // the PendingApp might have already failed toSend.retain(); - client.obtain(new GenericCallback() { - @Override - public void operationComplete(final int rc, PerChannelBookieClient pcbc) { - if (rc != BKException.Code.OK) { - completeAdd(rc, ledgerId, entryId, addr, cb, ctx); - } else { - pcbc.addEntry(ledgerId, masterKey, entryId, toSend, cb, ctx, options); - } - toSend.release(); - } - }, ledgerId); + client.obtain(ChannelReadyForAddEntryCallback.create( + this, toSend, ledgerId, entryId, addr, + ctx, cb, options, masterKey), + ledgerId); } finally { closeLock.readLock().unlock(); } } - + private void completeRead(final int rc, final long ledgerId, final long entryId, @@ -292,6 +287,69 @@ public void safeRun() { } } + private static class ChannelReadyForAddEntryCallback + implements GenericCallback { + private final Handle recyclerHandle; + + private BookieClient bookieClient; + private ByteBuf toSend; + private long ledgerId; + private long entryId; + private BookieSocketAddress addr; + private Object ctx; + private WriteCallback cb; + private int options; + private byte[] masterKey; + + static ChannelReadyForAddEntryCallback create( + BookieClient bookieClient, ByteBuf toSend, long ledgerId, + long entryId, BookieSocketAddress addr, Object ctx, + WriteCallback cb, int options, byte[] masterKey) { + ChannelReadyForAddEntryCallback callback = RECYCLER.get(); + callback.bookieClient = bookieClient; + callback.toSend = toSend; + callback.ledgerId = ledgerId; + callback.entryId = entryId; + callback.addr = addr; + callback.ctx = ctx; + callback.cb = cb; + callback.options = options; + callback.masterKey = masterKey; + return callback; + } + + @Override + public void operationComplete(final int rc, + PerChannelBookieClient pcbc) { + if (rc != BKException.Code.OK) { + bookieClient.completeAdd(rc, ledgerId, entryId, addr, cb, ctx); + } else { + pcbc.addEntry(ledgerId, masterKey, entryId, + toSend, cb, ctx, options); + } + + toSend.release(); + recycle(); + } + + private ChannelReadyForAddEntryCallback( + Handle recyclerHandle) { + this.recyclerHandle = recyclerHandle; + } + + private static final Recycler RECYCLER + = new Recycler() { + protected ChannelReadyForAddEntryCallback newObject( + Recycler.Handle recyclerHandle) { + return new ChannelReadyForAddEntryCallback(recyclerHandle); + } + }; + + public void recycle() { + recyclerHandle.recycle(this); + } + } + public void readEntryAndFenceLedger(final BookieSocketAddress addr, final long ledgerId, final byte[] masterKey, diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java index ffa030b56a9..f74a4c96ba7 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java @@ -1164,7 +1164,7 @@ public void operationComplete(Future future) throws Exception { abstract class CompletionValue { private final OpStatsLogger opLogger; private final OpStatsLogger timeoutOpLogger; - private final String operationName; + private final String operationName; // adding a comment protected Object ctx; protected long ledgerId; protected long entryId; From aa07dd08294ceb12a19304670271401b8405af32 Mon Sep 17 00:00:00 2001 From: Ivan Kelly Date: Thu, 2 Nov 2017 11:40:21 +0100 Subject: [PATCH 10/15] Adding another comment (fixup) --- .../org/apache/bookkeeper/proto/PerChannelBookieClient.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java index f74a4c96ba7..01ef079b16d 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java @@ -1169,7 +1169,7 @@ abstract class CompletionValue { protected long ledgerId; protected long entryId; protected long startTime; - protected Timeout timeout; + protected Timeout timeout; // another comment public CompletionValue(String operationName, Object ctx, From 326c1ecaa22e77ef39a0b1b0cee0d4c38fe17a37 Mon Sep 17 00:00:00 2001 From: eolivelli Date: Mon, 30 Oct 2017 22:26:22 -0700 Subject: [PATCH 11/15] Copy site from 4.5.0 to 4.5.1 Copy site directory in order to start new release notes for 4.5.1 Author: eolivelli Reviewers: Sijie Guo This closes #680 from eolivelli/release-notes-451-copy --- site/docs/4.5.1/admin/autorecovery.md | 128 +++++ site/docs/4.5.1/admin/bookies.md | 180 +++++++ site/docs/4.5.1/admin/geo-replication.md | 22 + site/docs/4.5.1/admin/metrics.md | 41 ++ site/docs/4.5.1/admin/perf.md | 3 + site/docs/4.5.1/admin/placement.md | 3 + site/docs/4.5.1/admin/upgrade.md | 73 +++ site/docs/4.5.1/api/distributedlog-api.md | 395 ++++++++++++++ site/docs/4.5.1/api/ledger-adv-api.md | 82 +++ site/docs/4.5.1/api/ledger-api.md | 473 +++++++++++++++++ site/docs/4.5.1/api/overview.md | 17 + site/docs/4.5.1/deployment/dcos.md | 142 +++++ site/docs/4.5.1/deployment/kubernetes.md | 4 + site/docs/4.5.1/deployment/manual.md | 56 ++ site/docs/4.5.1/development/codebase.md | 3 + site/docs/4.5.1/development/protocol.md | 148 ++++++ site/docs/4.5.1/example.md | 6 + site/docs/4.5.1/getting-started/concepts.md | 202 +++++++ .../4.5.1/getting-started/installation.md | 74 +++ .../docs/4.5.1/getting-started/run-locally.md | 16 + site/docs/4.5.1/overview/overview.md | 57 ++ site/docs/4.5.1/overview/releaseNotes.md | 501 ++++++++++++++++++ .../4.5.1/overview/releaseNotesTemplate.md | 17 + site/docs/4.5.1/reference/cli.md | 10 + site/docs/4.5.1/reference/config.md | 9 + site/docs/4.5.1/reference/metrics.md | 3 + site/docs/4.5.1/security/overview.md | 21 + site/docs/4.5.1/security/sasl.md | 202 +++++++ site/docs/4.5.1/security/tls.md | 210 ++++++++ site/docs/4.5.1/security/zookeeper.md | 41 ++ 30 files changed, 3139 insertions(+) create mode 100644 site/docs/4.5.1/admin/autorecovery.md create mode 100644 site/docs/4.5.1/admin/bookies.md create mode 100644 site/docs/4.5.1/admin/geo-replication.md create mode 100644 site/docs/4.5.1/admin/metrics.md create mode 100644 site/docs/4.5.1/admin/perf.md create mode 100644 site/docs/4.5.1/admin/placement.md create mode 100644 site/docs/4.5.1/admin/upgrade.md create mode 100644 site/docs/4.5.1/api/distributedlog-api.md create mode 100644 site/docs/4.5.1/api/ledger-adv-api.md create mode 100644 site/docs/4.5.1/api/ledger-api.md create mode 100644 site/docs/4.5.1/api/overview.md create mode 100644 site/docs/4.5.1/deployment/dcos.md create mode 100644 site/docs/4.5.1/deployment/kubernetes.md create mode 100644 site/docs/4.5.1/deployment/manual.md create mode 100644 site/docs/4.5.1/development/codebase.md create mode 100644 site/docs/4.5.1/development/protocol.md create mode 100644 site/docs/4.5.1/example.md create mode 100644 site/docs/4.5.1/getting-started/concepts.md create mode 100644 site/docs/4.5.1/getting-started/installation.md create mode 100644 site/docs/4.5.1/getting-started/run-locally.md create mode 100644 site/docs/4.5.1/overview/overview.md create mode 100644 site/docs/4.5.1/overview/releaseNotes.md create mode 100644 site/docs/4.5.1/overview/releaseNotesTemplate.md create mode 100644 site/docs/4.5.1/reference/cli.md create mode 100644 site/docs/4.5.1/reference/config.md create mode 100644 site/docs/4.5.1/reference/metrics.md create mode 100644 site/docs/4.5.1/security/overview.md create mode 100644 site/docs/4.5.1/security/sasl.md create mode 100644 site/docs/4.5.1/security/tls.md create mode 100644 site/docs/4.5.1/security/zookeeper.md diff --git a/site/docs/4.5.1/admin/autorecovery.md b/site/docs/4.5.1/admin/autorecovery.md new file mode 100644 index 00000000000..bd11a8886d3 --- /dev/null +++ b/site/docs/4.5.1/admin/autorecovery.md @@ -0,0 +1,128 @@ +--- +title: Using AutoRecovery +--- + +When a {% pop bookie %} crashes, all {% pop ledgers %} on that bookie become under-replicated. In order to bring all ledgers in your BookKeeper cluster back to full replication, you'll need to *recover* the data from any offline bookies. There are two ways to recover bookies' data: + +1. Using [manual recovery](#manual-recovery) +1. Automatically, using [*AutoRecovery*](#autorecovery) + +## Manual recovery + +You can manually recover failed bookies using the [`bookkeeper`](../../reference/cli) command-line tool. You need to specify: + +* that the `org.apache.bookkeeper.tools.BookKeeperTools` class needs to be run +* an IP and port for your BookKeeper cluster's ZooKeeper ensemble +* the IP and port for the failed bookie + +Here's an example: + +```bash +$ bookkeeper-server/bin/bookkeeper org.apache.bookkeeper.tools.BookKeeperTools \ + zk1.example.com:2181 \ # IP and port for ZooKeeper + 192.168.1.10:3181 # IP and port for the failed bookie +``` + +If you wish, you can also specify which bookie you'd like to rereplicate to. Here's an example: + +```bash +$ bookkeeper-server/bin/bookkeeper org.apache.bookkeeper.tools.BookKeeperTools \ + zk1.example.com:2181 \ # IP and port for ZooKeeper + 192.168.1.10:3181 \ # IP and port for the failed bookie + 192.168.1.11:3181 # IP and port for the bookie to rereplicate to +``` + +### The manual recovery process + +When you initiate a manual recovery process, the following happens: + +1. The client (the process running ) reads the metadata of active ledgers from ZooKeeper. +1. The ledgers that contain fragments from the failed bookie in their ensemble are selected. +1. A recovery process is initiated for each ledger in this list and the rereplication process is run for each ledger. +1. Once all the ledgers are marked as fully replicated, bookie recovery is finished. + +## AutoRecovery + +AutoRecovery is a process that: + +* automatically detects when a {% pop bookie %} in your BookKeeper cluster has become unavailable and then +* rereplicates all the {% pop ledgers %} that were stored on that bookie. + +AutoRecovery can be run in two ways: + +1. On dedicated nodes in your BookKeeper cluster +1. On the same machines on which your bookies are running + +## Running AutoRecovery + +You can start up AutoRecovery using the [`autorecovery`](../../reference/cli#bookkeeper-autorecovery) command of the [`bookkeeper`](../../reference/cli) CLI tool. + +```bash +$ bookkeeper-server/bin/bookkeeper autorecovery +``` + +> The most important thing to ensure when starting up AutoRecovery is that the ZooKeeper connection string specified by the [`zkServers`](../../reference/config#zkServers) parameter points to the right ZooKeeper cluster. + +If you start up AutoRecovery on a machine that is already running a bookie, then the AutoRecovery process will run alongside the bookie on a separate thread. + +You can also start up AutoRecovery on a fresh machine if you'd like to create a dedicated cluster of AutoRecovery nodes. + +## Configuration + +There are a handful of AutoRecovery-related configs in the [`bk_server.conf`](../../reference/config) configuration file. For a listing of those configs, see [AutoRecovery settings](../../reference/config#autorecovery-settings). + +## Disable AutoRecovery + +You can disable AutoRecovery at any time, for example during maintenance. Disabling AutoRecovery ensures that bookies' data isn't unnecessarily rereplicated when the bookie is only taken down for a short period of time, for example when the bookie is being updated or the configuration if being changed. + +You can disable AutoRecover using the [`bookkeeper`](../../reference/cli#bookkeeper-shell-autorecovery) CLI tool: + +```bash +$ bookkeeper-server/bin/bookkeeper shell autorecovery -disable +``` + +Once disabled, you can reenable AutoRecovery using the [`enable`](../../reference/cli#bookkeeper-shell-autorecovery) shell command: + +```bash +$ bookkeeper-server/bin/bookkeeper shell autorecovery -enable +``` + +## AutoRecovery architecture + +AutoRecovery has two components: + +1. The [**auditor**](#auditor) (see the [`Auditor`](../../api/javadoc/org/apache/bookkeeper/replication/Auditor.html) class) is a singleton node that watches bookies to see if they fail and creates rereplication tasks for the ledgers on failed bookies. +1. The [**replication worker**](#replication-worker) (see the [`ReplicationWorker`](../../api/javadoc/org/apache/bookkeeper/replication/ReplicationWorker.html) class) runs on each bookie and executes rereplication tasks provided by the auditor. + +Both of these components run as threads in the [`AutoRecoveryMain`](../../api/javadoc/org/apache/bookkeeper/replication/AutoRecoveryMain) process, which runs on each bookie in the cluster. All recovery nodes participate in leader election---using ZooKeeper---to decide which node becomes the auditor. Nodes that fail to become the auditor watch the elected auditor and run an election process again if they see that the auditor node has failed. + +### Auditor + +The auditor watches all bookies in the cluster that are registered with ZooKeeper. Bookies register with ZooKeeper at startup. If the bookie crashes or is killed, the bookie's registration in ZooKeeper disappears and the auditor is notified of the change in the list of registered bookies. + +When the auditor sees that a bookie has disappeared, it immediately scans the complete {% pop ledger %} list to find ledgers that have data stored on the failed bookie. Once it has a list of ledgers for that bookie, the auditor will publish a rereplication task for each ledger under the `/underreplicated/` [znode](https://zookeeper.apache.org/doc/current/zookeeperOver.html) in ZooKeeper. + +### Replication Worker + +Each replication worker watches for tasks being published by the auditor on the `/underreplicated/` znode in ZooKeeper. When a new task appears, the replication worker will try to get a lock on it. If it cannot acquire the lock, it will try the next entry. The locks are implemented using ZooKeeper ephemeral znodes. + +The replication worker will scan through the rereplication task's ledger for fragments of which its local bookie is not a member. When it finds fragments matching this criterion, it will replicate the entries of that fragment to the local bookie. If, after this process, the ledger is fully replicated, the ledgers entry under /underreplicated/ is deleted, and the lock is released. If there is a problem replicating, or there are still fragments in the ledger which are still underreplicated (due to the local bookie already being part of the ensemble for the fragment), then the lock is simply released. + +If the replication worker finds a fragment which needs rereplication, but does not have a defined endpoint (i.e. the final fragment of a ledger currently being written to), it will wait for a grace period before attempting rereplication. If the fragment needing rereplication still does not have a defined endpoint, the ledger is fenced and rereplication then takes place. + +This avoids the situation in which a client is writing to a ledger and one of the bookies goes down, but the client has not written an entry to that bookie before rereplication takes place. The client could continue writing to the old fragment, even though the ensemble for the fragment had changed. This could lead to data loss. Fencing prevents this scenario from happening. In the normal case, the client will try to write to the failed bookie within the grace period, and will have started a new fragment before rereplication starts. + +You can configure this grace period using the [`openLedgerRereplicationGracePeriod`](../../reference/config#openLedgerRereplicationGracePeriod) parameter. + +### The rereplication process + +The ledger rereplication process happens in these steps: + +1. The client goes through all ledger fragments in the ledger, selecting those that contain the failed bookie. +1. A recovery process is initiated for each ledger fragment in this list. + 1. The client selects a bookie to which all entries in the ledger fragment will be replicated; In the case of autorecovery, this will always be the local bookie. + 1. The client reads entries that belong to the ledger fragment from other bookies in the ensemble and writes them to the selected bookie. + 1. Once all entries have been replicated, the zookeeper metadata for the fragment is updated to reflect the new ensemble. + 1. The fragment is marked as fully replicated in the recovery tool. +1. Once all ledger fragments are marked as fully replicated, the ledger is marked as fully replicated. + diff --git a/site/docs/4.5.1/admin/bookies.md b/site/docs/4.5.1/admin/bookies.md new file mode 100644 index 00000000000..f9b1dcf94dc --- /dev/null +++ b/site/docs/4.5.1/admin/bookies.md @@ -0,0 +1,180 @@ +--- +title: BookKeeper administration +subtitle: A guide to deploying and administering BookKeeper +--- + +This document is a guide to deploying, administering, and maintaining BookKeeper. It also discusses [best practices](#best-practices) and [common problems](#common-problems). + +## Requirements + +A typical BookKeeper installation consists of an ensemble of {% pop bookies %} and a ZooKeeper quorum. The exact number of bookies depends on the quorum mode that you choose, desired throughput, and the number of clients using the installation simultaneously. + +The minimum number of bookies depends on the type of installation: + +* For *self-verifying* entries you should run at least three bookies. In this mode, clients store a message authentication code along with each {% pop entry %}. +* For *generic* entries you should run at least four + +There is no upper limit on the number of bookies that you can run in a single ensemble. + +### Performance + +To achieve optimal performance, BookKeeper requires each server to have at least two disks. It's possible to run a bookie with a single disk but performance will be significantly degraded. + +### ZooKeeper + +There is no constraint on the number of ZooKeeper nodes you can run with BookKeeper. A single machine running ZooKeeper in [standalone mode](https://zookeeper.apache.org/doc/current/zookeeperStarted.html#sc_InstallingSingleMode) is sufficient for BookKeeper, although for the sake of higher resilience we recommend running ZooKeeper in [quorum mode](https://zookeeper.apache.org/doc/current/zookeeperStarted.html#sc_RunningReplicatedZooKeeper) with multiple servers. + +## Starting and stopping bookies + +You can run bookies either in the foreground or in the background, using [nohup](https://en.wikipedia.org/wiki/Nohup). You can also run [local bookies](#local-bookie) for development purposes. + +To start a bookie in the foreground, use the [`bookie`](../../reference/cli#bookkeeper-bookie) command of the [`bookkeeper`](../../reference/cli#bookkeeper) CLI tool: + +```shell +$ bookkeeper-server/bin/bookkeeper bookie +``` + +To start a bookie in the background, use the [`bookkeeper-daemon.sh`](../../reference/cli#bookkeeper-daemon.sh) script and run `start bookie`: + +```shell +$ bookkeeper-server/bin/bookkeeper-daemon.sh start bookie +``` + +### Local bookies + +The instructions above showed you how to run bookies intended for production use. If you'd like to experiment with ensembles of bookies locally, you can use the [`localbookie`](../../reference/cli#bookkeeper-localbookie) command of the `bookkeeper` CLI tool and specify the number of bookies you'd like to run. + +This would spin up a local ensemble of 6 bookies: + +```shell +$ bookkeeper-server/bin/bookkeeper localbookie 6 +``` + +> When you run a local bookie ensemble, all bookies run in a single JVM process. + +## Configuring bookies + +There's a wide variety of parameters that you can set in the bookie configuration file in `bookkeeper-server/conf/bk_server.conf` of your [BookKeeper installation](../../reference/config). A full listing can be found in [Bookie configuration](../../reference/config). + +Some of the more important parameters to be aware of: + +Parameter | Description | Default +:---------|:------------|:------- +`bookiePort` | The TCP port that the bookie listens on | `3181` +`zkServers` | A comma-separated list of ZooKeeper servers in `hostname:port` format | `localhost:2181` +`journalDirectory` | The directory where the [log device](../../getting-started/concepts#log-device) stores the bookie's write-ahead log (WAL) | `/tmp/bk-txn` +`ledgerDirectories` | The directories where the [ledger device](../../getting-started/concepts#ledger-device) stores the bookie's ledger entries (as a comma-separated list) | `/tmp/bk-data` + +> Ideally, the directories specified `journalDirectory` and `ledgerDirectories` should be on difference devices. + +## Logging + +BookKeeper uses [slf4j](http://www.slf4j.org/) for logging, with [log4j](https://logging.apache.org/log4j/2.x/) bindings enabled by default. + +To enable logging for a bookie, create a `log4j.properties` file and point the `BOOKIE_LOG_CONF` environment variable to the configuration file. Here's an example: + +```shell +$ export BOOKIE_LOG_CONF=/some/path/log4j.properties +$ bookkeeper-server/bin/bookkeeper bookie +``` + +## Upgrading + +From time to time you may need to make changes to the filesystem layout of bookies---changes that are incompatible with previous versions of BookKeeper and require that directories used with previous versions are upgraded. If a filesystem upgrade is required when updating BookKeeper, the bookie will fail to start and return an error like this: + +``` +2017-05-25 10:41:50,494 - ERROR - [main:Bookie@246] - Directory layout version is less than 3, upgrade needed +``` + +BookKeeper provides a utility for upgrading the filesystem. You can perform an upgrade using the [`upgrade`](../../reference/cli#bookkeeper-upgrade) command of the `bookkeeper` CLI tool. When running `bookkeeper upgrade` you need to specify one of three flags: + +Flag | Action +:----|:------ +`--upgrade` | Performs an upgrade +`--rollback` | Performs a rollback to the initial filesystem version +`--finalize` | Marks the upgrade as complete + +### Upgrade pattern + +A standard upgrade pattern is to run an upgrade... + +```shell +$ bookkeeper-server/bin/bookkeeper upgrade --upgrade +``` + +...then check that everything is working normally, then kill the bookie. If everything is okay, finalize the upgrade... + +```shell +$ bookkeeper-server/bin/bookkeeper upgrade --finalize +``` + +...and then restart the server: + +```shell +$ bookkeeper-server/bin/bookkeeper bookie +``` + +If something has gone wrong, you can always perform a rollback: + +```shell +$ bookkeeper-server/bin/bookkeeper upgrade --rollback +``` + +## Formatting + +You can format bookie metadata in ZooKeeper using the [`metaformat`](../../reference/cli#bookkeeper-shell-metaformat) command of the [BookKeeper shell](../../reference/cli#the-bookkeeper-shell). + +By default, formatting is done in interactive mode, which prompts you to confirm the format operation if old data exists. You can disable confirmation using the `-nonInteractive` flag. If old data does exist, the format operation will abort *unless* you set the `-force` flag. Here's an example: + +```shell +$ bookkeeper-server/bin/bookkeeper shell metaformat +``` + +You can format the local filesystem data on a bookie using the [`bookieformat`](../../reference/cli#bookkeeper-shell-bookieformat) command on each bookie. Here's an example: + +```shell +$ bookkeeper-server/bin/bookkeeper shell bookieformat +``` + +> The `-force` and `-nonInteractive` flags are also available for the `bookieformat` command. + +## AutoRecovery + +For a guide to AutoRecovery in BookKeeper, see [this doc](../autorecovery). + +## Missing disks or directories + +Accidentally replacing disks or removing directories can cause a bookie to fail while trying to read a ledger fragment that, according to the ledger metadata, exists on the bookie. For this reason, when a bookie is started for the first time, its disk configuration is fixed for the lifetime of that bookie. Any change to its disk configuration, such as a crashed disk or an accidental configuration change, will result in the bookie being unable to start. That will throw an error like this: + +``` +2017-05-29 18:19:13,790 - ERROR - [main:BookieServer314] – Exception running bookie server : @ +org.apache.bookkeeper.bookie.BookieException$InvalidCookieException +.......at org.apache.bookkeeper.bookie.Cookie.verify(Cookie.java:82) +.......at org.apache.bookkeeper.bookie.Bookie.checkEnvironment(Bookie.java:275) +.......at org.apache.bookkeeper.bookie.Bookie.(Bookie.java:351) +``` + +If the change was the result of an accidental configuration change, the change can be reverted and the bookie can be restarted. However, if the change *cannot* be reverted, such as is the case when you want to add a new disk or replace a disk, the bookie must be wiped and then all its data re-replicated onto it. + +1. Increment the [`bookiePort`](../../reference/config#bookiePort) parameter in the [`bk_server.conf`](../../reference/config) +1. Ensure that all directories specified by [`journalDirectory`](../../reference/config#journalDirectory) and [`ledgerDirectories`](../../reference/config#ledgerDirectories) are empty. +1. [Start the bookie](#starting-and-stopping-bookies). +1. Run the following command to re-replicate the data: + + ```bash + $ bin/bookkeeper org.apache.bookkeeper.tools.BookKeeperTools \ + \ + \ + + ``` + + The ZooKeeper server, old bookie, and new bookie, are all identified by their external IP and `bookiePort` (3181 by default). Here's an example: + + ```bash + $ bin/bookkeeper org.apache.bookkeeper.tools.BookKeeperTools \ + zk1.example.com \ + 192.168.1.10:3181 \ + 192.168.1.10:3181 + ``` + + See the [AutoRecovery](../autorecovery) documentation for more info on the re-replication process. diff --git a/site/docs/4.5.1/admin/geo-replication.md b/site/docs/4.5.1/admin/geo-replication.md new file mode 100644 index 00000000000..38b972345ef --- /dev/null +++ b/site/docs/4.5.1/admin/geo-replication.md @@ -0,0 +1,22 @@ +--- +title: Geo-replication +subtitle: Replicate data across BookKeeper clusters +--- + +*Geo-replication* is the replication of data across BookKeeper clusters. In order to enable geo-replication for a group of BookKeeper clusters, + +## Global ZooKeeper + +Setting up a global ZooKeeper quorum is a lot like setting up a cluster-specific quorum. The crucial difference is that + +### Geo-replication across three clusters + +Let's say that you want to set up geo-replication across clusters in regions A, B, and C. First, the BookKeeper clusters in each region must have their own local (cluster-specific) ZooKeeper quorum. + +> BookKeeper clusters use global ZooKeeper only for metadata storage. Traffic from bookies to ZooKeeper should thus be fairly light in general. + +The crucial difference between using cluster-specific ZooKeeper and global ZooKeeper is that {% pop bookies %} is that you need to point all bookies to use the global ZooKeeper setup. + +## Region-aware placement polocy + +## Autorecovery diff --git a/site/docs/4.5.1/admin/metrics.md b/site/docs/4.5.1/admin/metrics.md new file mode 100644 index 00000000000..635135faf7e --- /dev/null +++ b/site/docs/4.5.1/admin/metrics.md @@ -0,0 +1,41 @@ +--- +title: Metric collection +--- + +BookKeeper enables metrics collection through a variety of [stats providers](#stats-providers). + +> For a full listing of available metrics, see the [Metrics](../../reference/metrics) reference doc. + +## Stats providers + +BookKeeper has stats provider implementations for four five sinks: + +Provider | Provider class name +:--------|:------------------- +[Codahale Metrics](https://mvnrepository.com/artifact/org.apache.bookkeeper.stats/codahale-metrics-provider) | `org.apache.bookkeeper.stats.CodahaleMetricsProvider` +[Prometheus](https://prometheus.io/) | `org.apache.bookkeeper.stats.PrometheusMetricsProvider` +[Finagle](https://twitter.github.io/finagle/guide/Metrics.html) | `org.apache.bookkeeper.stats.FinagleStatsProvider` +[Ostrich](https://github.com/twitter/ostrich) | `org.apache.bookkeeper.stats.OstrichProvider` +[Twitter Science Provider](https://mvnrepository.com/artifact/org.apache.bookkeeper.stats/twitter-science-provider) | `org.apache.bookkeeper.stats.TwitterStatsProvider` + +> The [Codahale Metrics]({{ site.github_master }}/bookkeeper-stats-providers/codahale-metrics-provider) stats provider is the default provider. + +## Enabling stats providers in bookies + +There are two stats-related [configuration parameters](../../reference/config#statistics) available for bookies: + +Parameter | Description | Default +:---------|:------------|:------- +`enableStatistics` | Whether statistics are enabled for the bookie | `false` +`statsProviderClass` | The stats provider class used by the bookie | `org.apache.bookkeeper.stats.CodahaleMetricsProvider` + + +To enable stats: + +* set the `enableStatistics` parameter to `true` +* set `statsProviderClass` to the desired provider (see the [table above](#stats-providers) for a listing of classes) + + diff --git a/site/docs/4.5.1/admin/perf.md b/site/docs/4.5.1/admin/perf.md new file mode 100644 index 00000000000..82956326e5d --- /dev/null +++ b/site/docs/4.5.1/admin/perf.md @@ -0,0 +1,3 @@ +--- +title: Performance tuning +--- diff --git a/site/docs/4.5.1/admin/placement.md b/site/docs/4.5.1/admin/placement.md new file mode 100644 index 00000000000..ded456e1aea --- /dev/null +++ b/site/docs/4.5.1/admin/placement.md @@ -0,0 +1,3 @@ +--- +title: Customized placement policies +--- diff --git a/site/docs/4.5.1/admin/upgrade.md b/site/docs/4.5.1/admin/upgrade.md new file mode 100644 index 00000000000..456df99a276 --- /dev/null +++ b/site/docs/4.5.1/admin/upgrade.md @@ -0,0 +1,73 @@ +--- +title: Upgrade +--- + +> If you have questions about upgrades (or need help), please feel free to reach out to us by [mailing list]({{ site.baseurl }}community/mailing-lists) or [Slack Channel]({{ site.baseurl }}community/slack). + +## Overview + +Consider the below guidelines in preparation for upgrading. + +- Always back up all your configuration files before upgrading. +- Read through the documentation and draft an upgrade plan that matches your specific requirements and environment before starting the upgrade process. + Put differently, don't start working through the guide on a live cluster. Read guide entirely, make a plan, then execute the plan. +- Pay careful consideration to the order in which components are upgraded. In general, you need to upgrade bookies first and then upgrade your clients. +- If autorecovery is running along with bookies, you need to pay attention to the upgrade sequence. +- Read the release notes carefully for each release. They contain not only information about noteworthy features, but also changes to configurations + that may impact your upgrade. +- Always upgrade one or a small set of bookies to canary new version before upgraing all bookies in your cluster. + +## Canary + +It is wise to canary an upgraded version in one or small set of bookies before upgrading all bookies in your live cluster. + +You can follow below steps on how to canary a upgraded version: + +1. Stop a Bookie. +2. Upgrade the binary and configuration. +3. Start the Bookie in `ReadOnly` mode. This can be used to verify if the Bookie of this new version can run well for read workload. +4. Once the Bookie is running at `ReadOnly` mode successfully for a while, restart the Bookie in `Write/Read` mode. +5. After step 4, the Bookie will serve both write and read traffic. + +### Rollback Canaries + +If problems occur during canarying an upgraded version, you can simply take down the problematic Bookie node. The remain bookies in the old cluster +will repair this problematic bookie node by autorecovery. Nothing needs to be worried about. + +## Upgrade Steps + +Once you determined a version is safe to upgrade in a few nodes in your cluster, you can perform following steps to upgrade all bookies in your cluster. + +1. Determine if autorecovery is running along with bookies. If yes, check if the clients (either new clients with new binary or old clients with new configurations) +are allowed to talk to old bookies; if clients are not allowed to talk to old bookies, please [disable autorecovery](../../reference/cli/#autorecovery-1) during upgrade. +2. Decide on performing a rolling upgrade or a downtime upgrade. +3. Upgrade all Bookies (more below) +4. If autorecovery was disabled during upgrade, [enable autorecovery](../../reference/cli/#autorecovery-1). +5. After all bookies are upgraded, build applications that use `BookKeeper client` against the new bookkeeper libraries and deploy the new versions. + +### Upgrade Bookies + +In a rolling upgrade scenario, upgrade one Bookie at a time. In a downtime upgrade scenario, take the entire cluster down, upgrade each Bookie, then start the cluster. + +For each Bookie: + +1. Stop the bookie. +2. Upgrade the software (either new binary or new configuration) +2. Start the bookie. + +## Upgrade Guides + +We describes the general upgrade method in Apache BookKeeper as above. We will cover the details for individual versions. + +### 4.4.x to 4.5.x upgrade + +There isn't any protocol related backward compabilities changes in 4.5.0. So you can follow the general upgrade sequence to upgrade from 4.4.x to 4.5.x. +However, we list a list of things that you might want to know. + +1. 4.5.x upgrades netty from 3.x to 4.x. The memory usage pattern might be changed a bit. Netty 4 uses more direct memory. Please pay attention to your memory usage + and adjust the JVM settings accordingly. +2. `multi journals` is a non-rollbackable feature. If you configure a bookie to use multiple journals on 4.5.x you can not roll the bookie back to use 4.4.x. You have + to take a bookie out and recover it if you want to rollback to 4.4.x. + +If you are planning to upgrade a non-secured cluster to a secured cluster enabling security features in 4.5.0, please read [BookKeeper Security](../../security/overview) for more details. + diff --git a/site/docs/4.5.1/api/distributedlog-api.md b/site/docs/4.5.1/api/distributedlog-api.md new file mode 100644 index 00000000000..c143384cd6d --- /dev/null +++ b/site/docs/4.5.1/api/distributedlog-api.md @@ -0,0 +1,395 @@ +--- +title: DistributedLog +subtitle: A higher-level API for managing BookKeeper entries +--- + +> DistributedLog began its life as a separate project under the Apache Foundation. It was merged into BookKeeper in 2017. + +The DistributedLog API is an easy-to-use interface for managing BookKeeper entries that enables you to use BookKeeper without needing to interact with [ledgers](../ledger-api) directly. + +DistributedLog (DL) maintains sequences of records in categories called *logs* (aka *log streams*). *Writers* append records to DL logs, while *readers* fetch and process those records. + +## Architecture + +The diagram below illustrates how the DistributedLog API works with BookKeeper: + +![DistributedLog API]({{ site.baseurl }}img/distributedlog.png) + +## Logs + +A *log* in DistributedLog is an ordered, immutable sequence of *log records*. + +The diagram below illustrates the anatomy of a log stream: + +![DistributedLog log]({{ site.baseurl }}img/logs.png) + +### Log records + +Each log record is a sequence of bytes. Applications are responsible for serializing and deserializing byte sequences stored in log records. + +Log records are written sequentially into a *log stream* and assigned with a a unique sequence number called a DLSN (DistributedLog Sequence Number). + +In addition to a DLSN, applications can assign their own sequence number when constructing log records. Application-defined sequence numbers are known as *TransactionIDs* (or *txid*). Either a DLSN or a TransactionID can be used for positioning readers to start reading from a specific log record. + +### Log segments + +Each log is broken down into *log segments* that contain subsets of records. Log segments are distributed and stored in BookKeeper. DistributedLog rolls the log segments based on the configured *rolling policy*, which be either + +* a configurable period of time (such as every 2 hours), or +* a configurable maximum size (such as every 128 MB). + +The data in logs is divided up into equally sized log segments and distributed evenly across {% pop bookies %}. This allows logs to scale beyond a size that would fit on a single server and spreads read traffic across the cluster. + +### Namespaces + +Log streams that belong to the same organization are typically categorized and managed under a *namespace*. DistributedLog namespaces essentially enable applications to locate log streams. Applications can perform the following actions under a namespace: + +* create streams +* delete streams +* truncate streams to a given sequence number (either a DLSN or a TransactionID) + +## Writers + +Through the DistributedLog API, writers write data into logs of their choice. All records are appended into logs in order. The sequencing is performed by the writer, which means that there is only one active writer for a log at any given time. + +DistributedLog guarantees correctness when two writers attempt to write to the same log when a network partition occurs using a *fencing* mechanism in the log segment store. + +### Write Proxy + +Log writers are served and managed in a service tier called the *Write Proxy* (see the diagram [above](#architecture)). The Write Proxy is used for accepting writes from a large number of clients. + +## Readers + +DistributedLog readers read records from logs of their choice, starting with a provided position. The provided position can be either a DLSN or a TransactionID. + +Readers read records from logs in strict order. Different readers can read records from different positions in the same log. + +Unlike other pub-sub systems, DistributedLog doesn't record or manage readers' positions. This means that tracking is the responsibility of applications, as different applications may have different requirements for tracking and coordinating positions. This is hard to get right with a single approach. Distributed databases, for example, might store reader positions along with SSTables, so they would resume applying transactions from the positions store in SSTables. Tracking reader positions could easily be done at the application level using various stores (such as ZooKeeper, the filesystem, or key-value stores). + +### Read Proxy + +Log records can be cached in a service tier called the *Read Proxy* to serve a large number of readers. See the diagram [above](#architecture). The Read Proxy is the analogue of the [Write Proxy](#write-proxy). + +## Guarantees + +The DistributedLog API for BookKeeper provides a number of guarantees for applications: + +* Records written by a [writer](#writers) to a [log](#logs) are appended in the order in which they are written. If a record **R1** is written by the same writer as a record **R2**, **R1** will have a smaller sequence number than **R2**. +* [Readers](#readers) see [records](#log-records) in the same order in which they are [written](#writers) to the log. +* All records are persisted on disk by BookKeeper before acknowledgements, which guarantees durability. +* For a log with a replication factor of N, DistributedLog tolerates up to N-1 server failures without losing any records. + +## API + +Documentation for the DistributedLog API can be found [here](https://distributedlog.incubator.apache.org/docs/latest/user_guide/api/core). + +> At a later date, the DistributedLog API docs will be added here. + + diff --git a/site/docs/4.5.1/api/ledger-adv-api.md b/site/docs/4.5.1/api/ledger-adv-api.md new file mode 100644 index 00000000000..f46950dd984 --- /dev/null +++ b/site/docs/4.5.1/api/ledger-adv-api.md @@ -0,0 +1,82 @@ +--- +title: The Advanced Ledger API +--- + +In release `4.5.0`, Apache BookKeeper introduces a few advanced API for advanced usage. +This sections covers these advanced APIs. + +> Before learn the advanced API, please read [Ledger API](../ledger-api) first. + +## LedgerHandleAdv + +[`LedgerHandleAdv`](../javadoc/org/apache/bookkeeper/client/LedgerHandleAdv) is an advanced extension of [`LedgerHandle`](../javadoc/org/apache/bookkeeper/client/LedgerHandle). +It allows user passing in an `entryId` when adding an entry. + +### Creating advanced ledgers + +Here's an exmaple: + +```java +byte[] passwd = "some-passwd".getBytes(); +LedgerHandleAdv handle = bkClient.createLedgerAdv( + 3, 3, 2, // replica settings + DigestType.CRC32, + passwd); +``` + +You can also create advanced ledgers asynchronously. + +```java +class LedgerCreationCallback implements AsyncCallback.CreateCallback { + public void createComplete(int returnCode, LedgerHandle handle, Object ctx) { + System.out.println("Ledger successfully created"); + } +} +client.asyncCreateLedgerAdv( + 3, // ensemble size + 3, // write quorum size + 2, // ack quorum size + BookKeeper.DigestType.CRC32, + password, + new LedgerCreationCallback(), + "some context" +); +``` + +Besides the APIs above, BookKeeper allows users providing `ledger-id` when creating advanced ledgers. + +```java +long ledgerId = ...; // the ledger id is generated externally. + +byte[] passwd = "some-passwd".getBytes(); +LedgerHandleAdv handle = bkClient.createLedgerAdv( + ledgerId, // ledger id generated externally + 3, 3, 2, // replica settings + DigestType.CRC32, + passwd); +``` + +> Please note, it is users' responsibility to provide a unique ledger id when using the API above. +> If a ledger already exists when users try to create an advanced ledger with same ledger id, +> a [LedgerExistsException](../javadoc/org/apache/bookkeeper/client/BKException.BKLedgerExistException.html) is thrown by the bookkeeper client. + +### Add Entries + +The normal [add entries api](ledger-api/#adding-entries-to-ledgers) in advanced ledgers are disabled. Instead, when users want to add entries +to advanced ledgers, an entry id is required to pass in along with the entry data when adding an entry. + +```java +long entryId = ...; // entry id generated externally + +ledger.addEntry(entryId, "Some entry data".getBytes()); +``` + +A few notes when using this API: + +- The entry id has to be non-negative. +- Clients are okay to add entries out of order. +- However, the entries are only acknowledged in a monotonic order starting from 0. + +### Read Entries + +The read entries api in advanced ledgers remain same as [normal ledgers](../ledger-api/#reading-entries-from-ledgers). diff --git a/site/docs/4.5.1/api/ledger-api.md b/site/docs/4.5.1/api/ledger-api.md new file mode 100644 index 00000000000..4e1070d717a --- /dev/null +++ b/site/docs/4.5.1/api/ledger-api.md @@ -0,0 +1,473 @@ +--- +title: The Ledger API +--- + +The ledger API is a lower-level API for BookKeeper that enables you to interact with {% pop ledgers %} directly. + +## The Java ledger API client + +To get started with the Java client for BookKeeper, install the `bookkeeper-server` library as a dependency in your Java application. + +> For a more in-depth tutorial that involves a real use case for BookKeeper, see the [Example application](../example-application) guide. + +## Installation + +The BookKeeper Java client library is available via [Maven Central](http://search.maven.org/) and can be installed using [Maven](#maven), [Gradle](#gradle), and other build tools. + +### Maven + +If you're using [Maven](https://maven.apache.org/), add this to your [`pom.xml`](https://maven.apache.org/guides/introduction/introduction-to-the-pom.html) build configuration file: + +```xml + +4.5.0 + + + + org.apache.bookkeeper + bookkeeper-server + ${bookkeeper.version} + +``` + +### Gradle + +If you're using [Gradle](https://gradle.org/), add this to your [`build.gradle`](https://spring.io/guides/gs/gradle/) build configuration file: + +```groovy +dependencies { + compile group: 'org.apache.bookkeeper', name: 'bookkeeper-server', version: '4.5.0' +} + +// Alternatively: +dependencies { + compile 'org.apache.bookkeeper:bookkeeper-server:4.5.0' +} +``` + +## Connection string + +When interacting with BookKeeper using the Java client, you need to provide your client with a connection string, for which you have three options: + +* Provide your entire ZooKeeper connection string, for example `zk1:2181,zk2:2181,zk3:2181`. +* Provide a host and port for one node in your ZooKeeper cluster, for example `zk1:2181`. In general, it's better to provide a full connection string (in case the ZooKeeper node you attempt to connect to is down). +* If your ZooKeeper cluster can be discovered via DNS, you can provide the DNS name, for example `my-zookeeper-cluster.com`. + +## Creating a new client + +In order to create a new [`BookKeeper`](../javadoc/org/apache/bookkeeper/client/BookKeeper) client object, you need to pass in a [connection string](#connection-string). Here is an example client object using a ZooKeeper connection string: + +```java +try { + String connectionString = "127.0.0.1:2181"; // For a single-node, local ZooKeeper cluster + BookKeeper bkClient = new BookKeeper(connectionString); +} catch (InterruptedException | IOException | KeeperException e) { + e.printStackTrace(); +} +``` + +> If you're running BookKeeper [locally](../../getting-started/run-locally), using the [`localbookie`](../../reference/cli#bookkeeper-localbookie) command, use `"127.0.0.1:2181"` for your connection string, as in the example above. + +There are, however, other ways that you can create a client object: + +* By passing in a [`ClientConfiguration`](../javadoc/org/apache/bookkeeper/conf/ClientConfiguration) object. Here's an example: + + ```java + ClientConfiguration config = new ClientConfiguration(); + config.setZkServers(zkConnectionString); + config.setAddEntryTimeout(2000); + BookKeeper bkClient = new BookKeeper(config); + ``` + +* By specifying a `ClientConfiguration` and a [`ZooKeeper`](http://zookeeper.apache.org/doc/current/api/org/apache/zookeeper/ZooKeeper.html) client object: + + ```java + ClientConfiguration config = new ClientConfiguration(); + config.setAddEntryTimeout(5000); + ZooKeeper zkClient = new ZooKeeper(/* client args */); + BookKeeper bkClient = new BookKeeper(config, zkClient); + ``` + +* Using the `forConfig` method: + + ```java + BookKeeper bkClient = BookKeeper.forConfig(conf).build(); + ``` + +## Creating ledgers + +The easiest way to create a {% pop ledger %} using the Java client is via the `createLedger` method, which creates a new ledger synchronously and returns a [`LedgerHandle`](../javadoc/org/apache/bookkeeper/client/LedgerHandle). You must specify at least a [`DigestType`](../javadoc/org/apache/bookkeeper/client/BookKeeper.DigestType) and a password. + +Here's an example: + +```java +byte[] password = "some-password".getBytes(); +LedgerHandle handle = bkClient.createLedger(BookKeeper.DigestType.MAC, password); +``` + +You can also create ledgers asynchronously + +### Create ledgers asynchronously + +```java +class LedgerCreationCallback implements AsyncCallback.CreateCallback { + public void createComplete(int returnCode, LedgerHandle handle, Object ctx) { + System.out.println("Ledger successfully created"); + } +} + +client.asyncCreateLedger( + 3, + 2, + BookKeeper.DigestType.MAC, + password, + new LedgerCreationCallback(), + "some context" +); +``` + +## Adding entries to ledgers + +```java +long entryId = ledger.addEntry("Some entry data".getBytes()); +``` + +### Add entries asynchronously + +## Reading entries from ledgers + +```java +Enumerator entries = handle.readEntries(1, 99); +``` + +To read all possible entries from the ledger: + +```java +Enumerator entries = + handle.readEntries(0, handle.getLastAddConfirmed()); + +while (entries.hasNextElement()) { + LedgerEntry entry = entries.nextElement(); + System.out.println("Successfully read entry " + entry.getId()); +} +``` + +### Reading entries after the LastAddConfirmed range + +`readUnconfirmedEntries` allowing to read after the LastAddConfirmed range. +It lets the client read without checking the local value of LastAddConfirmed, so that it is possible to read entries for which the writer has not received the acknowledge yet +For entries which are within the range 0..LastAddConfirmed BookKeeper guarantees that the writer has successfully received the acknowledge. +For entries outside that range it is possible that the writer never received the acknowledge and so there is the risk that the reader is seeing entries before the writer and this could result in a consistency issue in some cases. +With this method you can even read entries before the LastAddConfirmed and entries after it with one call, the expected consistency will be as described above. + +```java +Enumerator entries = + handle.readUnconfirmedEntries(0, lastEntryIdExpectedToRead); + +while (entries.hasNextElement()) { + LedgerEntry entry = entries.nextElement(); + System.out.println("Successfully read entry " + entry.getId()); +} +``` + +## Deleting ledgers + +{% pop Ledgers %} can also be deleted synchronously or asynchronously. + +```java +long ledgerId = 1234; + +try { + bkClient.deleteLedger(ledgerId); +} catch (Exception e) { + e.printStackTrace(); +} +``` + +### Delete entries asynchronously + +Exceptions thrown: + +* + +```java +class DeleteEntryCallback implements AsyncCallback.DeleteCallback { + public void deleteComplete() { + System.out.println("Delete completed"); + } +} +``` + +## Simple example + +> For a more involved BookKeeper client example, see the [example application](#example-application) below. + +In the code sample below, a BookKeeper client: + +* creates a ledger +* writes entries to the ledger +* closes the ledger (meaning no further writes are possible) +* re-opens the ledger for reading +* reads all available entries + +```java +// Create a client object for the local ensemble. This +// operation throws multiple exceptions, so make sure to +// use a try/catch block when instantiating client objects. +BookKeeper bkc = new BookKeeper("localhost:2181"); + +// A password for the new ledger +byte[] ledgerPassword = /* some sequence of bytes, perhaps random */; + +// Create a new ledger and fetch its identifier +LedgerHandle lh = bkc.createLedger(BookKeeper.DigestType.MAC, ledgerPassword); +long ledgerId = lh.getId(); + +// Create a buffer for four-byte entries +ByteBuffer entry = ByteBuffer.allocate(4); + +int numberOfEntries = 100; + +// Add entries to the ledger, then close it +for (int i = 0; i < numberOfEntries; i++){ + entry.putInt(i); + entry.position(0); + lh.addEntry(entry.array()); +} +lh.close(); + +// Open the ledger for reading +lh = bkc.openLedger(ledgerId, BookKeeper.DigestType.MAC, ledgerPassword); + +// Read all available entries +Enumeration entries = lh.readEntries(0, numberOfEntries - 1); + +while(entries.hasMoreElements()) { + ByteBuffer result = ByteBuffer.wrap(ls.nextElement().getEntry()); + Integer retrEntry = result.getInt(); + + // Print the integer stored in each entry + System.out.println(String.format("Result: %s", retrEntry)); +} + +// Close the ledger and the client +lh.close(); +bkc.close(); +``` + +Running this should return this output: + +```shell +Result: 0 +Result: 1 +Result: 2 +# etc +``` + +## Example application + +This tutorial walks you through building an example application that uses BookKeeper as the replicated log. The application uses the [BookKeeper Java client](../java-client) to interact with BookKeeper. + +> The code for this tutorial can be found in [this GitHub repo](https://github.com/ivankelly/bookkeeper-tutorial/). The final code for the `Dice` class can be found [here](https://github.com/ivankelly/bookkeeper-tutorial/blob/master/src/main/java/org/apache/bookkeeper/Dice.java). + +### Setup + +Before you start, you will need to have a BookKeeper cluster running locally on your machine. For installation instructions, see [Installation](../../getting-started/installation). + +To start up a cluster consisting of six {% pop bookies %} locally: + +```shell +$ bookkeeper-server/bin/bookkeeper localbookie 6 +``` + +You can specify a different number of bookies if you'd like. + +### Goal + +The goal of the dice application is to have + +* multiple instances of this application, +* possibly running on different machines, +* all of which display the exact same sequence of numbers. + +In other words, the log needs to be both durable and consistent, regardless of how many {% pop bookies %} are participating in the BookKeeper ensemble. If one of the bookies crashes or becomes unable to communicate with the other bookies in any way, it should *still* display the same sequence of numbers as the others. This tutorial will show you how to achieve this. + +To begin, download the base application, compile and run it. + +```shell +$ git clone https://github.com/ivankelly/bookkeeper-tutorial.git +$ mvn package +$ mvn exec:java -Dexec.mainClass=org.apache.bookkeeper.Dice +``` + +That should yield output that looks something like this: + +``` +[INFO] Scanning for projects... +[INFO] +[INFO] ------------------------------------------------------------------------ +[INFO] Building tutorial 1.0-SNAPSHOT +[INFO] ------------------------------------------------------------------------ +[INFO] +[INFO] --- exec-maven-plugin:1.3.2:java (default-cli) @ tutorial --- +[WARNING] Warning: killAfter is now deprecated. Do you need it ? Please comment on MEXEC-6. +Value = 4 +Value = 5 +Value = 3 +``` + +### The base application + +The application in this tutorial is a dice application. The `Dice` class below has a `playDice` function that generates a random number between 1 and 6 every second, prints the value of the dice roll, and runs indefinitely. + +```java +public class Dice { + Random r = new Random(); + + void playDice() throws InterruptedException { + while (true) { + Thread.sleep(1000); + System.out.println("Value = " + (r.nextInt(6) + 1)); + } + } +} +``` + +When you run the `main` function of this class, a new `Dice` object will be instantiated and then run indefinitely: + +```java +public class Dice { + // other methods + + public static void main(String[] args) throws InterruptedException { + Dice d = new Dice(); + d.playDice(); + } +} +``` + +### Leaders and followers (and a bit of background) + +To achieve this common view in multiple instances of the program, we need each instance to agree on what the next number in the sequence will be. For example, the instances must agree that 4 is the first number and 2 is the second number and 5 is the third number and so on. This is a difficult problem, especially in the case that any instance may go away at any time, and messages between the instances can be lost or reordered. + +Luckily, there are already algorithms to solve this. Paxos is an abstract algorithm to implement this kind of agreement, while Zab and Raft are more practical protocols. This video gives a good overview about how these algorithms usually look. They all have a similar core. + +It would be possible to run the Paxos to agree on each number in the sequence. However, running Paxos each time can be expensive. What Zab and Raft do is that they use a Paxos-like algorithm to elect a leader. The leader then decides what the sequence of events should be, putting them in a log, which the other instances can then follow to maintain the same state as the leader. + +Bookkeeper provides the functionality for the second part of the protocol, allowing a leader to write events to a log and have multiple followers tailing the log. However, bookkeeper does not do leader election. You will need a zookeeper or raft instance for that purpose. + +### Why not just use ZooKeeper? + +There are a number of reasons: + +1. Zookeeper's log is only exposed through a tree like interface. It can be hard to shoehorn your application into this. +2. A zookeeper ensemble of multiple machines is limited to one log. You may want one log per resource, which will become expensive very quickly. +3. Adding extra machines to a zookeeper ensemble does not increase capacity nor throughput. + +Bookkeeper can be seen as a means of exposing ZooKeeper's replicated log to applications in a scalable fashion. ZooKeeper is still used by BookKeeper, however, to maintain consistency guarantees, though clients don't need to interact with ZooKeeper directly. + +### Electing a leader + +We'll use zookeeper to elect a leader. A zookeeper instance will have started locally when you started the localbookie application above. To verify it's running, run the following command. + +```shell +$ echo stat | nc localhost 2181 +Zookeeper version: 3.4.6-1569965, built on 02/20/2014 09:09 GMT +Clients: + /127.0.0.1:59343[1](queued=0,recved=40,sent=41) + /127.0.0.1:49354[1](queued=0,recved=11,sent=11) + /127.0.0.1:49361[0](queued=0,recved=1,sent=0) + /127.0.0.1:59344[1](queued=0,recved=38,sent=39) + /127.0.0.1:59345[1](queued=0,recved=38,sent=39) + /127.0.0.1:59346[1](queued=0,recved=38,sent=39) + +Latency min/avg/max: 0/0/23 +Received: 167 +Sent: 170 +Connections: 6 +Outstanding: 0 +Zxid: 0x11 +Mode: standalone +Node count: 16 +``` + +To interact with zookeeper, we'll use the Curator client rather than the stock zookeeper client. Getting things right with the zookeeper client can be tricky, and curator removes a lot of the pointy corners for you. In fact, curator even provides a leader election recipe, so we need to do very little work to get leader election in our application. + +```java +public class Dice extends LeaderSelectorListenerAdapter implements Closeable { + + final static String ZOOKEEPER_SERVER = "127.0.0.1:2181"; + final static String ELECTION_PATH = "/dice-elect"; + + ... + + Dice() throws InterruptedException { + curator = CuratorFrameworkFactory.newClient(ZOOKEEPER_SERVER, + 2000, 10000, new ExponentialBackoffRetry(1000, 3)); + curator.start(); + curator.blockUntilConnected(); + + leaderSelector = new LeaderSelector(curator, ELECTION_PATH, this); + leaderSelector.autoRequeue(); + leaderSelector.start(); + } +``` + +In the constructor for Dice, we need to create the curator client. We specify four things when creating the client, the location of the zookeeper service, the session timeout, the connect timeout and the retry policy. + +The session timeout is a zookeeper concept. If the zookeeper server doesn't hear anything from the client for this amount of time, any leases which the client holds will be timed out. This is important in leader election. For leader election, the curator client will take a lease on ELECTION_PATH. The first instance to take the lease will become leader and the rest will become followers. However, their claim on the lease will remain in the cue. If the first instance then goes away, due to a crash etc., its session will timeout. Once the session times out, the lease will be released and the next instance in the queue will become the leader. The call to autoRequeue() will make the client queue itself again if it loses the lease for some other reason, such as if it was still alive, but it a garbage collection cycle caused it to lose its session, and thereby its lease. I've set the lease to be quite low so that when we test out leader election, transitions will be quite quick. The optimum length for session timeout depends very much on the use case. The other parameters are the connection timeout, i.e. the amount of time it will spend trying to connect to a zookeeper server before giving up, and the retry policy. The retry policy specifies how the client should respond to transient errors, such as connection loss. Operations that fail with transient errors can be retried, and this argument specifies how often the retries should occur. + +Finally, you'll have noticed that Dice now extends LeaderSelectorListenerAdapter and implements Closeable. Closeable is there to close the resource we have initialized in the constructor, the curator client and the leaderSelector. LeaderSelectorListenerAdapter is a callback that the leaderSelector uses to notify the instance that it is now the leader. It is passed as the third argument to the LeaderSelector constructor. + +```java + @Override + public void takeLeadership(CuratorFramework client) + throws Exception { + synchronized (this) { + leader = true; + try { + while (true) { + this.wait(); + } + } catch (InterruptedException ie) { + Thread.currentThread().interrupt(); + leader = false; + } + } + } +``` + +takeLeadership() is the callback called by LeaderSelector when the instance is leader. It should only return when the instance wants to give up leadership. In our case, we never do so we wait on the current object until we're interrupted. To signal to the rest of the program that we are leader we set a volatile boolean called leader to true. This is unset after we are interrupted. + +```java + void playDice() throws InterruptedException { + while (true) { + while (leader) { + Thread.sleep(1000); + System.out.println("Value = " + (r.nextInt(6) + 1) + + ", isLeader = " + leader); + } + } + } +``` + +Finally, we modify the `playDice` function to only generate random numbers when it is the leader. + +Run two instances of the program in two different terminals. You'll see that one becomes leader and prints numbers and the other just sits there. + +Now stop the leader using Control-Z. This will pause the process, but it won't kill it. You will be dropped back to the shell in that terminal. After a couple of seconds, the session timeout, you will see that the other instance has become the leader. Zookeeper will guarantee that only one instance is selected as leader at any time. + +Now go back to the shell that the original leader was on and wake up the process using fg. You'll see something like the following: + +```shell +... +... +Value = 4, isLeader = true +Value = 4, isLeader = true +^Z +[1]+ Stopped mvn exec:java -Dexec.mainClass=org.apache.bookkeeper.Dice +$ fg +mvn exec:java -Dexec.mainClass=org.apache.bookkeeper.Dice +Value = 3, isLeader = true +Value = 1, isLeader = false +``` diff --git a/site/docs/4.5.1/api/overview.md b/site/docs/4.5.1/api/overview.md new file mode 100644 index 00000000000..3eb649273c1 --- /dev/null +++ b/site/docs/4.5.1/api/overview.md @@ -0,0 +1,17 @@ +--- +title: BookKeeper API +--- + +BookKeeper offers a few APIs that applications can use to interact with it: + +* The [ledger API](../ledger-api) is a lower-level API that enables you to interact with {% pop ledgers %} directly +* The [Ledger Advanced API)(../ledger-adv-api) is an advanced extension to [Ledger API](../ledger-api) to provide more flexibilities to applications. +* The [DistributedLog API](../distributedlog-api) is a higher-level API that provides convenient abstractions. + +## Trade-offs + +The `Ledger API` provides direct access to ledgers and thus enables you to use BookKeeper however you'd like. + +However, in most of use cases, if you want a `log stream`-like abstraction, it requires you to manage things like tracking list of ledgers, +managing rolling ledgers and data retention on your own. In such cases, you are recommended to use [DistributedLog API](../distributedlog-api), +with semantics resembling continous log streams from the standpoint of applications. diff --git a/site/docs/4.5.1/deployment/dcos.md b/site/docs/4.5.1/deployment/dcos.md new file mode 100644 index 00000000000..3e174384ec1 --- /dev/null +++ b/site/docs/4.5.1/deployment/dcos.md @@ -0,0 +1,142 @@ +--- +title: Deploying BookKeeper on DC/OS +subtitle: Get up and running easily on an Apache Mesos cluster +logo: img/dcos-logo.png +--- + +[DC/OS](https://dcos.io/) (the DataCenter Operating System) is a distributed operating system used for deploying and managing applications and systems on [Apache Mesos](http://mesos.apache.org/). DC/OS is an open-source tool created and maintained by [Mesosphere](https://mesosphere.com/). + +BookKeeper is available as a [DC/OS package](http://universe.dcos.io/#/package/bookkeeper/version/latest) from the [Mesosphere DC/OS Universe](http://universe.dcos.io/#/packages). + +## Prerequisites + +In order to run BookKeeper on DC/OS, you will need: + +* DC/OS version [1.8](https://dcos.io/docs/1.8/) or higher +* A DC/OS cluster with at least three nodes +* The [DC/OS CLI tool](https://dcos.io/docs/1.8/usage/cli/install/) installed + +Each node in your DC/OS-managed Mesos cluster must have at least: + +* 1 CPU +* 1 GB of memory +* 10 GB of total persistent disk storage + +## Installing BookKeeper + +```shell +$ dcos package install bookkeeper --yes +``` + +This command will: + +* Install the `bookkeeper` subcommand for the `dcos` CLI tool +* Start a single {% pop bookie %} on the Mesos cluster with the [default configuration](../../reference/config) + +The bookie that is automatically started up uses the host mode of the network and by default exports the service at `agent_ip:3181`. + +> If you run `dcos package install bookkeeper` without setting the `--yes` flag, the install will run in interactive mode. For more information on the `package install` command, see the [DC/OS docs](https://docs.mesosphere.com/latest/cli/command-reference/dcos-package/dcos-package-install/). + +### Services + +To watch BookKeeper start up, click on the **Services** tab in the DC/OS [user interface](https://docs.mesosphere.com/latest/gui/) and you should see the `bookkeeper` package listed: + +![DC/OS services]({{ site.baseurl }}img/dcos/services.png) + +### Tasks + +To see which tasks have started, click on the `bookkeeper` service and you'll see an interface that looks like this; + +![DC/OS tasks]({{ site.baseurl }}img/dcos/tasks.png) + +## Scaling BookKeeper + +Once the first {% pop bookie %} has started up, you can click on the **Scale** tab to scale up your BookKeeper ensemble by adding more bookies (or scale down the ensemble by removing bookies). + +![DC/OS scale]({{ site.baseurl }}img/dcos/scale.png) + +## ZooKeeper Exhibitor + +ZooKeeper contains the information for all bookies in the ensemble. When deployed on DC/OS, BookKeeper uses a ZooKeeper instance provided by DC/OS. You can access a visual UI for ZooKeeper using [Exhibitor](https://github.com/soabase/exhibitor/wiki), which is available at [http://master.dcos/exhibitor](http://master.dcos/exhibitor). + +![ZooKeeper Exhibitor]({{ site.baseurl }}img/dcos/exhibitor.png) + +You should see a listing of IP/host information for all bookies under the `messaging/bookkeeper/ledgers/available` node. + +## Client connections + +To connect to bookies running on DC/OS using clients running within your Mesos cluster, you need to specify the ZooKeeper connection string for DC/OS's ZooKeeper cluster: + +``` +master.mesos:2181 +``` + +This is the *only* ZooKeeper host/port you need to include in your connection string. Here's an example using the [Java client](../../api/ledger-api#the-java-ledger-api-client): + +```java +BookKeeper bkClient = new BookKeeper("master.mesos:2181"); +``` + +If you're connecting using a client running outside your Mesos cluster, you need to supply the public-facing connection string for your DC/OS ZooKeeper cluster. + +## Configuring BookKeeper + +By default, the `bookkeeper` package will start up a BookKeeper ensemble consisting of one {% pop bookie %} with one CPU, 1 GB of memory, and a 70 MB persistent volume. + +You can supply a non-default configuration when installing the package using a JSON file. Here's an example command: + +```shell +$ dcos package install bookkeeper \ + --options=/path/to/config.json +``` + +You can then fetch the current configuration for BookKeeper at any time using the `package describe` command: + +```shell +$ dcos package describe bookkeeper \ + --config +``` + +### Available parameters + +> Not all [configurable parameters](../../reference/config) for BookKeeper are available for BookKeeper on DC/OS. Only the parameters show in the table below are available. + +Param | Type | Description | Default +:-----|:-----|:------------|:------- +`name` | String | The name of the DC/OS service. | `bookkeeper` +`cpus` | Integer | The number of CPU shares to allocate to each {% pop bookie %}. The minimum is 1. | `1` | +`instances` | Integer | The number of {% pop bookies %} top run. The minimum is 1. | `1` +`mem` | Number | The memory, in MB, to allocate to each BookKeeper task | `1024.0` (1 GB) +`volume_size` | Number | The persistent volume size, in MB | `70` +`zk_client` | String | The connection string for the ZooKeeper client instance | `master.mesos:2181` +`service_port` | Integer | The BookKeeper export service port, using `PORT0` in Marathon | `3181` + +### Example JSON configuration + +Here's an example JSON configuration object for BookKeeper on DC/OS: + +```json +{ + "instances": 5, + "cpus": 3, + "mem": 2048.0, + "volume_size": 250 +} +``` + +If that configuration were stored in a file called `bk-config.json`, you could apply that configuration upon installating the BookKeeper package using this command: + +```shell +$ dcos package install bookkeeper \ + --options=./bk-config.json +``` + +## Uninstalling BookKeeper + +You can shut down and uninstall the `bookkeeper` from DC/OS at any time using the `package uninstall` command: + +```shell +$ dcos package uninstall bookkeeper +Uninstalled package [bookkeeper] version [4.5.0] +Thank you for using bookkeeper. +``` diff --git a/site/docs/4.5.1/deployment/kubernetes.md b/site/docs/4.5.1/deployment/kubernetes.md new file mode 100644 index 00000000000..f65172112a7 --- /dev/null +++ b/site/docs/4.5.1/deployment/kubernetes.md @@ -0,0 +1,4 @@ +--- +title: Deploying BookKeeper on Kubernetes +logo: img/kubernetes-logo.png +--- diff --git a/site/docs/4.5.1/deployment/manual.md b/site/docs/4.5.1/deployment/manual.md new file mode 100644 index 00000000000..daafd5556f5 --- /dev/null +++ b/site/docs/4.5.1/deployment/manual.md @@ -0,0 +1,56 @@ +--- +title: Manual deployment +--- + +The easiest way to deploy BookKeeper is using schedulers like [DC/OS](../dcos), but you can also deploy BookKeeper clusters manually. A BookKeeper cluster consists of two main components: + +* A [ZooKeeper](#zookeeper-setup) cluster that is used for configuration- and coordination-related tasks +* An [ensemble](#starting-up-bookies) of {% pop bookies %} + +## ZooKeeper setup + +We won't provide a full guide to setting up a ZooKeeper cluster here. We recommend that you consult [this guide](https://zookeeper.apache.org/doc/current/zookeeperAdmin.html) in the official ZooKeeper documentation. + +## Starting up bookies + +Once your ZooKeeper cluster is up and running, you can start up as many {% pop bookies %} as you'd like to form a cluster. Before starting up each bookie, you need to modify the bookie's configuration to make sure that it points to the right ZooKeeper cluster. + +On each bookie host, you need to [download](../../getting-started/installation#download) the BookKeeper package as a tarball. Once you've done that, you need to configure the bookie by setting values in the `bookkeeper-server/conf/bk_server.conf` config file. The one parameter that you will absolutely need to change is the [`zkServers`](../../config#zkServers) parameter, which you will need to set to the ZooKeeper connection string for your ZooKeeper cluster. Here's an example: + +```properties +zkServers=100.0.0.1:2181,100.0.0.2:2181,100.0.0.3:2181 +``` + +> A full listing of configurable parameters available in `bookkeeper-server/conf/bk_server.conf` can be found in the [Configuration](../../reference/config) reference manual. + +Once the bookie's configuration is set, you can start it up using the [`bookie`](../../reference/cli#bookkeeper-bookie) command of the [`bookkeeper`](../../reference/cli#bookkeeper) CLI tool: + +```shell +$ bookkeeper-server/bin/bookkeeper bookie +``` + +> You can also build BookKeeper [by cloning it from source](../../getting-started/installation#clone) or [using Maven](../../getting-started/installation#build-using-maven). + +### System requirements + +{% include system-requirements.md %} + +## Cluster metadata setup + +Once you've started up a cluster of bookies, you need to set up cluster metadata for the cluster by running the following command from any bookie in the cluster: + +```shell +$ bookkeeper-server/bin/bookkeeper shell metaformat +``` + +You can run in the formatting + +> The `metaformat` command performs all the necessary ZooKeeper cluster metadata tasks and thus only needs to be run *once* and from *any* bookie in the BookKeeper cluster. + +Once cluster metadata formatting has been completed, your BookKeeper cluster is ready to go! + + diff --git a/site/docs/4.5.1/development/codebase.md b/site/docs/4.5.1/development/codebase.md new file mode 100644 index 00000000000..9a83073ea4c --- /dev/null +++ b/site/docs/4.5.1/development/codebase.md @@ -0,0 +1,3 @@ +--- +title: The BookKeeper codebase +--- diff --git a/site/docs/4.5.1/development/protocol.md b/site/docs/4.5.1/development/protocol.md new file mode 100644 index 00000000000..6d17aa0ed45 --- /dev/null +++ b/site/docs/4.5.1/development/protocol.md @@ -0,0 +1,148 @@ +--- +title: The BookKeeper protocol +--- + +BookKeeper uses a special replication protocol for guaranteeing persistent storage of entries in an ensemble of bookies. + +> This document assumes that you have some knowledge of leader election and log replication and how these can be used in a distributed system. If not, we recommend reading the [example application](../../api/ledger-api#example-application) documentation first. + +## Ledgers + +{% pop Ledgers %} are the basic building block of BookKeeper and the level at which BookKeeper makes its persistent storage guarantees. A replicated log consists of an ordered list of ledgers. See [Ledgers to logs](#ledgers-to-logs) for info on building a replicated log from ledgers. + +Ledgers are composed of metadata and {% pop entries %}. The metadata is stored in ZooKeeper, which provides a *compare-and-swap* (CAS) operation. Entries are stored on storage nodes known as {% pop bookies %}. + +A ledger has a single writer and multiple readers (SWMR). + +### Ledger metadata + +A ledger's metadata contains the following: + +Parameter | Name | Meaning +:---------|:-----|:------- +Identifer | | A 64-bit integer, unique within the system +Ensemble size | **E** | The number of nodes the ledger is stored on +Write quorum size | **Qw** | The number of nodes each entry is written to. In effect, the max replication for the entry. +Ack quorum size | **Qa** | The number of nodes an entry must be acknowledged on. In effect, the minimum replication for the entry. +Current state | | The current status of the ledger. One of `OPEN`, `CLOSED`, or `IN_RECOVERY`. +Last entry | | The last entry in the ledger or `NULL` is the current state is not `CLOSED`. + +In addition, each ledger's metadata consists of one or more *fragments*. Each fragment is either + +* the first entry of a fragment or +* a list of bookies for the fragment. + +When creating a ledger, the following invariant must hold: + +**E >= Qw >= Qa** + +Thus, the ensemble size (**E**) must be larger than the write quorum size (**Qw**), which must in turn be larger than the ack quorum size (**Qa**). If that condition does not hold, then the ledger creation operation will fail. + +### Ensembles + +When a ledger is created, **E** bookies are chosen for the entries of that ledger. The bookies are the initial ensemble of the ledger. A ledger can have multiple ensembles, but an entry has only one ensemble. Changes in the ensemble involve a new fragment being added to the ledger. + +Take the following example. In this ledger, with ensemble size of 3, there are two fragments and thus two ensembles, one starting at entry 0, the second at entry 12. The second ensemble differs from the first only by its first element. This could be because bookie1 has failed and therefore had to be replaced. + +First entry | Bookies +:-----------|:------- +0 | B1, B2, B3 +12 | B4, B2, B3 + +### Write quorums + +Each entry in the log is written to **Qw** nodes. This is considered the write quorum for that entry. The write quorum is the subsequence of the ensemble, **Qw** in length, and starting at the bookie at index (entryid % **E**). + +For example, in a ledger of **E** = 4, **Qw**, and **Qa** = 2, with an ensemble consisting of B1, B2, B3, and B4, the write quorums for the first 6 entries will be: + +Entry | Write quorum +:-----|:------------ +0 | B1, B2, B3 +1 | B2, B3, B4 +2 | B3, B4, B1 +3 | B4, B1, B2 +4 | B1, B2, B3 +5 | B2, B3, B4 + +There are only **E** distinct write quorums in any ensemble. If **Qw** = **Qa**, then there is only one, as no striping occurs. + +### Ack quorums + +The ack quorum for an entry is any subset of the write quorum of size **Qa**. If **Qa** bookies acknowledge an entry, it means it has been fully replicated. + +### Guarantees + +The system can tolerate **Qa** – 1 failures without data loss. + +Bookkeeper guarantees that: + +1. All updates to a ledger will be read in the same order as they were written. +2. All clients will read the same sequence of updates from the ledger. + +## Writing to ledgers + +writer, ensuring that entry ids are sequential is trivial. A bookie acknowledges a write once it has been persisted to disk and is therefore durable. Once **Qa** bookies from the write quorum acknowledge the write, the write is acknowledged to the client, but only if all entries with lower entry ids in the ledger have already been acknowledged to the client. + +The entry written contains the ledger id, the entry id, the last add confirmed and the payload. The last add confirmed is the last entry which had been acknowledged to the client when this entry was written. Sending this with the entry speeds up recovery of the ledger in the case that the writer crashes. + +Another client can also read entries in the ledger up as far as the last add confirmed, as we guarantee that all entries thus far have been replicated on Qa nodes, and therefore all future readers will be able to also read it. However, to read like this, the ledger should be opened with a non-fencing open. Otherwise, it would kill the writer. + +If a node fails to acknowledge a write, the writer will create a new ensemble by replacing the failed node in the current ensemble. It creates a new fragment with this ensemble, starting from the first message that has not been acknowledged to the client. Creating the new fragment involves making a CAS write to the metadata. If the CAS write fails, someone else has modified something in the ledger metadata. This concurrent modification could have been caused by recovery or {% pop rereplication %}. We reread the metadata. If the state of the ledger is no longer `OPEN`, we send an error to the client for any outstanding writes. Otherwise, we try to replace the failed node again. + +### Closing a ledger as a writer + +Closing a ledger is straightforward for a writer. The writer makes a CAS write to the metadata, changing the state to `CLOSED` and setting the last entry of the ledger to the last entry which we have acknowledged to the client. + +If the CAS write fails, it means someone else has modified the metadata. We reread the metadata, and retry closing as long as the state of the ledger is still `OPEN`. If the state is `IN_RECOVERY` we send an error to the client. If the state is `CLOSED` and the last entry is the same as the last entry we have acknowledged to the client, we complete the close operation successfully. If the last entry is different from what we have acknowledged to the client, we send an error to the client. + +### Closing a ledger as a reader + +A reader can also force a ledger to close. Forcing the ledger to close will prevent any writer from adding new entries to the ledger. This is called {% pop fencing %}. This can occur when a writer has crashed or become unavailable, and a new writer wants to take over writing to the log. The new writer must ensure that it has seen all updates from the previous writer, and prevent the previous writer from making any new updates before making any updates of its own. + +To recover a ledger, we first update the state in the metadata to IN_RECOVERY. We then send a fence message to all the bookies in the last fragment of the ledger. When a bookie receives a fence message for a ledger, the fenced state of the ledger is persisted to disk. Once we receive a response from at least (**Qw** - **Qa**)+1 bookies from each write quorum in the ensemble, the ledger is fenced. + +By ensuring we have received a response from at last (**Qw** - **Qa**) + 1 bookies in each write quorum, we ensure that, if the old writer is alive and tries to add a new entry there will be no write quorum in which Qa bookies will accept the write. If the old writer tries to update the ensemble, it will fail on the CAS metadata write, and then see that the ledger is in IN_RECOVERY state, and that it therefore shouldn’t try to write to it. + +The old writer will be able to write entries to individual bookies (we can’t guarantee that the fence message reaches all bookies), but as it will not be able reach ack quorum, it will not be able to send a success response to its client. The client will get a LedgerFenced error instead. + +It is important to note that when you get a ledger fenced message for an entry, it doesn’t mean that the entry has not been written. It means that the entry may or may not have been written, and this can only be determined after the ledger is recovered. In effect, LedgerFenced should be treated like a timeout. + +Once the ledger is fenced, recovery can begin. Recovery means finding the last entry of the ledger and closing the ledger. To find the last entry of the ledger, the client asks all bookies for the highest last add confirmed value they have seen. It waits until it has received a response at least (**Qw** - **Qa**) + 1 bookies from each write quorum, and takes the highest response as the entry id to start reading forward from. It then starts reading forward in the ledger, one entry at a time, replicating all entries it sees to the entire write quorum for that entry. Once it can no longer read any more entries, it updates the state in the metadata to `CLOSED`, and sets the last entry of the ledger to the last entry it wrote. Multiple readers can try to recovery a ledger at the same time, but as the metadata write is CAS they will all converge on the same last entry of the ledger. + +## Ledgers to logs + +In BookKeeper, {% pop ledgers %} can be used to build a replicated log for your system. All guarantees provided by BookKeeper are at the ledger level. Guarantees on the whole log can be built using the ledger guarantees and any consistent datastore with a compare-and-swap (CAS) primitive. BookKeeper uses ZooKeeper as the datastore but others could theoretically be used. + +A log in BookKeeper is built from some number of ledgers, with a fixed order. A ledger represents a single segment of the log. A ledger could be the whole period that one node was the leader, or there could be multiple ledgers for a single period of leadership. However, there can only ever be one leader that adds entries to a single ledger. Ledgers cannot be reopened for writing once they have been closed/recovered. + +> BookKeeper does *not* provide leader election. You must use a system like ZooKeeper for this. + +In many cases, leader election is really leader suggestion. Multiple nodes could think that they are leader at any one time. It is the job of the log to guarantee that only one can write changes to the system. + +### Opening a log + +Once a node thinks it is leader for a particular log, it must take the following steps: + +1. Read the list of ledgers for the log +1. {% pop Fence %} the last two ledgers in the list. Two ledgers are fenced because because the writer may be writing to the second-to-last ledger while adding the last ledger to the list. +1. Create a new ledger +1. Add the new ledger to the ledger list +1. Write the new ledger back to the datastore using a CAS operation + +The fencing in step 2 and the CAS operation in step 5 prevent two nodes from thinking that they have leadership at any one time. + +The CAS operation will fail if the list of ledgers has changed between reading it and writing back the new list. When the CAS operation fails, the leader must start at step 1 again. Even better, they should check that they are in fact still the leader with the system that is providing leader election. The protocol will work correctly without this step, though it will be able to make very little progress if two nodes think they are leader and are duelling for the log. + +The node must not serve any writes until step 5 completes successfully. + +### Rolling ledgers + +The leader may wish to close the current ledger and open a new one every so often. Ledgers can only be deleted as a whole. If you don't roll the log, you won't be able to clean up old entries in the log without a leader change. By closing the current ledger and adding a new one, the leader allows the log to be truncated whenever that data is no longer needed. The steps for rolling the log is similar to those for creating a new ledger. + +1. Create a new ledger +1. Add the new ledger to the ledger list +1. Write the new ledger list to the datastore using CAS +1. Close the previous ledger + +By deferring the closing of the previous ledger until step 4, we can continue writing to the log while we perform metadata update operations to add the new ledger. This is safe as long as you fence the last 2 ledgers when acquiring leadership. + diff --git a/site/docs/4.5.1/example.md b/site/docs/4.5.1/example.md new file mode 100644 index 00000000000..7dbc6979d0c --- /dev/null +++ b/site/docs/4.5.1/example.md @@ -0,0 +1,6 @@ +--- +title: Example doc +subtitle: Just for experimentation purposes. +--- + +{% pop ledger %} diff --git a/site/docs/4.5.1/getting-started/concepts.md b/site/docs/4.5.1/getting-started/concepts.md new file mode 100644 index 00000000000..7a3c92847b2 --- /dev/null +++ b/site/docs/4.5.1/getting-started/concepts.md @@ -0,0 +1,202 @@ +--- +title: BookKeeper concepts and architecture +subtitle: The core components and how they work +prev: ../run-locally +--- + +BookKeeper is a service that provides persistent storage of streams of log [entries](#entries)---aka *records*---in sequences called [ledgers](#ledgers). BookKeeper replicates stored entries across multiple servers. + +## Basic terms + +In BookKeeper: + +* each unit of a log is an [*entry*](#entries) (aka record) +* streams of log entries are called [*ledgers*](#ledgers) +* individual servers storing ledgers of entries are called [*bookies*](#bookies) + +BookKeeper is designed to be reliable and resilient to a wide variety of failures. Bookies can crash, corrupt data, or discard data, but as long as there are enough bookies behaving correctly in the ensemble the service as a whole will behave correctly. + +## Entries + +> **Entries** contain the actual data written to ledgers, along with some important metadata. + +BookKeeper entries are sequences of bytes that are written to [ledgers](#ledgers). Each entry has the following fields: + +Field | Java type | Description +:-----|:----------|:----------- +Ledger number | `long` | The ID of the ledger to which the entry has been written +Entry number | `long` | The unique ID of the entry +Last confirmed (LC) | `long` | The ID of the last recorded entry +Data | `byte[]` | The entry's data (written by the client application) +Authentication code | `byte[]` | The message auth code, which includes *all* other fields in the entry + +## Ledgers + +> **Ledgers** are the basic unit of storage in BookKeeper. + +Ledgers are sequences of entries, while each entry is a sequence of bytes. Entries are written to a ledger: + +* sequentially, and +* at most once. + +This means that ledgers have *append-only* semantics. Entries cannot be modified once they've been written to a ledger. Determining the proper write order is the responsbility of [client applications](#clients). + +## Clients and APIs + +> BookKeeper clients have two main roles: they create and delete ledgers, and they read entries from and write entries to ledgers. +> +> BookKeeper provides both a lower-level and a higher-level API for ledger interaction. + +There are currently two APIs that can be used for interacting with BookKeeper: + +* The [ledger API](../../api/ledger-api) is a lower-level API that enables you to interact with {% pop ledgers %} directly. +* The [DistributedLog API](../../api/distributedlog-api) is a higher-level API that enables you to use BookKeeper without directly interacting with ledgers. + +In general, you should choose the API based on how much granular control you need over ledger semantics. The two APIs can also both be used within a single application. + +## Bookies + +> **Bookies** are individual BookKeeper servers that handle ledgers (more specifically, fragments of ledgers). Bookies function as part of an ensemble. + +A bookie is an individual BookKeeper storage server. Individual bookies store fragments of ledgers, not entire ledgers (for the sake of performance). For any given ledger **L**, an *ensemble* is the group of bookies storing the entries in **L**. + +Whenever entries are written to a ledger, those entries are {% pop striped %} across the ensemble (written to a sub-group of bookies rather than to all bookies). + +### Motivation + +> BookKeeper was initially inspired by the NameNode server in HDFS but its uses now extend far beyond this. + +The initial motivation for BookKeeper comes from the [Hadoop](http://hadoop.apache.org/) ecosystem. In the [Hadoop Distributed File System](https://wiki.apache.org/hadoop/HDFS) (HDFS), a special node called the [NameNode](https://wiki.apache.org/hadoop/NameNode) logs all operations in a reliable fashion, which ensures that recovery is possible in case of crashes. + +The NameNode, however, served only as initial inspiration for BookKeeper. The applications for BookKeeper extend far beyond this and include essentially any application that requires an append-based storage system. BookKeeper provides a number of advantages for such applications: + +* Highly efficient writes +* High fault tolerance via replication of messages within ensembles of bookies +* High throughput for write operations via {% pop striping %} (across as many bookies as you wish) + +## Metadata storage + +BookKeeper requires a metadata storage service to store information related to [ledgers](#ledgers) and available bookies. BookKeeper currently uses [ZooKeeper](https://zookeeper.apache.org) for this and other tasks. + +## Data management in bookies + +Bookies manage data in a [log-structured](https://en.wikipedia.org/wiki/Log-structured_file_system) way, which is implemented using three types of files: + +* [journals](#journals) +* [entry logs](#entry-logs) +* [index files](#index-files) + +### Journals + +A journal file contains BookKeeper transaction logs. Before any update to a ledger takes place, the bookie ensures that a transaction describing the update is written to non-volatile storage. A new journal file is created once the bookie starts or the older journal file reaches the journal file size threshold. + +### Entry logs + +An entry log file manages the written entries received from BookKeeper clients. Entries from different ledgers are aggregated and written sequentially, while their offsets are kept as pointers in a [ledger cache](#ledger-cache) for fast lookup. + +A new entry log file is created once the bookie starts or the older entry log file reaches the entry log size threshold. Old entry log files are removed by the Garbage Collector Thread once they are not associated with any active ledger. + +### Index files + +An index file is created for each ledger, which comprises a header and several fixed-length index pages that record the offsets of data stored in entry log files. + +Since updating index files would introduce random disk I/O index files are updated lazily by a sync thread running in the background. This ensures speedy performance for updates. Before index pages are persisted to disk, they are gathered in a ledger cache for lookup. + +### Ledger cache + +Ledger indexes pages are cached in a memory pool, which allows for more efficient management of disk head scheduling. + +### Adding entries + +When a client instructs a {% pop bookie %} to write an entry to a ledger, the entry will go through the following steps to be persisted on disk: + +1. The entry is appended to an [entry log](#entry-logs) +1. The index of the entry is updated in the [ledger cache](#ledger-cache) +1. A transaction corresponding to this entry update is appended to the [journal](#journals) +1. A response is sent to the BookKeeper client + +> For performance reasons, the entry log buffers entries in memory and commits them in batches, while the ledger cache holds index pages in memory and flushes them lazily. This process is described in more detail in the [Data flush](#data-flush) section below. + +### Data flush + +Ledger index pages are flushed to index files in the following two cases: + +* The ledger cache memory limit is reached. There is no more space available to hold newer index pages. Dirty index pages will be evicted from the ledger cache and persisted to index files. +* A background thread synchronous thread is responsible for flushing index pages from the ledger cache to index files periodically. + +Besides flushing index pages, the sync thread is responsible for rolling journal files in case that journal files use too much disk space. The data flush flow in the sync thread is as follows: + +* A `LastLogMark` is recorded in memory. The `LastLogMark` indicates that those entries before it have been persisted (to both index and entry log files) and contains two parts: + 1. A `txnLogId` (the file ID of a journal) + 1. A `txnLogPos` (offset in a journal) +* Dirty index pages are flushed from the ledger cache to the index file, and entry log files are flushed to ensure that all buffered entries in entry log files are persisted to disk. + + Ideally, a bookie only needs to flush index pages and entry log files that contain entries before `LastLogMark`. There is, however, no such information in the ledger and entry log mapping to journal files. Consequently, the thread flushes the ledger cache and entry log entirely here, and may flush entries after the `LastLogMark`. Flushing more is not a problem, though, just redundant. +* The `LastLogMark` is persisted to disk, which means that entries added before `LastLogMark` whose entry data and index page were also persisted to disk. It is now time to safely remove journal files created earlier than `txnLogId`. + +If the bookie has crashed before persisting `LastLogMark` to disk, it still has journal files containing entries for which index pages may not have been persisted. Consequently, when this bookie restarts, it inspects journal files to restore those entries and data isn't lost. + +Using the above data flush mechanism, it is safe for the sync thread to skip data flushing when the bookie shuts down. However, in the entry logger it uses a buffered channel to write entries in batches and there might be data buffered in the buffered channel upon a shut down. The bookie needs to ensure that the entry log flushes its buffered data during shutdown. Otherwise, entry log files become corrupted with partial entries. + +### Data compaction + +On bookies, entries of different ledgers are interleaved in entry log files. A bookie runs a garbage collector thread to delete un-associated entry log files to reclaim disk space. If a given entry log file contains entries from a ledger that has not been deleted, then the entry log file would never be removed and the occupied disk space never reclaimed. In order to avoid such a case, a bookie server compacts entry log files in a garbage collector thread to reclaim disk space. + +There are two kinds of compaction running with different frequency: minor compaction and major compaction. The differences between minor compaction and major compaction lies in their threshold value and compaction interval. + +* The garbage collection threshold is the size percentage of an entry log file occupied by those undeleted ledgers. The default minor compaction threshold is 0.2, while the major compaction threshold is 0.8. +* The garbage collection interval is how frequently to run the compaction. The default minor compaction interval is 1 hour, while the major compaction threshold is 1 day. + +> If either the threshold or interval is set to less than or equal to zero, compaction is disabled. + +The data compaction flow in the garbage collector thread is as follows: + +* The thread scans entry log files to get their entry log metadata, which records a list of ledgers comprising an entry log and their corresponding percentages. +* With the normal garbage collection flow, once the bookie determines that a ledger has been deleted, the ledger will be removed from the entry log metadata and the size of the entry log reduced. +* If the remaining size of an entry log file reaches a specified threshold, the entries of active ledgers in the entry log will be copied to a new entry log file. +* Once all valid entries have been copied, the old entry log file is deleted. + +## ZooKeeper metadata + +BookKeeper requires a ZooKeeper installation for storing [ledger](#ledger) metadata. Whenever you construct a [`BookKeeper`](../../api/javadoc/org/apache/bookkeeper/client/BookKeeper) client object, you need to pass a list of ZooKeeper servers as a parameter to the constructor, like this: + +```java +String zkConnectionString = "127.0.0.1:2181"; +BookKeeper bkClient = new BookKeeper(zkConnectionString); +``` + +> For more info on using the BookKeeper Java client, see [this guide](../../api/ledger-api#the-java-ledger-api-client). + +## Ledger manager + +A *ledger manager* handles ledgers' metadata (which is stored in ZooKeeper). BookKeeper offers two types of ledger managers: the [flat ledger manager](#flat-ledger-manager) and the [hierarchical ledger manager](#hierarchical-ledger-manager). Both ledger managers extend the [`AbstractZkLedgerManager`](../../api/javadoc/org/apache/bookkeeper/meta/AbstractZkLedgerManager) abstract class. + +> #### Use the flat ledger manager in most cases +> The flat ledger manager is the default and is recommended for nearly all use cases. The hierarchical ledger manager is better suited only for managing very large numbers of BookKeeper ledgers (> 50,000). + +### Flat ledger manager + +The *flat ledger manager*, implemented in the [`FlatLedgerManager`](../../api/javadoc/org/apache/bookkeeper/meta/FlatLedgerManager.html) class, stores all ledgers' metadata in child nodes of a single ZooKeeper path. The flat ledger manager creates [sequential nodes](https://zookeeper.apache.org/doc/trunk/zookeeperProgrammers.html#Sequence+Nodes+--+Unique+Naming) to ensure the uniqueness of the ledger ID and prefixes all nodes with `L`. Bookie servers manage their own active ledgers in a hash map so that it's easy to find which ledgers have been deleted from ZooKeeper and then garbage collect them. + +The flat ledger manager's garbage collection follow proceeds as follows: + +* All existing ledgers are fetched from ZooKeeper (`zkActiveLedgers`) +* All ledgers currently active within the bookie are fetched (`bkActiveLedgers`) +* The currently actively ledgers are looped through to determine which ledgers don't currently exist in ZooKeeper. Those are then garbage collected. +* The *hierarchical ledger manager* stores ledgers' metadata in two-level [znodes](https://zookeeper.apache.org/doc/current/zookeeperOver.html#Nodes+and+ephemeral+nodes). + +### Hierarchical ledger manager + +The *hierarchical ledger manager*, implemented in the [`HierarchicalLedgerManager`](../../api/javadoc/org/apache/bookkeeper/meta/HierarchicalLedgerManager) class, first obtains a global unique ID from ZooKeeper using an [`EPHEMERAL_SEQUENTIAL`](https://zookeeper.apache.org/doc/current/api/org/apache/zookeeper/CreateMode.html#EPHEMERAL_SEQUENTIAL) znode. Since ZooKeeper's sequence counter has a format of `%10d` (10 digits with 0 padding, for example `0000000001`), the hierarchical ledger manager splits the generated ID into 3 parts: + +```shell +{level1 (2 digits)}{level2 (4 digits)}{level3 (4 digits)} +``` + +These three parts are used to form the actual ledger node path to store ledger metadata: + +```shell +{ledgers_root_path}/{level1}/{level2}/L{level3} +``` + +For example, ledger 0000000001 is split into three parts, 00, 0000, and 00001, and stored in znode `/{ledgers_root_path}/00/0000/L0001`. Each znode could have as many 10,000 ledgers, which avoids the problem of the child list being larger than the maximum ZooKeeper packet size (which is the [limitation](https://issues.apache.org/jira/browse/BOOKKEEPER-39) that initially prompted the creation of the hierarchical ledger manager). diff --git a/site/docs/4.5.1/getting-started/installation.md b/site/docs/4.5.1/getting-started/installation.md new file mode 100644 index 00000000000..fac16ddd390 --- /dev/null +++ b/site/docs/4.5.1/getting-started/installation.md @@ -0,0 +1,74 @@ +--- +title: BookKeeper installation +subtitle: Download or clone BookKeeper and build it locally +next: ../run-locally +--- + +{% capture download_url %}http://apache.claz.org/bookkeeper/bookkeeper-{{ site.latest_release }}/bookkeeper-{{ site.latest_release }}-src.tar.gz{% endcapture %} + +You can install BookKeeper either by [downloading](#download) a [GZipped](http://www.gzip.org/) tarball package or [cloning](#clone) the BookKeeper repository. + +## Requirements + +* [Unix environment](http://www.opengroup.org/unix) +* [Java Development Kit 1.6](http://www.oracle.com/technetwork/java/javase/downloads/index.html) or later +* [Maven 3.0](https://maven.apache.org/install.html) or later + +## Download + +You can download Apache BookKeeper releases from one of many [Apache mirrors](http://www.apache.org/dyn/closer.cgi/bookkeeper). Here's an example for the [apache.claz.org](http://apache.claz.org/bookkeeper) mirror: + +```shell +$ curl -O {{ download_url }} +$ tar xvf bookkeeper-{{ site.latest_release }}-src.tar.gz +$ cd bookkeeper-{{ site.latest_release }} +``` + +## Clone + +To build BookKeeper from source, clone the repository, either from the [GitHub mirror]({{ site.github_repo }}) or from the [Apache repository](http://git.apache.org/bookkeeper.git/): + +```shell +# From the GitHub mirror +$ git clone {{ site.github_repo}} + +# From Apache directly +$ git clone git://git.apache.org/bookkeeper.git/ +``` + +## Build using Maven + +Once you have the BookKeeper on your local machine, either by [downloading](#download) or [cloning](#clone) it, you can then build BookKeeper from source using Maven: + +```shell +$ mvn package +``` + +> You can skip tests by adding the `-DskipTests` flag when running `mvn package`. + +### Useful Maven commands + +Some other useful Maven commands beyond `mvn package`: + +Command | Action +:-------|:------ +`mvn clean` | Removes build artifacts +`mvn compile` | Compiles JAR files from Java sources +`mvn compile findbugs:findbugs` | Compile using the Maven [FindBugs](http://gleclaire.github.io/findbugs-maven-plugin) plugin +`mvn install` | Install the BookKeeper JAR locally in your local Maven cache (usually in the `~/.m2` directory) +`mvn deploy` | Deploy the BookKeeper JAR to the Maven repo (if you have the proper credentials) +`mvn verify` | Performs a wide variety of verification and validation tasks +`mvn apache-rat:check` | Run Maven using the [Apache Rat](http://creadur.apache.org/rat/apache-rat-plugin/) plugin +`mvn compile javadoc:aggregate` | Build Javadocs locally +`mvn package assembly:single` | Build a complete distribution using the Maven [Assembly](http://maven.apache.org/plugins/maven-assembly-plugin/) plugin + +## Package directory + +The BookKeeper project contains several subfolders that you should be aware of: + +Subfolder | Contains +:---------|:-------- +[`bookkeeper-server`]({{ site.github_repo }}/tree/master/bookkeeper-server) | The BookKeeper server and client +[`bookkeeper-benchmark`]({{ site.github_repo }}/tree/master/bookkeeper-benchmark) | A benchmarking suite for measuring BookKeeper performance +[`bookkeeper-stats`]({{ site.github_repo }}/tree/master/bookkeeper-stats) | A BookKeeper stats library +[`bookkeeper-stats-providers`]({{ site.github_repo }}/tree/master/bookkeeper-stats-providers) | BookKeeper stats providers diff --git a/site/docs/4.5.1/getting-started/run-locally.md b/site/docs/4.5.1/getting-started/run-locally.md new file mode 100644 index 00000000000..ab33642c852 --- /dev/null +++ b/site/docs/4.5.1/getting-started/run-locally.md @@ -0,0 +1,16 @@ +--- +title: Run bookies locally +prev: ../installation +next: ../concepts +toc_disable: true +--- + +{% pop Bookies %} are individual BookKeeper servers. You can run an ensemble of bookies locally on a single machine using the [`localbookie`](../../reference/cli#bookkeeper-localbookie) command of the `bookkeeper` CLI tool and specifying the number of bookies you'd like to include in the ensemble. + +This would start up an ensemble with 10 bookies: + +```shell +$ bookeeper-server/bin/bookeeper localbookie 10 +``` + +> When you start up an ensemble using `localbookie`, all bookies run in a single JVM process. diff --git a/site/docs/4.5.1/overview/overview.md b/site/docs/4.5.1/overview/overview.md new file mode 100644 index 00000000000..ffc79806990 --- /dev/null +++ b/site/docs/4.5.1/overview/overview.md @@ -0,0 +1,57 @@ +--- +title: Apache BookKeeper&trades; 4.5.0 Documentation +--- + + +This documentation is for Apache BookKeeper™ version `4.5.0`. + +Apache BookKeeper™ is a scalable, fault tolerant and low latency storage service optimized for realtime workloads. +It offers `durability`, `replication` and `strong consistency` as essentials for building reliable real-time applications. + +It is suitable for being used in following scenerios: + +- [WAL](https://en.wikipedia.org/wiki/Write-ahead_logging) (Write-Ahead-Logging), e.g. HDFS [namenode](https://hadoop.apache.org/docs/r2.5.2/hadoop-project-dist/hadoop-hdfs/HDFSHighAvailabilityWithNFS.html#BookKeeper_as_a_Shared_storage_EXPERIMENTAL). +- Message Store, e.g. [Apache Pulsar](https://pulsar.incubator.apache.org/). +- Offset/Cursor Store, e.g. Apache Pulsar. +- Object/Blob Store, e.g. storing snapshots to replicated state machines. + +Learn more about Apache BookKeeper and what it can do for your organization: + +- [Apache BookKeeper 4.5.0 Release Notes](../releaseNotes) +- [Java API docs](../../api/javadoc) + +Or start using Apache BookKeeper today. + +### Users + +- **Concepts**: Start with [concepts](../../getting-started/concepts). This will help you to fully understand + the other parts of the documentation, including the setup, integration and operation guides. +- **Getting Started**: Install [Apache BookKeeper](../../getting-started/installation) and run bookies [locally](../../getting-started/run-locally) +- **API**: Read the [API](../../api/overview) documentation to learn how to use Apache BookKeeper to build your applications. +- **Deployment**: The [Deployment Guide](../../deployment/manual) shows how to deploy Apache BookKeeper to production clusters. + +### Administrators + +- **Operations**: The [Admin Guide](../../admin/bookies) shows how to run Apache BookKeeper on production, what are the production + considerations and best practices. + +### Contributors + +- **Details**: Learn [design details](../../development/protocol) to know more internals. diff --git a/site/docs/4.5.1/overview/releaseNotes.md b/site/docs/4.5.1/overview/releaseNotes.md new file mode 100644 index 00000000000..f16ead5ac31 --- /dev/null +++ b/site/docs/4.5.1/overview/releaseNotes.md @@ -0,0 +1,501 @@ +--- +title: Apache BookKeeper 4.5.0 Release Notes +--- + +This is the sixth release of BookKeeper as an Apache Top Level Project! + +The 4.5.1 release is a bugfix release which fixes a bunch of issues found from users of 4.5.0. + +Apache BookKeeper users are encouraged to upgrade to 4.5.1. The technical details of this release are summarized +below. + +## Highlights + +TODO + +### Dependencies Upgrade + +Here is a list of dependencies upgraded in 4.5.0: + +- Moved the developement from Java 7 to Java 8. +- Upgrade Protobuf to `2.6`. +- Upgrade ZooKeeper from `3.4` to `3.5`. +- Upgrade Netty to `4.1`. +- Upgrade Guava to `20.0`. +- Upgrade SLF4J to `1.7.25`. +- Upgrade Codahale to `3.1.0`. + +### Security + +Prior to this release, Apache BookKeeper only supports simple `DIGEST-MD5` type authentication. + +With this release of Apache BookKeeper, a number of feature are introduced that can be used, together of separately, +to secure a BookKeeper cluster. + +The following security features are currently supported. + +- Authentication of connections to bookies from clients, using either `TLS` or `SASL (Kerberos). +- Authentication of connections from clients, bookies, autorecovery daemons to `ZooKeeper`, when using zookeeper + based ledger managers. +- Encryption of data transferred between bookies and clients, between bookies and autorecovery daemons using `TLS`. + +It's worth noting that those security features are optional - non-secured clusters are supported, as well as a mix +of authenticated, unauthenticated, encrypted and non-encrypted clients. + +For more details, have a look at [BookKeeper Security](../../security/overview). + +### Public API + +There are multiple new client features introduced in 4.5.0. + +#### LedgerHandleAdv + +The [Ledger API] is the low level API provides by BookKeeper for interacting with `ledgers` in a bookkeeper cluster. +It is simple but not flexible on ledger id or entry id generation. Apache BookKeeper introduces `LedgerHandleAdv` +as an extension of existing `LedgerHandle` for advanced usage. The new `LedgerHandleAdv` allows applications providing +its own `ledger-id` and assigning `entry-id` on adding entries. + +See [Ledger Advanced API](../../api/ledger-adv-api) for more details. + +#### Long Poll + +`Long Poll` is a main feature that [DistributedLog](https://distributedlog.io) uses to achieve low-latency tailing. +This big feature has been merged back in 4.5.0 and available to BookKeeper users. + +This feature includes two main changes, one is `LastAddConfirmed` piggyback, while the other one is a new `long poll` read API. + +The first change piggyback the latest `LastAddConfirm` along with the read response, so your `LastAddConfirmed` will be automatically advanced +when your read traffic continues. It significantly reduces the traffic to explicitly polling `LastAddConfirmed` and hence reduces the end-to-end latency. + +The second change provides a new `long poll` read API, allowing tailing-reads without polling `LastAddConfirmed` everytime after readers exhaust known entries. +Although `long poll` API brings great latency improvements on tailing reads, it is still a very low-level primitive. +It is still recommended to use high level API (e.g. [DistributedLog API](../../api/distributedlog-api)) for tailing and streaming use cases. + +See [Streaming Reads](https://distributedlog.incubator.apache.org/docs/latest/user_guide/design/main.html#streaming-reads) for more details. + +#### Explicit LAC + +Prior to 4.5.0, the `LAC` is only advanced when subsequent entries are added. If there is no subsequent entries added, +the last entry written will not be visible to readers until the ledger is closed. High-level client (e.g. DistributedLog) or applications +has to work around this by writing some sort of `control records` to advance `LAC`. + +In 4.5.0, a new `explicit lac` feature is introduced to periodically advance `LAC` if there are not subsequent entries added. This feature +can be enabled by setting `explicitLacInterval` to a positive value. + +### Performance + +There are a lot for performance related bug fixes and improvements in 4.5.0. These changes includes: + +- Upgraded netty from 3.x to 4.x to leverage buffer pooling and reduce memory copies. +- Moved developement from Java 7 to Java 8 to take advantage of Java 8 features. +- A lot of improvements around scheduling and threading on `bookies`. +- Delay ensemble change to improve tail latency. +- Parallel ledger recovery to improve the recovery speed. +- ... + +We outlined following four changes as below. For a complete list of performance improvements, please checkout the `full list of changes` at the end. + +#### Netty 4 Upgrade + +The major performance improvement introduced in 4.5.0, is upgrading netty from 3.x to [4.x](http://netty.io/wiki/new-and-noteworthy-in-4.0.html). + +For more details, please read [upgrade guide](../../admin/upgrade) about the netty related tips when upgrading bookkeeper from 4.4.0 to 4.5.0. + +#### Delay Ensemble Change + +`Ensemble Change` is a feature that Apache BookKeeper uses to achieve high availability. However it is an expensive metadata operation. +Especially when Apache BookKeeper is deployed in a multiple data-centers environment, losing a data center will cause churn of metadata +operations due to ensemble changes. `Delay Ensemble Change` is introduced in 4.5.0 to overcome this problem. Enabling this feature means +an `Ensemble Change` will only occur when clients can't receive enough valid responses to satisfy `ack-quorum` constraint. This feature +improves the tail latency. + +To enable this feature, please set `delayEnsembleChange` to `true` on your clients. + +#### Parallel Ledger Recovery + +BookKeeper clients recovers entries one-by-one during ledger recovery. If a ledger has very large volumn of traffic, it will have +large number of entries to recover when client failures occur. BookKeeper introduces `parallel ledger recovery` in 4.5.0 to allow +batch recovery to improve ledger recovery speed. + +To enable this feature, please set `enableParallelRecoveryRead` to `true` on your clients. You can also set `recoveryReadBatchSize` +to control the batch size of recovery read. + +#### Multiple Journals + +Prior to 4.5.0, bookies are only allowed to configure one journal device. If you want to have high write bandwidth, you can raid multiple +disks into one device and mount that device for jouranl directory. However because there is only one journal thread, this approach doesn't +actually improve the write bandwidth. + +BookKeeper introduces multiple journal directories support in 4.5.0. Users can configure multiple devices for journal directories. + +To enable this feature, please use `journalDirectories` rather than `journalDirectory`. + +### Operations + +#### LongHierarchicalLedgerManager + +Apache BookKeeper supports pluggable metadata store. By default, it uses Apache ZooKeeper as its metadata store. Among the zookeeper-based +ledger manager implementations, `HierarchicalLedgerManager` is the most popular and widely adopted ledger manager. However it has a major +limitation, which it assumes `ledger-id` is a 32-bits integer. It limits the number of ledgers to `2^32`. + +`LongHierarchicalLedgerManager` is introduced to overcome this limitation. + +See [Ledger Manager](../../getting-started/concepts/#ledger-manager) for more details. + +#### Weight-based placement policy + +`Rack-Aware` and `Region-Aware` placement polices are the two available placement policies in BookKeeper client. It places ensembles based +on users' configured network topology. However they both assume that all nodes are equal. `weight-based` placement is introduced in 4.5.0 to +improve the existing placement polices. `weight-based` placement was not built as separated polices. It is built in the existing placement policies. +If you are using `Rack-Aware` or `Region-Aware`, you can simply enable `weight-based` placement by setting `diskWeightBasedPlacementEnabled` to `true`. + +#### Customized Ledger Metadata + +A `Map` is introduced in ledger metadata in 4.5.0. Clients now are allowed to pass in a key/value map when creating ledgers. +This customized ledger metadata can be later on used by user defined placement policy. This extends the flexibility of bookkeeper API. + +#### Add Prometheus stats provider + +A new [Prometheus](https://prometheus.io/) [stats provider](https://github.com/apache/bookkeeper/tree/master/bookkeeper-stats-providers/prometheus-metrics-provider) +is introduce in 4.5.0. It simplies the metric collection when running bookkeeper on [kubernetes](https://kubernetes.io/). + +#### Add more tools in BookieShell + +`BookieShell` is the tool provided by Apache BooKeeper to operate clusters. There are multiple importants tools introduced in 4.5.0, for example, `decommissionbookie`, +`expandstorage`, `lostbookierecoverydelay`, `triggeraudit`. + +For the complete list of commands in `BookieShell`, please read [BookKeeper CLI tool reference](../../reference/cli). + +## Full list of changes + +### JIRA + +#### Sub-task +

+ +#### Bug +
    +
  • [BOOKKEEPER-852] - Release LedgerDescriptor and master-key objects when not used anymore +
  • +
  • [BOOKKEEPER-903] - MetaFormat BookieShell Command is not deleting UnderReplicatedLedgers list from the ZooKeeper +
  • +
  • [BOOKKEEPER-907] - for ReadLedgerEntriesCmd, EntryFormatter should be configurable and HexDumpEntryFormatter should be one of them +
  • +
  • [BOOKKEEPER-908] - Case to handle BKLedgerExistException +
  • +
  • [BOOKKEEPER-924] - addEntry() is susceptible to spurious wakeups +
  • +
  • [BOOKKEEPER-927] - Extend BOOKKEEPER-886 to LedgerHandleAdv too (BOOKKEEPER-886: Allow to disable ledgers operation throttling) +
  • +
  • [BOOKKEEPER-933] - ClientConfiguration always inherits System properties +
  • +
  • [BOOKKEEPER-938] - LedgerOpenOp should use digestType from metadata +
  • +
  • [BOOKKEEPER-939] - Fix typo in bk-merge-pr.py +
  • +
  • [BOOKKEEPER-940] - Fix findbugs warnings after bumping to java 8 +
  • +
  • [BOOKKEEPER-952] - Fix RegionAwarePlacementPolicy +
  • +
  • [BOOKKEEPER-955] - in BookKeeperAdmin listLedgers method currentRange variable is not getting updated to next iterator when it has run out of elements +
  • +
  • [BOOKKEEPER-956] - HierarchicalLedgerManager doesn't work for ledgerid of length 9 and 10 because of order issue in HierarchicalLedgerRangeIterator +
  • +
  • [BOOKKEEPER-958] - ZeroBuffer readOnlyBuffer returns ByteBuffer with 0 remaining bytes for length > 64k +
  • +
  • [BOOKKEEPER-959] - ClientAuthProvider and BookieAuthProvider Public API used Protobuf Shaded classes +
  • +
  • [BOOKKEEPER-976] - Fix license headers with "Copyright 2016 The Apache Software Foundation" +
  • +
  • [BOOKKEEPER-980] - BookKeeper Tools doesn't process the argument correctly +
  • +
  • [BOOKKEEPER-981] - NullPointerException in RackawareEnsemblePlacementPolicy while running in Docker Container +
  • +
  • [BOOKKEEPER-984] - BookieClientTest.testWriteGaps tested +
  • +
  • [BOOKKEEPER-986] - Handle Memtable flush failure +
  • +
  • [BOOKKEEPER-987] - BookKeeper build is broken due to the shade plugin for commit ecbb053e6e +
  • +
  • [BOOKKEEPER-988] - Missing license headers +
  • +
  • [BOOKKEEPER-989] - Enable travis CI for bookkeeper git +
  • +
  • [BOOKKEEPER-999] - BookKeeper client can leak threads +
  • +
  • [BOOKKEEPER-1013] - Fix findbugs errors on latest master +
  • +
  • [BOOKKEEPER-1018] - Allow client to select older V2 protocol (no protobuf) +
  • +
  • [BOOKKEEPER-1020] - Fix Explicit LAC tests on master +
  • +
  • [BOOKKEEPER-1021] - Improve the merge script to handle github reviews api +
  • +
  • [BOOKKEEPER-1031] - ReplicationWorker.rereplicate fails to call close() on ReadOnlyLedgerHandle +
  • +
  • [BOOKKEEPER-1044] - Entrylogger is not readding rolled logs back to the logChannelsToFlush list when exception happens while trying to flush rolled logs +
  • +
  • [BOOKKEEPER-1047] - Add missing error code in ZK setData return path +
  • +
  • [BOOKKEEPER-1058] - Ignore already deleted ledger on replication audit +
  • +
  • [BOOKKEEPER-1061] - BookieWatcher should not do ZK blocking operations from ZK async callback thread +
  • +
  • [BOOKKEEPER-1065] - OrderedSafeExecutor should only have 1 thread per bucket +
  • +
  • [BOOKKEEPER-1071] - BookieRecoveryTest is failing due to a Netty4 IllegalReferenceCountException +
  • +
  • [BOOKKEEPER-1072] - CompactionTest is flaky when disks are almost full +
  • +
  • [BOOKKEEPER-1073] - Several stats provider related changes. +
  • +
  • [BOOKKEEPER-1074] - Remove JMX Bean +
  • +
  • [BOOKKEEPER-1075] - BK LedgerMetadata: more memory-efficient parsing of configs +
  • +
  • [BOOKKEEPER-1076] - BookieShell should be able to read the 'FENCE' entry in the log +
  • +
  • [BOOKKEEPER-1077] - BookKeeper: Local Bookie Journal and ledger paths +
  • +
  • [BOOKKEEPER-1079] - shell lastMark throws NPE +
  • +
  • [BOOKKEEPER-1098] - ZkUnderreplicationManager can build up an unbounded number of watchers +
  • +
  • [BOOKKEEPER-1101] - BookKeeper website menus not working under https +
  • +
  • [BOOKKEEPER-1102] - org.apache.bookkeeper.client.BookKeeperDiskSpaceWeightedLedgerPlacementTest.testDiskSpaceWeightedBookieSelectionWithBookiesBeingAdded is unreliable +
  • +
  • [BOOKKEEPER-1103] - LedgerMetadataCreateTest bug in ledger id generation causes intermittent hang +
  • +
  • [BOOKKEEPER-1104] - BookieInitializationTest.testWithDiskFullAndAbilityToCreateNewIndexFile testcase is unreliable +
  • +
+ +#### Improvement + + +#### New Feature +
    +
  • [BOOKKEEPER-390] - Provide support for ZooKeeper authentication +
  • +
  • [BOOKKEEPER-391] - Support Kerberos authentication of bookkeeper +
  • +
  • [BOOKKEEPER-575] - Bookie SSL support +
  • +
  • [BOOKKEEPER-670] - Longpoll Read & Piggyback Support +
  • +
  • [BOOKKEEPER-912] - Allow EnsemblePlacementPolicy to choose bookies using ledger custom data (multitenancy support) +
  • +
  • [BOOKKEEPER-928] - Add custom client supplied metadata field to LedgerMetadata +
  • +
  • [BOOKKEEPER-930] - Option to disable Bookie networking +
  • +
  • [BOOKKEEPER-941] - Introduce Feature Switches For controlling client and server behavior +
  • +
  • [BOOKKEEPER-948] - Provide an option to add more ledger/index directories to a bookie +
  • +
  • [BOOKKEEPER-950] - Ledger placement policy to accomodate different storage capacity of bookies +
  • +
  • [BOOKKEEPER-969] - Security Support +
  • +
  • [BOOKKEEPER-983] - BookieShell Command for LedgerDelete +
  • +
  • [BOOKKEEPER-991] - bk shell - Get a list of all on disk files +
  • +
  • [BOOKKEEPER-992] - ReadLog Command Enhancement +
  • +
  • [BOOKKEEPER-1019] - Support for reading entries after LAC (causal consistency driven by out-of-band communications) +
  • +
  • [BOOKKEEPER-1034] - When all disks are full, start Bookie in RO mode if RO mode is enabled +
  • +
  • [BOOKKEEPER-1067] - Add Prometheus stats provider +
  • +
+ +#### Story + + +#### Task + + +#### Test +
    +
  • [BOOKKEEPER-967] - Create new testsuite for testing RackAwareEnsemblePlacementPolicy using ScriptBasedMapping. +
  • +
  • [BOOKKEEPER-1045] - Execute tests in different JVM processes +
  • +
  • [BOOKKEEPER-1064] - ConcurrentModificationException in AuditorLedgerCheckerTest +
  • +
  • [BOOKKEEPER-1078] - Local BookKeeper enhancements for testability +
  • +
  • [BOOKKEEPER-1097] - GC test when no WritableDirs +
  • +
+ +#### Wish +
    +
  • [BOOKKEEPER-943] - Reduce log level of AbstractZkLedgerManager for register/unregister ReadOnlyLedgerHandle +
  • +
+ +### Github + +- [https://github.com/apache/bookkeeper/milestone/1](https://github.com/apache/bookkeeper/milestone/1) diff --git a/site/docs/4.5.1/overview/releaseNotesTemplate.md b/site/docs/4.5.1/overview/releaseNotesTemplate.md new file mode 100644 index 00000000000..c7845ae51bf --- /dev/null +++ b/site/docs/4.5.1/overview/releaseNotesTemplate.md @@ -0,0 +1,17 @@ +--- +title: Apache BookKeeper 4.5.0 Release Notes +--- + +[provide a summary of this release] + +Apache BookKeeper users are encouraged to upgrade to 4.5.0. The technical details of this release are summarized +below. + +## Highlights + +[List the highlights] + +## Details + +[list to issues list] + diff --git a/site/docs/4.5.1/reference/cli.md b/site/docs/4.5.1/reference/cli.md new file mode 100644 index 00000000000..8beb36ff071 --- /dev/null +++ b/site/docs/4.5.1/reference/cli.md @@ -0,0 +1,10 @@ +--- +title: BookKeeper CLI tool reference +subtitle: A reference guide to the command-line tools that you can use to administer BookKeeper +--- + +{% include cli.html id="bookkeeper" %} + +## The BookKeeper shell + +{% include shell.html %} diff --git a/site/docs/4.5.1/reference/config.md b/site/docs/4.5.1/reference/config.md new file mode 100644 index 00000000000..8997b6b62f0 --- /dev/null +++ b/site/docs/4.5.1/reference/config.md @@ -0,0 +1,9 @@ +--- +title: BookKeeper configuration +subtitle: A reference guide to all of BookKeeper's configurable parameters +--- + + +The table below lists parameters that you can set to configure {% pop bookies %}. All configuration takes place in the `bk_server.conf` file in the `bookkeeper-server/conf` directory of your [BookKeeper installation](../../getting-started/installing). + +{% include config.html id="bk_server" %} diff --git a/site/docs/4.5.1/reference/metrics.md b/site/docs/4.5.1/reference/metrics.md new file mode 100644 index 00000000000..8bd6fe0a165 --- /dev/null +++ b/site/docs/4.5.1/reference/metrics.md @@ -0,0 +1,3 @@ +--- +title: BookKeeper metrics reference +--- diff --git a/site/docs/4.5.1/security/overview.md b/site/docs/4.5.1/security/overview.md new file mode 100644 index 00000000000..62da8edae76 --- /dev/null +++ b/site/docs/4.5.1/security/overview.md @@ -0,0 +1,21 @@ +--- +title: BookKeeper Security +next: ../tls +--- + +In the 4.5.0 release, the BookKeeper community added a number of features that can be used, together or separately, to secure a BookKeeper cluster. +The following security measures are currently supported: + +1. Authentication of connections to bookies from clients, using either [TLS](../tls) or [SASL (Kerberos)](../sasl). +2. Authentication of connections from clients, bookies, autorecovery daemons to [ZooKeeper](../zookeeper), when using zookeeper based ledger managers. +3. Encryption of data transferred between bookies and clients, between bookies and autorecovery daemons using [TLS](../tls). + +It’s worth noting that security is optional - non-secured clusters are supported, as well as a mix of authenticated, unauthenticated, encrypted and non-encrypted clients. + +NOTE: currently `authorization` is not yet available in `4.5.0`. The Apache BookKeeper community is looking for adding this feature in subsequent releases. + +## Next Steps + +- [Encryption and Authentication using TLS](../tls) +- [Authentication using SASL](../sasl) +- [ZooKeeper Authentication](../zookeeper) diff --git a/site/docs/4.5.1/security/sasl.md b/site/docs/4.5.1/security/sasl.md new file mode 100644 index 00000000000..ffb972a8936 --- /dev/null +++ b/site/docs/4.5.1/security/sasl.md @@ -0,0 +1,202 @@ +--- +title: Authentication using SASL +prev: ../tls +next: ../zookeeper +--- + +Bookies support client authentication via SASL. Currently we only support GSSAPI (Kerberos). We will start +with a general description of how to configure `SASL` for bookies, clients and autorecovery daemons, followed +by mechanism-specific details and wrap up with some operational details. + +## SASL configuration for Bookies + +1. Select the mechanisms to enable in the bookies. `GSSAPI` is the only mechanism currently supported by BookKeeper. +2. Add a `JAAS` config file for the selected mechanisms as described in the examples for setting up [GSSAPI (Kerberos)](#kerberos). +3. Pass the `JAAS` config file location as JVM parameter to each Bookie. For example: + + ```shell + -Djava.security.auth.login.config=/etc/bookkeeper/bookie_jaas.conf + ``` + +4. Enable SASL auth plugin in bookies, by setting `bookieAuthProviderFactoryClass` to `org.apache.bookkeeper.sasl.SASLBookieAuthProviderFactory`. + + + ```shell + bookieAuthProviderFactoryClass=org.apache.bookkeeper.sasl.SASLBookieAuthProviderFactory + ``` + +5. If you are running `autorecovery` along with bookies, then you want to enable SASL auth plugin for `autorecovery`, by setting + `clientAuthProviderFactoryClass` to `org.apache.bookkeeper.sasl.SASLClientProviderFactory`. + + ```shell + clientAuthProviderFactoryClass=org.apache.bookkeeper.sasl.SASLClientProviderFactory + ``` + +6. Follow the steps in [GSSAPI (Kerberos)](#kerberos) to configure SASL. + +#### Important Notes + +1. `Bookie` is a section name in the JAAS file used by each bookie. This section tells the bookie which principal to use + and the location of the keytab where the principal is stored. It allows the bookie to login using the keytab specified in this section. +2. `Auditor` is a section name in the JASS file used by `autorecovery` daemon (it can be co-run with bookies). This section tells the + `autorecovery` daemon which principal to use and the location of the keytab where the principal is stored. It allows the bookie to + login using the keytab specified in this section. +3. The `Client` section is used to authenticate a SASL connection with ZooKeeper. It also allows the bookies to set ACLs on ZooKeeper nodes + which locks these nodes down so that only the bookies can modify it. It is necessary to have the same primary name across all bookies. + If you want to use a section name other than `Client`, set the system property `zookeeper.sasl.client` to the appropriate name + (e.g `-Dzookeeper.sasl.client=ZKClient`). +4. ZooKeeper uses `zookeeper` as the service name by default. If you want to change this, set the system property + `zookeeper.sasl.client.username` to the appropriate name (e.g. `-Dzookeeper.sasl.client.username=zk`). + +## SASL configuration for Clients + +To configure `SASL` authentication on the clients: + +1. Select a `SASL` mechanism for authentication and add a `JAAS` config file for the selected mechanism as described in the examples for + setting up [GSSAPI (Kerberos)](#kerberos). +2. Pass the `JAAS` config file location as JVM parameter to each client JVM. For example: + + ```shell + -Djava.security.auth.login.config=/etc/bookkeeper/bookkeeper_jaas.conf + ``` + +3. Configure the following properties in bookkeeper `ClientConfiguration`: + + ```shell + clientAuthProviderFactoryClass=org.apache.bookkeeper.sasl.SASLClientProviderFactory + ``` + +Follow the steps in [GSSAPI (Kerberos)](#kerberos) to configure SASL for the selected mechanism. + +## Authentication using SASL/Kerberos + +### Prerequisites + +#### Kerberos + +If your organization is already using a Kerberos server (for example, by using `Active Directory`), there is no need to +install a new server just for BookKeeper. Otherwise you will need to install one, your Linux vendor likely has packages +for `Kerberos` and a short guide on how to install and configure it ([Ubuntu](https://help.ubuntu.com/community/Kerberos), +[Redhat](https://access.redhat.com/documentation/en-US/Red_Hat_Enterprise_Linux/6/html/Managing_Smart_Cards/installing-kerberos.html)). +Note that if you are using Oracle Java, you will need to download JCE policy files for your Java version and copy them to `$JAVA_HOME/jre/lib/security`. + +#### Kerberos Principals + +If you are using the organization’s Kerberos or Active Directory server, ask your Kerberos administrator for a principal +for each Bookie in your cluster and for every operating system user that will access BookKeeper with Kerberos authentication +(via clients and tools). + +If you have installed your own Kerberos, you will need to create these principals yourself using the following commands: + +```shell +sudo /usr/sbin/kadmin.local -q 'addprinc -randkey bookkeeper/{hostname}@{REALM}' +sudo /usr/sbin/kadmin.local -q "ktadd -k /etc/security/keytabs/{keytabname}.keytab bookkeeper/{hostname}@{REALM}" +``` + +##### All hosts must be reachable using hostnames + +It is a *Kerberos* requirement that all your hosts can be resolved with their FQDNs. + +### Configuring Bookies + +1. Add a suitably modified JAAS file similar to the one below to each Bookie’s config directory, let’s call it `bookie_jaas.conf` +for this example (note that each bookie should have its own keytab): + + ``` + Bookie { + com.sun.security.auth.module.Krb5LoginModule required + useKeyTab=true + storeKey=true + keyTab="/etc/security/keytabs/bookie.keytab" + principal="bookkeeper/bk1.hostname.com@EXAMPLE.COM"; + }; + // ZooKeeper client authentication + Client { + com.sun.security.auth.module.Krb5LoginModule required + useKeyTab=true + storeKey=true + keyTab="/etc/security/keytabs/bookie.keytab" + principal="bookkeeper/bk1.hostname.com@EXAMPLE.COM"; + }; + // If you are running `autorecovery` along with bookies + Auditor { + com.sun.security.auth.module.Krb5LoginModule required + useKeyTab=true + storeKey=true + keyTab="/etc/security/keytabs/bookie.keytab" + principal="bookkeeper/bk1.hostname.com@EXAMPLE.COM"; + }; + ``` + + The `Bookie` section in the JAAS file tells the bookie which principal to use and the location of the keytab where this principal is stored. + It allows the bookie to login using the keytab specified in this section. See [notes](#notes) for more details on Zookeeper’s SASL configuration. + +2. Pass the name of the JAAS file as a JVM parameter to each Bookie: + + ```shell + -Djava.security.auth.login.config=/etc/bookkeeper/bookie_jaas.conf + ``` + + You may also wish to specify the path to the `krb5.conf` file + (see [JDK’s Kerberos Requirements](https://docs.oracle.com/javase/8/docs/technotes/guides/security/jgss/tutorials/KerberosReq.html) for more details): + + ```shell + -Djava.security.krb5.conf=/etc/bookkeeper/krb5.conf + ``` + +3. Make sure the keytabs configured in the JAAS file are readable by the operating system user who is starting the Bookies. + +4. Enable SASL authentication plugin in the bookies by setting following parameters. + + ```shell + bookieAuthProviderFactoryClass=org.apache.bookkeeper.sasl.SASLBookieAuthProviderFactory + # if you run `autorecovery` along with bookies + clientAuthProviderFactoryClass=org.apache.bookkeeper.sasl.SASLClientProviderFactory + ``` + +### Configuring Clients + +To configure SASL authentication on the clients: + +1. Clients will authenticate to the cluster with their own principal (usually with the same name as the user running the client), + so obtain or create these principals as needed. Then create a `JAAS` file for each principal. The `BookKeeper` section describes + how the clients like writers and readers can connect to the Bookies. The following is an example configuration for a client using + a keytab (recommended for long-running processes): + + ``` + BookKeeper { + com.sun.security.auth.module.Krb5LoginModule required + useKeyTab=true + storeKey=true + keyTab="/etc/security/keytabs/bookkeeper.keytab" + principal="bookkeeper-client-1@EXAMPLE.COM"; + }; + ``` + + +2. Pass the name of the JAAS file as a JVM parameter to the client JVM: + + ```shell + -Djava.security.auth.login.config=/etc/bookkeeper/bookkeeper_jaas.conf + ``` + + You may also wish to specify the path to the `krb5.conf` file (see + [JDK’s Kerberos Requirements](https://docs.oracle.com/javase/8/docs/technotes/guides/security/jgss/tutorials/KerberosReq.html) for more details). + + ```shell + -Djava.security.krb5.conf=/etc/bookkeeper/krb5.conf + ``` + + +3. Make sure the keytabs configured in the `bookkeeper_jaas.conf` are readable by the operating system user who is starting bookkeeper client. + +4. Enable SASL authentication plugin in the client by setting following parameters. + + ```shell + clientAuthProviderFactoryClass=org.apache.bookkeeper.sasl.SASLClientProviderFactory + ``` + +## Enabling Logging for SASL + +To enable SASL debug output, you can set `sun.security.krb5.debug` system property to `true`. + diff --git a/site/docs/4.5.1/security/tls.md b/site/docs/4.5.1/security/tls.md new file mode 100644 index 00000000000..cd250ab2aa5 --- /dev/null +++ b/site/docs/4.5.1/security/tls.md @@ -0,0 +1,210 @@ +--- +title: Encryption and Authentication using TLS +prev: ../overview +next: ../sasl +--- + +Apache BookKeeper allows clients and autorecovery daemons to communicate over TLS, although this is not enabled by default. + +## Overview + +The bookies need their own key and certificate in order to use TLS. Clients can optionally provide a key and a certificate +for mutual authentication. Each bookie or client can also be configured with a truststore, which is used to +determine which certificates (bookie or client identities) to trust (authenticate). + +The truststore can be configured in many ways. To understand the truststore, consider the following two examples: + +1. the truststore contains one or many certificates; +2. it contains a certificate authority (CA). + +In (1), with a list of certificates, the bookie or client will trust any certificate listed in the truststore. +In (2), with a CA, the bookie or client will trust any certificate that was signed by the CA in the truststore. + +(TBD: benefits) + +## Generate TLS key and certificate + +The first step of deploying TLS is to generate the key and the certificate for each machine in the cluster. +You can use Java’s `keytool` utility to accomplish this task. We will generate the key into a temporary keystore +initially so that we can export and sign it later with CA. + +```shell +keytool -keystore bookie.keystore.jks -alias localhost -validity {validity} -genkey +``` + +You need to specify two parameters in the above command: + +1. `keystore`: the keystore file that stores the certificate. The *keystore* file contains the private key of + the certificate; hence, it needs to be kept safely. +2. `validity`: the valid time of the certificate in days. + +
+Ensure that common name (CN) matches exactly with the fully qualified domain name (FQDN) of the server. +The client compares the CN with the DNS domain name to ensure that it is indeed connecting to the desired server, not a malicious one. +
+ +## Creating your own CA + +After the first step, each machine in the cluster has a public-private key pair, and a certificate to identify the machine. +The certificate, however, is unsigned, which means that an attacker can create such a certificate to pretend to be any machine. + +Therefore, it is important to prevent forged certificates by signing them for each machine in the cluster. +A `certificate authority (CA)` is responsible for signing certificates. CA works likes a government that issues passports — +the government stamps (signs) each passport so that the passport becomes difficult to forge. Other governments verify the stamps +to ensure the passport is authentic. Similarly, the CA signs the certificates, and the cryptography guarantees that a signed +certificate is computationally difficult to forge. Thus, as long as the CA is a genuine and trusted authority, the clients have +high assurance that they are connecting to the authentic machines. + +```shell +openssl req -new -x509 -keyout ca-key -out ca-cert -days 365 +``` + +The generated CA is simply a *public-private* key pair and certificate, and it is intended to sign other certificates. + +The next step is to add the generated CA to the clients' truststore so that the clients can trust this CA: + +```shell +keytool -keystore bookie.truststore.jks -alias CARoot -import -file ca-cert +``` + +NOTE: If you configure the bookies to require client authentication by setting `sslClientAuthentication` to `true` on the +[bookie config](../../reference/config), then you must also provide a truststore for the bookies and it should have all the CA +certificates that clients keys were signed by. + +```shell +keytool -keystore client.truststore.jks -alias CARoot -import -file ca-cert +``` + +In contrast to the keystore, which stores each machine’s own identity, the truststore of a client stores all the certificates +that the client should trust. Importing a certificate into one’s truststore also means trusting all certificates that are signed +by that certificate. As the analogy above, trusting the government (CA) also means trusting all passports (certificates) that +it has issued. This attribute is called the chain of trust, and it is particularly useful when deploying TLS on a large BookKeeper cluster. +You can sign all certificates in the cluster with a single CA, and have all machines share the same truststore that trusts the CA. +That way all machines can authenticate all other machines. + +## Signing the certificate + +The next step is to sign all certificates in the keystore with the CA we generated. First, you need to export the certificate from the keystore: + +```shell +keytool -keystore bookie.keystore.jks -alias localhost -certreq -file cert-file +``` + +Then sign it with the CA: + +```shell +openssl x509 -req -CA ca-cert -CAkey ca-key -in cert-file -out cert-signed -days {validity} -CAcreateserial -passin pass:{ca-password} +``` + +Finally, you need to import both the certificate of the CA and the signed certificate into the keystore: + +```shell +keytool -keystore bookie.keystore.jks -alias CARoot -import -file ca-cert +keytool -keystore bookie.keystore.jks -alias localhost -import -file cert-signed +``` + +The definitions of the parameters are the following: + +1. `keystore`: the location of the keystore +2. `ca-cert`: the certificate of the CA +3. `ca-key`: the private key of the CA +4. `ca-password`: the passphrase of the CA +5. `cert-file`: the exported, unsigned certificate of the bookie +6. `cert-signed`: the signed certificate of the bookie + +(TBD: add a script to automatically generate truststores and keystores.) + +## Configuring Bookies + +Bookies support TLS for connections on the same service port. In order to enable TLS, you need to configure `tlsProvider` to be either +`JDK` or `OpenSSL`. If `OpenSSL` is configured, it will use `netty-tcnative-boringssl-static`, which loads a corresponding binding according +to the platforms to run bookies. + +> Current `OpenSSL` implementation doesn't depend on the system installed OpenSSL library. If you want to leverage the OpenSSL installed on +the system, you can check [this example](http://netty.io/wiki/forked-tomcat-native.html) on how to replaces the JARs on the classpath with +netty bindings to leverage installed OpenSSL. + +The following TLS configs are needed on the bookie side: + +```shell +tlsProvider=OpenSSL +# key store +tlsKeyStoreType=JKS +tlsKeyStore=/var/private/tls/bookie.keystore.jks +tlsKeyStorePasswordPath=/var/private/tls/bookie.keystore.passwd +# trust store +tlsTrustStoreType=JKS +tlsTrustStore=/var/private/tls/bookie.truststore.jks +tlsTrustStorePasswordPath=/var/private/tls/bookie.truststore.passwd +``` + +NOTE: it is important to restrict access to the store files and corresponding password files via filesystem permissions. + +Optional settings that are worth considering: + +1. tlsClientAuthentication=false: Enable/Disable using TLS for authentication. This config when enabled will authenticate the other end + of the communication channel. It should be enabled on both bookies and clients for mutual TLS. +2. tlsEnabledCipherSuites= A cipher suite is a named combination of authentication, encryption, MAC and key exchange + algorithm used to negotiate the security settings for a network connection using TLS network protocol. By default, + it is null. [OpenSSL Ciphers](https://www.openssl.org/docs/man1.0.2/apps/ciphers.html) + [JDK Ciphers](http://docs.oracle.com/javase/8/docs/technotes/guides/security/StandardNames.html#ciphersuites) +3. tlsEnabledProtocols = TLSv1.2,TLSv1.1,TLSv1 (list out the TLS protocols that you are going to accept from clients). + By default, it is not set. + +To verify the bookie's keystore and truststore are setup correctly you can run the following command: + +```shell +openssl s_client -debug -connect localhost:3181 -tls1 +``` + +NOTE: TLSv1 should be listed under `tlsEnabledProtocols`. + +In the output of this command you should see the server's certificate: + +```shell +-----BEGIN CERTIFICATE----- +{variable sized random bytes} +-----END CERTIFICATE----- +``` + +If the certificate does not show up or if there are any other error messages then your keystore is not setup correctly. + +## Configuring Clients + +TLS is supported only for the new BookKeeper client (BookKeeper versions 4.5.0 and higher), the older clients are not +supported. The configs for TLS will be the same as bookies. + +If client authentication is not required by the bookies, the following is a minimal configuration example: + +```shell +tlsProvider=OpenSSL +clientTrustStore=/var/private/tls/client.truststore.jks +clientTrustStorePasswordPath=/var/private/tls/client.truststore.passwd +``` + +If client authentication is required, then a keystore must be created for each client, and the bookies' truststores must +trust the certificate in the client's keystore. This may be done using commands that are similar to what we used for +the [bookie keystore](#bookie-keystore). + +And the following must also be configured: + +```shell +tlsClientAuthentication=true +clientKeyStore=/var/private/tls/client.keystore.jks +clientKeyStorePasswordPath=/var/private/tls/client.keystore.passwd +``` + +NOTE: it is important to restrict access to the store files and corresponding password files via filesystem permissions. + +(TBD: add example to use tls in bin/bookkeeper script?) + +## Enabling TLS Logging + +You can enable TLS debug logging at the JVM level by starting the bookies and/or clients with `javax.net.debug` system property. For example: + +```shell +-Djavax.net.debug=all +``` + +You can find more details on this in [Oracle documentation](http://docs.oracle.com/javase/8/docs/technotes/guides/security/jsse/ReadDebug.html) on +[debugging SSL/TLS connections](http://docs.oracle.com/javase/8/docs/technotes/guides/security/jsse/ReadDebug.html). diff --git a/site/docs/4.5.1/security/zookeeper.md b/site/docs/4.5.1/security/zookeeper.md new file mode 100644 index 00000000000..e16be69a1d3 --- /dev/null +++ b/site/docs/4.5.1/security/zookeeper.md @@ -0,0 +1,41 @@ +--- +title: ZooKeeper Authentication +prev: ../sasl +--- + +## New Clusters + +To enable `ZooKeeper` authentication on Bookies or Clients, there are two necessary steps: + +1. Create a `JAAS` login file and set the appropriate system property to point to it as described in [GSSAPI (Kerberos)](../sasl#notes). +2. Set the configuration property `zkEnableSecurity` in each bookie to `true`. + +The metadata stored in `ZooKeeper` is such that only certain clients will be able to modify and read the corresponding znodes. +The rationale behind this decision is that the data stored in ZooKeeper is not sensitive, but inappropriate manipulation of znodes can cause cluster +disruption. + +## Migrating Clusters + +If you are running a version of BookKeeper that does not support security or simply with security disabled, and you want to make the cluster secure, +then you need to execute the following steps to enable ZooKeeper authentication with minimal disruption to your operations. + +1. Perform a rolling restart setting the `JAAS` login file, which enables bookie or clients to authenticate. At the end of the rolling restart, + bookies (or clients) are able to manipulate znodes with strict ACLs, but they will not create znodes with those ACLs. +2. Perform a second rolling restart of bookies, this time setting the configuration parameter `zkEnableSecurity` to true, which enables the use + of secure ACLs when creating znodes. +3. Currently we don't have provide a tool to set acls on old znodes. You are recommended to set it manually using ZooKeeper tools. + +It is also possible to turn off authentication in a secured cluster. To do it, follow these steps: + +1. Perform a rolling restart of bookies setting the `JAAS` login file, which enable bookies to authenticate, but setting `zkEnableSecurity` to `false`. + At the end of rolling restart, bookies stop creating znodes with secure ACLs, but are still able to authenticate and manipulate all znodes. +2. You can use ZooKeeper tools to manually reset all ACLs under the znode set in `zkLedgersRootPath`, which defaults to `/ledgers`. +3. Perform a second rolling restart of bookies, this time omitting the system property that sets the `JAAS` login file. + +## Migrating the ZooKeeper ensemble + +It is also necessary to enable authentication on the `ZooKeeper` ensemble. To do it, we need to perform a rolling restart of the ensemble and +set a few properties. Please refer to the ZooKeeper documentation for more details. + +1. [Apache ZooKeeper Documentation](http://zookeeper.apache.org/doc/r3.4.6/zookeeperProgrammers.html#sc_ZooKeeperAccessControl) +2. [Apache ZooKeeper Wiki](https://cwiki.apache.org/confluence/display/ZOOKEEPER/Zookeeper+and+SASL) From 34f1b0579f2b146e234cf9b247fc83d9b829fb62 Mon Sep 17 00:00:00 2001 From: Ivan Kelly Date: Tue, 31 Oct 2017 19:31:21 -0700 Subject: [PATCH 12/15] Recycle PendingAddOps Avoid creating a new PendingAddOp object for each entry added, thus saving on garbage. Originally commit 55ba4723 on the yahoo-4.3 branch. Author: Ivan Kelly Author: Matteo Merli Reviewers: Jia Zhai , Sijie Guo This closes #664 from ivankelly/yahoo-bp-1 --- .../bookkeeper/client/LedgerHandle.java | 49 +++---- .../bookkeeper/client/LedgerHandleAdv.java | 38 ++--- .../bookkeeper/client/PendingAddOp.java | 132 +++++++++++++++--- 3 files changed, 141 insertions(+), 78 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java index c61c85be084..acb97e040c4 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java @@ -822,8 +822,8 @@ public void asyncAddEntry(final byte[] data, final int offset, final int length, public void asyncAddEntry(ByteBuf data, final AddCallback cb, final Object ctx) { data.retain(); - PendingAddOp op = new PendingAddOp(LedgerHandle.this, cb, ctx); - doAsyncAddEntry(op, data, cb, ctx); + PendingAddOp op = PendingAddOp.create(this, data, cb, ctx); + doAsyncAddEntry(op); } /** @@ -864,17 +864,16 @@ public void asyncAddEntry(final long entryId, final byte[] data, final int offse */ void asyncRecoveryAddEntry(final byte[] data, final int offset, final int length, final AddCallback cb, final Object ctx) { - PendingAddOp op = new PendingAddOp(LedgerHandle.this, cb, ctx).enableRecoveryAdd(); - doAsyncAddEntry(op, Unpooled.wrappedBuffer(data, offset, length), cb, ctx); + PendingAddOp op = PendingAddOp.create(this, Unpooled.wrappedBuffer(data, offset, length), cb, ctx) + .enableRecoveryAdd(); + doAsyncAddEntry(op); } - protected void doAsyncAddEntry(final PendingAddOp op, final ByteBuf data, final AddCallback cb, final Object ctx) { + protected void doAsyncAddEntry(final PendingAddOp op) { if (throttler != null) { throttler.acquire(); } - final long entryId; - final long currentLength; boolean wasClosed = false; synchronized(this) { // synchronized on this to ensure that @@ -882,12 +881,11 @@ protected void doAsyncAddEntry(final PendingAddOp op, final ByteBuf data, final // updating lastAddPushed if (metadata.isClosed()) { wasClosed = true; - entryId = -1; - currentLength = 0; } else { - entryId = ++lastAddPushed; - currentLength = addToLength(data.readableBytes()); + long entryId = ++lastAddPushed; + long currentLedgerLength = addToLength(op.payload.readableBytes()); op.setEntryId(entryId); + op.setLedgerLength(currentLedgerLength); pendingAddOps.add(op); } } @@ -899,8 +897,8 @@ protected void doAsyncAddEntry(final PendingAddOp op, final ByteBuf data, final @Override public void safeRun() { LOG.warn("Attempt to add to closed ledger: {}", ledgerId); - cb.addComplete(BKException.Code.LedgerClosedException, - LedgerHandle.this, INVALID_ENTRY_ID, ctx); + op.cb.addComplete(BKException.Code.LedgerClosedException, + LedgerHandle.this, INVALID_ENTRY_ID, op.ctx); } @Override @@ -909,32 +907,17 @@ public String toString() { } }); } catch (RejectedExecutionException e) { - cb.addComplete(bk.getReturnRc(BKException.Code.InterruptedException), - LedgerHandle.this, INVALID_ENTRY_ID, ctx); + op.cb.addComplete(bk.getReturnRc(BKException.Code.InterruptedException), + LedgerHandle.this, INVALID_ENTRY_ID, op.ctx); } return; } try { - bk.getMainWorkerPool().submitOrdered(ledgerId, new SafeRunnable() { - @Override - public void safeRun() { - ByteBuf toSend = macManager.computeDigestAndPackageForSending(entryId, lastAddConfirmed, - currentLength, data); - try { - op.initiate(toSend, data.readableBytes()); - } finally { - toSend.release(); - } - } - @Override - public String toString() { - return String.format("AsyncAddEntry(lid=%d, eid=%d)", ledgerId, entryId); - } - }); + bk.getMainWorkerPool().submitOrdered(ledgerId, op); } catch (RejectedExecutionException e) { - cb.addComplete(bk.getReturnRc(BKException.Code.InterruptedException), - LedgerHandle.this, INVALID_ENTRY_ID, ctx); + op.cb.addComplete(bk.getReturnRc(BKException.Code.InterruptedException), + LedgerHandle.this, INVALID_ENTRY_ID, op.ctx); } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandleAdv.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandleAdv.java index 39f9932927c..28324b1b416 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandleAdv.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandleAdv.java @@ -153,15 +153,15 @@ public void asyncAddEntry(final long entryId, final byte[] data, final int offse private void asyncAddEntry(final long entryId, ByteBuf data, final AddCallback cb, final Object ctx) { - PendingAddOp op = new PendingAddOp(this, cb, ctx); + PendingAddOp op = PendingAddOp.create(this, data, cb, ctx); op.setEntryId(entryId); + if ((entryId <= this.lastAddConfirmed) || pendingAddOps.contains(op)) { LOG.error("Trying to re-add duplicate entryid:{}", entryId); - cb.addComplete(BKException.Code.DuplicateEntryIdException, - LedgerHandleAdv.this, entryId, ctx); + op.submitCallback(BKException.Code.DuplicateEntryIdException); return; } - doAsyncAddEntry(op, data, cb, ctx); + doAsyncAddEntry(op); } /** @@ -170,7 +170,7 @@ private void asyncAddEntry(final long entryId, ByteBuf data, * unaltered in the base class. */ @Override - protected void doAsyncAddEntry(final PendingAddOp op, final ByteBuf data, final AddCallback cb, final Object ctx) { + protected void doAsyncAddEntry(final PendingAddOp op) { if (throttler != null) { throttler.acquire(); } @@ -185,9 +185,10 @@ protected void doAsyncAddEntry(final PendingAddOp op, final ByteBuf data, final wasClosed = true; currentLength = 0; } else { - currentLength = addToLength(length); + currentLength = addToLength(op.payload.readableBytes()); pendingAddOps.add(op); } + op.setLedgerLength(currentLength); } if (wasClosed) { @@ -197,8 +198,8 @@ protected void doAsyncAddEntry(final PendingAddOp op, final ByteBuf data, final @Override public void safeRun() { LOG.warn("Attempt to add to closed ledger: {}", ledgerId); - cb.addComplete(BKException.Code.LedgerClosedException, - LedgerHandleAdv.this, op.getEntryId(), ctx); + op.cb.addComplete(BKException.Code.LedgerClosedException, + LedgerHandleAdv.this, op.getEntryId(), op.ctx); } @Override public String toString() { @@ -206,28 +207,17 @@ public String toString() { } }); } catch (RejectedExecutionException e) { - cb.addComplete(bk.getReturnRc(BKException.Code.InterruptedException), - LedgerHandleAdv.this, op.getEntryId(), ctx); + op.cb.addComplete(bk.getReturnRc(BKException.Code.InterruptedException), + LedgerHandleAdv.this, op.getEntryId(), op.ctx); } return; } try { - bk.getMainWorkerPool().submit(new SafeRunnable() { - @Override - public void safeRun() { - ByteBuf toSend = macManager.computeDigestAndPackageForSending(op.getEntryId(), lastAddConfirmed, - currentLength, data); - try { - op.initiate(toSend, toSend.readableBytes()); - } finally { - toSend.release(); - } - } - }); + bk.getMainWorkerPool().submitOrdered(ledgerId, op); } catch (RejectedExecutionException e) { - cb.addComplete(bk.getReturnRc(BKException.Code.InterruptedException), - LedgerHandleAdv.this, op.getEntryId(), ctx); + op.cb.addComplete(bk.getReturnRc(BKException.Code.InterruptedException), + LedgerHandleAdv.this, op.getEntryId(), op.ctx); } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingAddOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingAddOp.java index 00a36b3168d..f4d05b77ba9 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingAddOp.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingAddOp.java @@ -17,7 +17,10 @@ */ package org.apache.bookkeeper.client; +import static com.google.common.base.Preconditions.checkNotNull; import io.netty.buffer.ByteBuf; +import io.netty.util.Recycler; +import io.netty.util.Recycler.Handle; import io.netty.util.ReferenceCountUtil; import io.netty.util.Timeout; import io.netty.util.TimerTask; @@ -48,9 +51,10 @@ * * */ -class PendingAddOp implements WriteCallback, TimerTask { +class PendingAddOp extends SafeRunnable implements WriteCallback, TimerTask { private final static Logger LOG = LoggerFactory.getLogger(PendingAddOp.class); + ByteBuf payload; ByteBuf toSend; AddCallback cb; Object ctx; @@ -64,21 +68,38 @@ class PendingAddOp implements WriteCallback, TimerTask { boolean isRecoveryAdd = false; long requestTimeNanos; - final int timeoutSec; + int timeoutSec; Timeout timeout = null; OpStatsLogger addOpLogger; - boolean callbackTriggered = false; - - PendingAddOp(LedgerHandle lh, AddCallback cb, Object ctx) { - this.lh = lh; - this.cb = cb; - this.ctx = ctx; - this.entryId = LedgerHandle.INVALID_ENTRY_ID; - - this.ackSet = lh.distributionSchedule.getAckSet(); - this.addOpLogger = lh.bk.getAddOpLogger(); - this.timeoutSec = lh.bk.getConf().getAddEntryQuorumTimeout(); + long currentLedgerLength; + int pendingWriteRequests; + boolean callbackTriggered; + boolean hasRun; + + static PendingAddOp create(LedgerHandle lh, ByteBuf payload, AddCallback cb, Object ctx) { + PendingAddOp op = RECYCLER.get(); + op.lh = lh; + op.isRecoveryAdd = false; + op.cb = cb; + op.ctx = ctx; + op.entryId = LedgerHandle.INVALID_ENTRY_ID; + op.currentLedgerLength = -1; + op.payload = payload; + op.entryLength = payload.readableBytes(); + + op.completed = false; + op.ackSet = lh.distributionSchedule.getAckSet(); + op.addOpLogger = lh.bk.getAddOpLogger(); + if (op.timeout != null) { + op.timeout.cancel(); + } + op.timeout = null; + op.timeoutSec = lh.bk.getConf().getAddEntryQuorumTimeout(); + op.pendingWriteRequests = 0; + op.callbackTriggered = false; + op.hasRun = false; + return op; } /** @@ -94,6 +115,10 @@ void setEntryId(long entryId) { this.entryId = entryId; } + void setLedgerLength(long ledgerLength) { + this.currentLedgerLength = ledgerLength; + } + long getEntryId() { return this.entryId; } @@ -103,6 +128,7 @@ void sendWriteRequest(int bookieIndex) { lh.bk.getBookieClient().addEntry(lh.metadata.currentEnsemble.get(bookieIndex), lh.ledgerId, lh.ledgerKey, entryId, toSend, this, bookieIndex, flags); + ++pendingWriteRequests; } @Override @@ -182,21 +208,31 @@ void unsetSuccessAndSendWriteRequest(int bookieIndex) { sendWriteRequest(bookieIndex); } - void initiate(ByteBuf toSend, int entryLength) { + /** + * Initiate the add operation + */ + public void safeRun() { + hasRun = true; if (callbackTriggered) { - // this should only be true if the request was failed due to another request ahead in the pending queue, + // this should only be true if the request was failed due + // to another request ahead in the pending queue, // so we can just ignore this request + maybeRecycle(); return; } if (timeoutSec > -1) { - this.timeout = lh.bk.getBookieClient().scheduleTimeout(this, timeoutSec, TimeUnit.SECONDS); + this.timeout = lh.bk.getBookieClient().scheduleTimeout( + this, timeoutSec, TimeUnit.SECONDS); } + this.requestTimeNanos = MathUtils.nowInNano(); - this.toSend = toSend; - // Retain the buffer until all writes are complete - this.toSend.retain(); - this.entryLength = entryLength; + checkNotNull(lh); + checkNotNull(lh.macManager); + + this.toSend = lh.macManager.computeDigestAndPackageForSending( + entryId, lh.lastAddConfirmed, currentLedgerLength, + payload); // Iterate over set and trigger the sendWriteRequests DistributionSchedule.WriteSet writeSet @@ -213,6 +249,7 @@ void initiate(ByteBuf toSend, int entryLength) { @Override public void writeComplete(int rc, long ledgerId, long entryId, BookieSocketAddress addr, Object ctx) { int bookieIndex = (Integer) ctx; + --pendingWriteRequests; if (!lh.metadata.currentEnsemble.get(bookieIndex).equals(addr)) { // ensemble has already changed, failure of this addr is immaterial @@ -246,6 +283,7 @@ public void writeComplete(int rc, long ledgerId, long entryId, BookieSocketAddre sendAddSuccessCallbacks(); // I am already finished, ignore incoming responses. // otherwise, we might hit the following error handling logic, which might cause bad things. + maybeRecycle(); return; } @@ -292,7 +330,6 @@ public void writeComplete(int rc, long ledgerId, long entryId, BookieSocketAddre if (ackQuorum && !completed) { completed = true; - ackSet.recycle(); sendAddSuccessCallbacks(); } @@ -324,6 +361,8 @@ void submitCallback(final int rc) { } cb.addComplete(rc, lh, entryId, ctx); callbackTriggered = true; + + maybeRecycle(); } @Override @@ -348,4 +387,55 @@ public boolean equals(Object o) { return (this == o); } + private final Handle recyclerHandle; + private static final Recycler RECYCLER = new Recycler() { + protected PendingAddOp newObject(Recycler.Handle handle) { + return new PendingAddOp(handle); + } + }; + + private PendingAddOp(Handle recyclerHandle) { + this.recyclerHandle = recyclerHandle; + } + + private void maybeRecycle() { + // The reference to PendingAddOp can be held in 3 places + // - LedgerHandle#pendingAddOp + // This reference is released when the callback is run + // - The executor + // Released after safeRun finishes + // - BookieClient + // Holds a reference from the point the addEntry requests are + // sent. + // The object can only be recycled after all references are + // released, otherwise we could end up recycling twice and all + // joy that goes along with that. + if (hasRun && callbackTriggered && pendingWriteRequests == 0) { + recycle(); + } + } + + private void recycle() { + entryId = LedgerHandle.INVALID_ENTRY_ID; + currentLedgerLength = -1; + payload = null; + toSend = null; + cb = null; + ctx = null; + ackSet.recycle(); + ackSet = null; + lh = null; + isRecoveryAdd = false; + addOpLogger = null; + completed = false; + pendingWriteRequests = 0; + callbackTriggered = false; + hasRun = false; + if (timeout != null) { + timeout.cancel(); + } + timeout = null; + + recyclerHandle.recycle(this); + } } From 5eaa25f498ada0d93cd3002403906ac324d21cdc Mon Sep 17 00:00:00 2001 From: Jia Zhai Date: Tue, 31 Oct 2017 19:43:18 -0700 Subject: [PATCH 13/15] ISSUE #666: Introduce registration client for bookkeeper client to do bookie discover Descriptions of the changes in this PR: while in BOOKKEEPER-628, It is trying to improve/generalize the bookie registration process. This PR follows the work of #663, which provide bookie side registration, and it is for client side work, it tries to introduce RegistrationClient for bookkeeper client to do bookie discover. This PR follows the work of #663, which provide bookie side registration, Author: Jia Zhai Reviewers: Sijie Guo This closes #667 from zhaijack/client_registration2, closes #666 --- .../benchmark/BenchThroughputLatency.java | 2 +- .../bookkeeper/benchmark/TestClient.java | 3 - .../apache/bookkeeper/bookie/BookieShell.java | 2 +- .../apache/bookkeeper/client/BookKeeper.java | 132 +++---- .../bookkeeper/client/BookKeeperAdmin.java | 85 ++--- .../bookkeeper/client/BookieInfoReader.java | 6 +- .../bookkeeper/client/BookieWatcher.java | 358 ++++-------------- .../client/impl/BookKeeperBuilderImpl.java | 13 +- .../bookkeeper/conf/ClientConfiguration.java | 28 ++ .../discover/RegistrationClient.java | 89 ++++- .../discover/ZKRegistrationClient.java | 346 +++++++++++++++++ .../http/BKHttpServiceProvider.java | 4 +- .../bookkeeper/http/ListBookiesService.java | 2 +- .../apache/bookkeeper/proto/BookieServer.java | 3 +- .../bookkeeper/replication/Auditor.java | 31 +- .../replication/ReplicationWorker.java | 12 +- .../server/service/BookieService.java | 4 +- .../bookkeeper/util/LocalBookKeeper.java | 5 +- .../org/apache/bookkeeper/auth/TestAuth.java | 26 +- ...rDiskSpaceWeightedLedgerPlacementTest.java | 3 +- .../bookkeeper/client/BookKeeperTest.java | 6 +- .../client/BookKeeperTestClient.java | 5 +- .../bookkeeper/client/TestBookieWatcher.java | 2 +- .../bookkeeper/client/TestReadTimeout.java | 26 +- .../bookkeeper/test/BookieFailureTest.java | 4 - 25 files changed, 688 insertions(+), 509 deletions(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/ZKRegistrationClient.java diff --git a/bookkeeper-benchmark/src/main/java/org/apache/bookkeeper/benchmark/BenchThroughputLatency.java b/bookkeeper-benchmark/src/main/java/org/apache/bookkeeper/benchmark/BenchThroughputLatency.java index 49febbefd4b..d9de4ac0429 100644 --- a/bookkeeper-benchmark/src/main/java/org/apache/bookkeeper/benchmark/BenchThroughputLatency.java +++ b/bookkeeper-benchmark/src/main/java/org/apache/bookkeeper/benchmark/BenchThroughputLatency.java @@ -82,7 +82,7 @@ static class Context { public BenchThroughputLatency(int ensemble, int writeQuorumSize, int ackQuorumSize, byte[] passwd, int numberOfLedgers, int sendLimit, ClientConfiguration conf) - throws KeeperException, IOException, InterruptedException { + throws BKException, IOException, InterruptedException { this.sem = new Semaphore(conf.getThrottleValue()); bk = new BookKeeper(conf); this.counter = new AtomicLong(0); diff --git a/bookkeeper-benchmark/src/main/java/org/apache/bookkeeper/benchmark/TestClient.java b/bookkeeper-benchmark/src/main/java/org/apache/bookkeeper/benchmark/TestClient.java index 3c11c520afe..6c1ee0501b9 100644 --- a/bookkeeper-benchmark/src/main/java/org/apache/bookkeeper/benchmark/TestClient.java +++ b/bookkeeper-benchmark/src/main/java/org/apache/bookkeeper/benchmark/TestClient.java @@ -52,7 +52,6 @@ import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.zookeeper.KeeperException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -192,8 +191,6 @@ public void run() { executor.shutdown(); } catch (ExecutionException ee) { LOG.error("Exception in worker", ee); - } catch (KeeperException ke) { - LOG.error("Error accessing zookeeper", ke); } catch (BKException e) { LOG.error("Error accessing bookkeeper", e); } catch (IOException ioe) { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieShell.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieShell.java index 481c6477cc0..2d261453e04 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieShell.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieShell.java @@ -1129,7 +1129,7 @@ public int runCmd(CommandLine cmdLine) throws Exception { bookies.addAll(availableBookies); } else if (cmdLine.hasOption("ro")) { Collection roBookies = bka - .getReadOnlyBookiesAsync(); + .getReadOnlyBookies(); bookies.addAll(roBookies); } for (BookieSocketAddress b : bookies) { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeper.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeper.java index 8716c3a2ffb..8e148129e33 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeper.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeper.java @@ -40,8 +40,9 @@ import java.util.concurrent.locks.ReentrantReadWriteLock; import org.apache.bookkeeper.client.AsyncCallback.CreateCallback; import org.apache.bookkeeper.client.AsyncCallback.DeleteCallback; -import org.apache.bookkeeper.client.AsyncCallback.OpenCallback; import org.apache.bookkeeper.client.AsyncCallback.IsClosedCallback; +import org.apache.bookkeeper.client.AsyncCallback.OpenCallback; +import org.apache.bookkeeper.client.BKException.ZKException; import org.apache.bookkeeper.client.BookieInfoReader.BookieInfo; import org.apache.bookkeeper.client.SyncCallbackUtils.SyncCreateAdvCallback; import org.apache.bookkeeper.client.SyncCallbackUtils.SyncCreateCallback; @@ -51,7 +52,10 @@ import org.apache.bookkeeper.client.api.CreateBuilder; import org.apache.bookkeeper.client.api.DeleteBuilder; import org.apache.bookkeeper.client.api.OpenBuilder; +import org.apache.bookkeeper.conf.AbstractConfiguration; import org.apache.bookkeeper.conf.ClientConfiguration; +import org.apache.bookkeeper.discover.RegistrationClient; +import org.apache.bookkeeper.discover.ZKRegistrationClient; import org.apache.bookkeeper.feature.Feature; import org.apache.bookkeeper.feature.FeatureProvider; import org.apache.bookkeeper.feature.SettableFeatureProvider; @@ -64,13 +68,11 @@ import org.apache.bookkeeper.proto.BookieClient; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; import org.apache.bookkeeper.stats.NullStatsLogger; -import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.stats.OpStatsLogger; +import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.util.OrderedSafeExecutor; import org.apache.bookkeeper.util.ReflectionUtils; import org.apache.bookkeeper.util.SafeRunnable; -import org.apache.bookkeeper.zookeeper.BoundExponentialBackoffRetryPolicy; -import org.apache.bookkeeper.zookeeper.ZooKeeperClient; import org.apache.commons.configuration.ConfigurationException; import org.apache.commons.lang.SystemUtils; import org.apache.zookeeper.KeeperException; @@ -94,7 +96,7 @@ public class BookKeeper implements org.apache.bookkeeper.client.api.BookKeeper { static final Logger LOG = LoggerFactory.getLogger(BookKeeper.class); - final ZooKeeper zk; + final RegistrationClient regClient; final EventLoopGroup eventLoopGroup; // The stats logger for this client. @@ -115,9 +117,6 @@ public class BookKeeper implements org.apache.bookkeeper.client.api.BookKeeper { // whether the event loop group is one we created, or is owned by whoever // instantiated us boolean ownEventLoopGroup = false; - // whether the zk handle is one we created, or is owned by whoever - // instantiated us - boolean ownZKHandle = false; final BookieClient bookieClient; final BookieWatcher bookieWatcher; @@ -287,7 +286,7 @@ public Builder featureProvider(FeatureProvider featureProvider) { return this; } - public BookKeeper build() throws IOException, InterruptedException, KeeperException { + public BookKeeper build() throws IOException, InterruptedException, BKException { Preconditions.checkNotNull(statsLogger, "No stats logger provided"); return new BookKeeper(conf, zk, eventLoopGroup, statsLogger, dnsResolver, requestTimer, featureProvider); } @@ -305,13 +304,13 @@ public static Builder forConfig(final ClientConfiguration conf) { * A list of one of more servers on which zookeeper is running. The * client assumes that the running bookies have been registered with * zookeeper under the path - * {@link BookieWatcher#bookieRegistrationPath} + * {@link AbstractConfiguration#getZkAvailableBookiesPath()} * @throws IOException * @throws InterruptedException * @throws KeeperException */ public BookKeeper(String servers) throws IOException, InterruptedException, - KeeperException { + BKException { this(new ClientConfiguration().setZkServers(servers)); } @@ -327,13 +326,17 @@ public BookKeeper(String servers) throws IOException, InterruptedException, * @throws KeeperException */ public BookKeeper(final ClientConfiguration conf) - throws IOException, InterruptedException, KeeperException { + throws IOException, InterruptedException, BKException { this(conf, null, null, NullStatsLogger.INSTANCE, null, null, null); } - private static ZooKeeper validateZooKeeper(ZooKeeper zk) throws NullPointerException { + private static ZooKeeper validateZooKeeper(ZooKeeper zk) throws NullPointerException, IOException { Preconditions.checkNotNull(zk, "No zookeeper instance provided"); + if (!zk.getState().isConnected()) { + LOG.error("Unconnected zookeeper handle passed to bookkeeper"); + throw new IOException(KeeperException.create(KeeperException.Code.CONNECTIONLOSS)); + } return zk; } @@ -358,8 +361,7 @@ private static EventLoopGroup validateEventLoopGroup(EventLoopGroup eventLoopGro * @throws KeeperException */ public BookKeeper(ClientConfiguration conf, ZooKeeper zk) - throws IOException, InterruptedException, KeeperException { - + throws IOException, InterruptedException, BKException { this(conf, validateZooKeeper(zk), null, NullStatsLogger.INSTANCE, null, null, null); } @@ -381,7 +383,7 @@ public BookKeeper(ClientConfiguration conf, ZooKeeper zk) * @throws KeeperException if the passed zk handle is not connected */ public BookKeeper(ClientConfiguration conf, ZooKeeper zk, EventLoopGroup eventLoopGroup) - throws IOException, InterruptedException, KeeperException { + throws IOException, InterruptedException, BKException { this(conf, validateZooKeeper(zk), validateEventLoopGroup(eventLoopGroup), NullStatsLogger.INSTANCE, null, null, null); } @@ -396,28 +398,49 @@ private BookKeeper(ClientConfiguration conf, DNSToSwitchMapping dnsResolver, HashedWheelTimer requestTimer, FeatureProvider featureProvider) - throws IOException, InterruptedException, KeeperException { + throws IOException, InterruptedException, BKException { this.conf = conf; this.delayEnsembleChange = conf.getDelayEnsembleChange(); this.reorderReadSequence = conf.isReorderReadSequenceEnabled(); - // initialize zookeeper client - if (zkc == null) { - this.zk = ZooKeeperClient.newBuilder() - .connectString(conf.getZkServers()) - .sessionTimeoutMs(conf.getZkTimeout()) - .operationRetryPolicy(new BoundExponentialBackoffRetryPolicy(conf.getZkTimeout(), - conf.getZkTimeout(), 0)) - .statsLogger(statsLogger) - .build(); - this.ownZKHandle = true; + // initialize resources + ThreadFactoryBuilder tfb = new ThreadFactoryBuilder().setNameFormat( + "BookKeeperClientScheduler-%d"); + this.scheduler = Executors + .newSingleThreadScheduledExecutor(tfb.build()); + this.mainWorkerPool = OrderedSafeExecutor.newBuilder() + .name("BookKeeperClientWorker") + .numThreads(conf.getNumWorkerThreads()) + .statsLogger(statsLogger) + .traceTaskExecution(conf.getEnableTaskExecutionStats()) + .traceTaskWarnTimeMicroSec(conf.getTaskExecutionWarnTimeMicros()) + .build(); + + // initialize stats logger + this.statsLogger = statsLogger.scope(BookKeeperClientStats.CLIENT_SCOPE); + initOpLoggers(this.statsLogger); + + // initialize feature provider + if (null == featureProvider) { + this.featureProvider = SettableFeatureProvider.DISABLE_ALL; } else { - if (!zkc.getState().isConnected()) { - LOG.error("Unconnected zookeeper handle passed to bookkeeper"); - throw KeeperException.create(KeeperException.Code.CONNECTIONLOSS); - } - this.zk = zkc; - this.ownZKHandle = false; + this.featureProvider = featureProvider; + } + this.disableEnsembleChangeFeature = + this.featureProvider.getFeature(conf.getDisableEnsembleChangeFeatureName()); + + // initialize registration client + try { + Class regClientCls = conf.getRegistrationClientClass(); + this.regClient = ReflectionUtils.newInstance(regClientCls); + this.regClient.initialize( + conf, + scheduler, + statsLogger, + java.util.Optional.ofNullable(zkc)); + } catch (ConfigurationException ce) { + LOG.error("Failed to initialize registration client", ce); + throw new IOException("Failed to initialize registration client", ce); } // initialize event loop group @@ -440,25 +463,6 @@ private BookKeeper(ClientConfiguration conf, this.ownTimer = false; } - if (null == featureProvider) { - this.featureProvider = SettableFeatureProvider.DISABLE_ALL; - } else { - this.featureProvider = featureProvider; - } - - // get features - this.disableEnsembleChangeFeature = this.featureProvider.getFeature(conf.getDisableEnsembleChangeFeatureName()); - - // initialize scheduler - ThreadFactoryBuilder tfb = new ThreadFactoryBuilder().setNameFormat( - "BookKeeperClientScheduler-%d"); - this.scheduler = Executors - .newSingleThreadScheduledExecutor(tfb.build()); - - // initialize stats logger - this.statsLogger = statsLogger.scope(BookKeeperClientStats.CLIENT_SCOPE); - initOpLoggers(this.statsLogger); - // initialize the ensemble placement this.placementPolicy = initializeEnsemblePlacementPolicy(conf, dnsResolver, this.requestTimer, this.featureProvider, this.statsLogger); @@ -482,18 +486,11 @@ private BookKeeper(ClientConfiguration conf, } else { this.readLACSpeculativeRequestPolicy = Optional.absent(); } - // initialize main worker pool - this.mainWorkerPool = OrderedSafeExecutor.newBuilder() - .name("BookKeeperClientWorker") - .numThreads(conf.getNumWorkerThreads()) - .statsLogger(statsLogger) - .traceTaskExecution(conf.getEnableTaskExecutionStats()) - .traceTaskWarnTimeMicroSec(conf.getTaskExecutionWarnTimeMicros()) - .build(); + // initialize bookie client this.bookieClient = new BookieClient(conf, this.eventLoopGroup, this.mainWorkerPool, statsLogger); - this.bookieWatcher = new BookieWatcher(conf, this.scheduler, this.placementPolicy, this); + this.bookieWatcher = new BookieWatcher(conf, this.placementPolicy, regClient); if (conf.getDiskWeightBasedPlacementEnabled()) { LOG.info("Weighted ledger placement enabled"); ThreadFactoryBuilder tFBuilder = new ThreadFactoryBuilder() @@ -510,7 +507,12 @@ private BookKeeper(ClientConfiguration conf, } // initialize ledger manager - this.ledgerManagerFactory = LedgerManagerFactory.newLedgerManagerFactory(conf, this.zk); + try { + this.ledgerManagerFactory = + LedgerManagerFactory.newLedgerManagerFactory(conf, ((ZKRegistrationClient) regClient).getZk()); + } catch (KeeperException ke) { + throw new ZKException(); + } this.ledgerManager = new CleanupLedgerManager(ledgerManagerFactory.newLedgerManager()); this.ledgerIdGenerator = ledgerManagerFactory.newLedgerIdGenerator(); this.explicitLacInterval = conf.getExplictLacInterval(); @@ -638,7 +640,7 @@ public static DigestType fromApiDigestType(org.apache.bookkeeper.client.api.Dige } ZooKeeper getZkHandle() { - return zk; + return ((ZKRegistrationClient) regClient).getZk(); } protected ClientConfiguration getConf() { @@ -1371,9 +1373,7 @@ public void close() throws BKException, InterruptedException { if (ownEventLoopGroup) { eventLoopGroup.shutdownGracefully(); } - if (ownZKHandle) { - zk.close(); - } + this.regClient.close(); } private final void initOpLoggers(StatsLogger stats) { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperAdmin.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperAdmin.java index d2e5db3f376..aae36b872bd 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperAdmin.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperAdmin.java @@ -22,6 +22,7 @@ import static com.google.common.base.Charsets.UTF_8; import static com.google.common.base.Preconditions.checkArgument; +import static org.apache.bookkeeper.util.BookKeeperConstants.READONLY; import com.google.common.collect.Lists; import com.google.common.collect.Maps; @@ -32,17 +33,16 @@ import java.util.Collection; import java.util.Enumeration; import java.util.HashMap; -import java.util.HashSet; import java.util.Iterator; import java.util.LinkedList; import java.util.List; import java.util.Map; +import java.util.Map.Entry; import java.util.NoSuchElementException; import java.util.Optional; import java.util.Random; import java.util.Set; import java.util.UUID; -import java.util.Map.Entry; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeoutException; @@ -54,8 +54,9 @@ import org.apache.bookkeeper.client.SyncCallbackUtils.SyncReadCallback; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.ServerConfiguration; -import org.apache.bookkeeper.meta.LedgerManager.LedgerRangeIterator; +import org.apache.bookkeeper.discover.RegistrationClient.RegistrationListener; import org.apache.bookkeeper.meta.LedgerManager; +import org.apache.bookkeeper.meta.LedgerManager.LedgerRangeIterator; import org.apache.bookkeeper.meta.LedgerManagerFactory; import org.apache.bookkeeper.meta.LedgerUnderreplicationManager; import org.apache.bookkeeper.meta.ZkLedgerUnderreplicationManager; @@ -68,15 +69,15 @@ import org.apache.bookkeeper.replication.ReplicationException.BKAuditException; import org.apache.bookkeeper.replication.ReplicationException.CompatibilityException; import org.apache.bookkeeper.replication.ReplicationException.UnavailableException; +import org.apache.bookkeeper.stats.NullStatsLogger; +import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.util.BookKeeperConstants; import org.apache.bookkeeper.util.IOUtils; +import org.apache.bookkeeper.util.ZkUtils; import org.apache.bookkeeper.zookeeper.ZooKeeperClient; import org.apache.zookeeper.AsyncCallback; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; -import org.apache.bookkeeper.stats.NullStatsLogger; -import org.apache.bookkeeper.stats.StatsLogger; -import org.apache.bookkeeper.util.ZkUtils; import org.apache.zookeeper.ZKUtil; import org.apache.zookeeper.ZooKeeper; import org.apache.zookeeper.data.ACL; @@ -91,13 +92,6 @@ public class BookKeeperAdmin implements AutoCloseable { private final static Logger LOG = LoggerFactory.getLogger(BookKeeperAdmin.class); private final static Logger VERBOSE = LoggerFactory.getLogger("verbose"); - // ZK client instance - private ZooKeeper zk; - private final boolean ownsZK; - - // ZK ledgers related String constants - private final String bookiesPath; - // BookKeeper client instance private BookKeeper bkc; private final boolean ownsBK; @@ -138,7 +132,7 @@ public class BookKeeperAdmin implements AutoCloseable { * Throws this exception if there is an error instantiating the * BookKeeper client. */ - public BookKeeperAdmin(String zkServers) throws IOException, InterruptedException, KeeperException { + public BookKeeperAdmin(String zkServers) throws IOException, InterruptedException, BKException { this(new ClientConfiguration().setZkServers(zkServers)); } @@ -160,18 +154,9 @@ public BookKeeperAdmin(String zkServers) throws IOException, InterruptedExceptio * Throws this exception if there is an error instantiating the * BookKeeper client. */ - public BookKeeperAdmin(ClientConfiguration conf) throws IOException, InterruptedException, KeeperException { - // Create the ZooKeeper client instance - zk = ZooKeeperClient.newBuilder() - .connectString(conf.getZkServers()) - .sessionTimeoutMs(conf.getZkTimeout()) - .build(); - ownsZK = true; - - // Create the bookie path - bookiesPath = conf.getZkAvailableBookiesPath(); + public BookKeeperAdmin(ClientConfiguration conf) throws IOException, InterruptedException, BKException { // Create the BookKeeper client instance - bkc = new BookKeeper(conf, zk); + bkc = new BookKeeper(conf); ownsBK = true; this.lfr = new LedgerFragmentReplicator(bkc, NullStatsLogger.INSTANCE); this.mFactory = bkc.ledgerManagerFactory; @@ -189,9 +174,6 @@ public BookKeeperAdmin(ClientConfiguration conf) throws IOException, Interrupted public BookKeeperAdmin(final BookKeeper bkc, StatsLogger statsLogger) { this.bkc = bkc; ownsBK = false; - this.zk = bkc.zk; - ownsZK = false; - this.bookiesPath = bkc.getConf().getZkAvailableBookiesPath(); this.lfr = new LedgerFragmentReplicator(bkc, statsLogger); this.mFactory = bkc.ledgerManagerFactory; } @@ -212,18 +194,6 @@ public void close() throws InterruptedException, BKException { if (ownsBK) { bkc.close(); } - if (ownsZK) { - zk.close(); - } - } - - /** - * Get {@link org.apache.zookeeper.ZooKeeper} used by bookkeeper admin client. - * - * @return zookeeper client used by bookkeeper admin client - */ - public ZooKeeper getZooKeeper() { - return zk; } /** @@ -242,17 +212,8 @@ public Collection getAvailableBookies() * @return a collection of bookie addresses * @throws BKException if there are issues trying to read the list. */ - public Collection getReadOnlyBookiesSync() throws BKException { - return bkc.bookieWatcher.getReadOnlyBookiesSync(); - } - - /** - * Get a list of readonly bookies asynchronously (may be slightly out of date). - * - * @return a collection of bookie addresses - */ - public Collection getReadOnlyBookiesAsync() { - return bkc.bookieWatcher.getReadOnlyBookiesAsync(); + public Collection getReadOnlyBookies() throws BKException { + return bkc.bookieWatcher.getReadOnlyBookies(); } /** @@ -262,9 +223,9 @@ public Collection getReadOnlyBookiesAsync() { * * @param listener the listener to notify */ - public void notifyBookiesChanged(final BookiesListener listener) + public void watchWritableBookiesChanged(final RegistrationListener listener) throws BKException { - bkc.bookieWatcher.notifyBookiesChanged(listener); + bkc.regClient.watchWritableBookies(listener); } /** @@ -274,9 +235,9 @@ public void notifyBookiesChanged(final BookiesListener listener) * * @param listener the listener to notify */ - public void notifyReadOnlyBookiesChanged(final BookiesListener listener) + public void watchReadOnlyBookiesChanged(final RegistrationListener listener) throws BKException { - bkc.bookieWatcher.notifyReadOnlyBookiesChanged(listener); + bkc.regClient.watchReadOnlyBookies(listener); } /** @@ -1066,6 +1027,15 @@ public static boolean format(ClientConfiguration conf, zkAcls, CreateMode.PERSISTENT); } + // create readonly bookies node if not exists + if (null == zkc.exists(conf.getZkAvailableBookiesPath() + "/" + READONLY, false)) { + zkc.create( + conf.getZkAvailableBookiesPath() + "/" + READONLY, + new byte[0], + zkAcls, + CreateMode.PERSISTENT); + } + // If old data was there then confirm with admin. if (ledgerRootExists) { boolean confirm = false; @@ -1274,7 +1244,8 @@ public void triggerAudit() throw new UnavailableException("Autorecovery is disabled. So giving up!"); } - BookieSocketAddress auditorId = AuditorElector.getCurrentAuditor(new ServerConfiguration(bkc.conf), zk); + BookieSocketAddress auditorId = + AuditorElector.getCurrentAuditor(new ServerConfiguration(bkc.conf), bkc.getZkHandle()); if (auditorId == null) { LOG.error("No auditor elected, though Autorecovery is enabled. So giving up."); throw new UnavailableException("No auditor elected, though Autorecovery is enabled. So giving up."); @@ -1308,7 +1279,7 @@ public void triggerAudit() public void decommissionBookie(BookieSocketAddress bookieAddress) throws CompatibilityException, UnavailableException, KeeperException, InterruptedException, IOException, BKAuditException, TimeoutException, BKException { - if (getAvailableBookies().contains(bookieAddress) || getReadOnlyBookiesAsync().contains(bookieAddress)) { + if (getAvailableBookies().contains(bookieAddress) || getReadOnlyBookies().contains(bookieAddress)) { LOG.error("Bookie: {} is not shutdown yet", bookieAddress); throw BKException.create(BKException.Code.IllegalOpException); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieInfoReader.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieInfoReader.java index 70fe2bd9923..2ea5bef6895 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieInfoReader.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieInfoReader.java @@ -29,14 +29,13 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; - -import org.apache.commons.collections4.CollectionUtils; import org.apache.bookkeeper.client.WeightedRandomSelection.WeightedObject; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookieClient; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GetBookieInfoCallback; import org.apache.bookkeeper.proto.BookkeeperProtocol; +import org.apache.commons.collections4.CollectionUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -234,6 +233,7 @@ public boolean completeUnlessQueued() { } public void start() { + this.bk.regClient.watchWritableBookies(bookies -> availableBookiesChanged(bookies.getValue())); scheduler.scheduleAtFixedRate(new Runnable() { @Override public void run() { @@ -402,7 +402,7 @@ Map getBookieInfo() throws BKException, Interru Collection bookies; bookies = bk.bookieWatcher.getBookies(); - bookies.addAll(bk.bookieWatcher.getReadOnlyBookiesAsync()); + bookies.addAll(bk.bookieWatcher.getReadOnlyBookies()); totalSent.set(bookies.size()); for (BookieSocketAddress b : bookies) { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieWatcher.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieWatcher.java index b11271f23c9..2bba1d602d4 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieWatcher.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieWatcher.java @@ -17,43 +17,26 @@ */ package org.apache.bookkeeper.client; -import static org.apache.bookkeeper.util.SafeRunnable.safeRun; - -import java.io.IOException; +import com.google.common.cache.Cache; +import com.google.common.cache.CacheBuilder; +import com.google.common.cache.RemovalListener; +import com.google.common.cache.RemovalNotification; import java.util.ArrayList; -import java.util.Collection; +import java.util.Collections; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Set; -import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.RejectedExecutionException; -import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; - +import java.util.function.Function; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.client.BKException.BKInterruptedException; import org.apache.bookkeeper.client.BKException.BKNotEnoughBookiesException; +import org.apache.bookkeeper.client.BKException.MetaStoreException; +import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.conf.ClientConfiguration; +import org.apache.bookkeeper.discover.RegistrationClient; import org.apache.bookkeeper.net.BookieSocketAddress; -import org.apache.bookkeeper.util.BookKeeperConstants; -import org.apache.bookkeeper.util.SafeRunnable; -import org.apache.zookeeper.AsyncCallback.ChildrenCallback; -import org.apache.zookeeper.CreateMode; -import org.apache.zookeeper.KeeperException; -import org.apache.zookeeper.KeeperException.Code; -import org.apache.zookeeper.KeeperException.NodeExistsException; -import org.apache.zookeeper.WatchedEvent; -import org.apache.zookeeper.Watcher; -import org.apache.zookeeper.Watcher.Event.EventType; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.google.common.cache.Cache; -import com.google.common.cache.CacheBuilder; -import com.google.common.cache.RemovalListener; -import com.google.common.cache.RemovalNotification; - -import java.util.Map; -import org.apache.bookkeeper.util.ZkUtils; -import org.apache.zookeeper.data.ACL; /** * This class is responsible for maintaining a consistent view of what bookies @@ -62,157 +45,73 @@ * replacement * */ -class BookieWatcher implements Watcher, ChildrenCallback { - static final Logger logger = LoggerFactory.getLogger(BookieWatcher.class); - - public static int ZK_CONNECT_BACKOFF_SEC = 1; - private static final Set EMPTY_SET = new HashSet(); - - // Bookie registration path in ZK - private final String bookieRegistrationPath; +@Slf4j +class BookieWatcher { + + private static final Function EXCEPTION_FUNC = cause -> { + if (cause instanceof BKException) { + log.error("Failed to get bookie list : ", cause); + return (BKException) cause; + } else if (cause instanceof InterruptedException) { + log.error("Interrupted reading bookie list : ", cause); + return new BKInterruptedException(); + } else { + return new MetaStoreException(); + } + }; - final BookKeeper bk; - final ScheduledExecutorService scheduler; - final EnsemblePlacementPolicy placementPolicy; + private final ClientConfiguration conf; + private final RegistrationClient registrationClient; + private final EnsemblePlacementPolicy placementPolicy; // Bookies that will not be preferred to be chosen in a new ensemble final Cache quarantinedBookies; - SafeRunnable reReadTask = new SafeRunnable() { - @Override - public void safeRun() { - readBookies(); - } - }; - private ReadOnlyBookieWatcher readOnlyBookieWatcher; + private volatile Set writableBookies = Collections.emptySet(); + private volatile Set readOnlyBookies = Collections.emptySet(); public BookieWatcher(ClientConfiguration conf, - ScheduledExecutorService scheduler, EnsemblePlacementPolicy placementPolicy, - BookKeeper bk) throws KeeperException, InterruptedException { - this.bk = bk; - // ZK bookie registration path - this.bookieRegistrationPath = conf.getZkAvailableBookiesPath(); - this.scheduler = scheduler; + RegistrationClient registrationClient) { + this.conf = conf; this.placementPolicy = placementPolicy; - readOnlyBookieWatcher = new ReadOnlyBookieWatcher(conf, bk); + this.registrationClient = registrationClient; this.quarantinedBookies = CacheBuilder.newBuilder() .expireAfterWrite(conf.getBookieQuarantineTimeSeconds(), TimeUnit.SECONDS) .removalListener(new RemovalListener() { @Override public void onRemoval(RemovalNotification bookie) { - logger.info("Bookie {} is no longer quarantined", bookie.getKey()); + log.info("Bookie {} is no longer quarantined", bookie.getKey()); } }).build(); } - void notifyBookiesChanged(final BookiesListener listener) throws BKException { + public Set getBookies() throws BKException { try { - bk.getZkHandle().getChildren(this.bookieRegistrationPath, - new Watcher() { - public void process(WatchedEvent event) { - // listen children changed event from ZooKeeper - if (event.getType() == EventType.NodeChildrenChanged) { - listener.availableBookiesChanged(); - } - } - }); - } catch (KeeperException ke) { - logger.error("Error registering watcher with zookeeper", ke); - throw new BKException.ZKException(); - } catch (InterruptedException ie) { + return FutureUtils.result(registrationClient.getWritableBookies(), EXCEPTION_FUNC).getValue(); + } catch (BKInterruptedException ie) { Thread.currentThread().interrupt(); - logger.error("Interrupted registering watcher with zookeeper", ie); - throw new BKException.BKInterruptedException(); + throw ie; } } - void notifyReadOnlyBookiesChanged(final BookiesListener listener) throws BKException { - readOnlyBookieWatcher.notifyBookiesChanged(listener); - } - - public Collection getReadOnlyBookiesSync() throws BKException { - try { - String znode = this.bookieRegistrationPath + "/" + BookKeeperConstants.READONLY; - List children = bk.getZkHandle().getChildren(znode, false); - return convertToBookieAddresses(children); - } catch (KeeperException ke) { - logger.error("Failed to get read only bookie list : ", ke); - throw new BKException.ZKException(); - } catch (InterruptedException ie) { - Thread.currentThread().interrupt(); - logger.error("Interrupted reading read only bookie list", ie); - throw new BKException.BKInterruptedException(); - } - } - - public Collection getBookies() throws BKException { + public Set getReadOnlyBookies() throws BKException { try { - List children = bk.getZkHandle().getChildren(this.bookieRegistrationPath, false); - children.remove(BookKeeperConstants.READONLY); - return convertToBookieAddresses(children); - } catch (KeeperException ke) { - logger.error("Failed to get bookie list : ", ke); - throw new BKException.ZKException(); - } catch (InterruptedException ie) { + return FutureUtils.result(registrationClient.getReadOnlyBookies(), EXCEPTION_FUNC).getValue(); + } catch (BKInterruptedException ie) { Thread.currentThread().interrupt(); - logger.error("Interrupted reading bookie list", ie); - throw new BKException.BKInterruptedException(); + throw ie; } } - Collection getReadOnlyBookiesAsync() { - return new HashSet(readOnlyBookieWatcher.getReadOnlyBookies()); - } - - public void readBookies() { - readBookies(this); - } - - public void readBookies(ChildrenCallback callback) { - bk.getZkHandle().getChildren(this.bookieRegistrationPath, this, callback, null); - } - - @Override - public void process(WatchedEvent event) { - readBookies(); - } - - @Override - public void processResult(int rc, String path, Object ctx, List children) { - - if (rc != KeeperException.Code.OK.intValue()) { - //logger.error("Error while reading bookies", KeeperException.create(Code.get(rc), path)); - // try the read after a second again - try { - scheduler.schedule(reReadTask, ZK_CONNECT_BACKOFF_SEC, TimeUnit.SECONDS); - } catch (RejectedExecutionException ree) { - logger.warn("Failed to schedule reading bookies task : ", ree); - } - return; - } - - // Just exclude the 'readonly' znode to exclude r-o bookies from - // available nodes list. - children.remove(BookKeeperConstants.READONLY); - - HashSet newBookieAddrs = convertToBookieAddresses(children); - + // this callback is already not executed in zookeeper thread + private synchronized void processWritableBookiesChanged(Set newBookieAddrs) { // Update watcher outside ZK callback thread, to avoid deadlock in case some other // component is trying to do a blocking ZK operation - bk.getMainWorkerPool().submitOrdered(path, safeRun(() -> { - synchronized (BookieWatcher.this) { - Set readonlyBookies = readOnlyBookieWatcher.getReadOnlyBookies(); - placementPolicy.onClusterChanged(newBookieAddrs, readonlyBookies); - if (bk.conf.getDiskWeightBasedPlacementEnabled()) { - // start collecting bookieInfo for the newly joined bookies, if any - bk.bookieInfoReader.availableBookiesChanged(newBookieAddrs); - } - } - })); - + this.writableBookies = newBookieAddrs; + placementPolicy.onClusterChanged(newBookieAddrs, readOnlyBookies); // we don't need to close clients here, because: // a. the dead bookies will be removed from topology, which will not be used in new ensemble. // b. the read sequence will be reordered based on znode availability, so most of the reads @@ -229,45 +128,27 @@ public void processResult(int rc, String path, Object ctx, List children // } } - private static HashSet convertToBookieAddresses(List children) { - // Read the bookie addresses into a set for efficient lookup - HashSet newBookieAddrs = new HashSet(); - for (String bookieAddrString : children) { - BookieSocketAddress bookieAddr; - try { - bookieAddr = new BookieSocketAddress(bookieAddrString); - } catch (IOException e) { - logger.error("Could not parse bookie address: " + bookieAddrString + ", ignoring this bookie"); - continue; - } - newBookieAddrs.add(bookieAddr); - } - return newBookieAddrs; + private synchronized void processReadOnlyBookiesChanged(Set readOnlyBookies) { + this.readOnlyBookies = readOnlyBookies; + placementPolicy.onClusterChanged(writableBookies, readOnlyBookies); } /** * Blocks until bookies are read from zookeeper, used in the {@link BookKeeper} constructor. - * @throws InterruptedException - * @throws KeeperException + * + * @throws BKException when failed to read bookies */ - public void readBookiesBlocking() throws InterruptedException, KeeperException { - // Read readonly bookies first - readOnlyBookieWatcher.readROBookiesBlocking(); - - final LinkedBlockingQueue queue = new LinkedBlockingQueue(); - readBookies(new ChildrenCallback() { - public void processResult(int rc, String path, Object ctx, List children) { - bk.getMainWorkerPool().submitOrdered(path, safeRun(() -> { - BookieWatcher.this.processResult(rc, path, ctx, children); - queue.add(rc); - })); - } - }); - int rc = queue.take(); + public void readBookiesBlocking() throws BKException { + this.registrationClient.watchReadOnlyBookies(bookies -> processReadOnlyBookiesChanged(bookies.getValue())); + this.registrationClient.watchWritableBookies(bookies -> processWritableBookiesChanged(bookies.getValue())); - if (rc != KeeperException.Code.OK.intValue()) { - throw KeeperException.create(Code.get(rc)); + try { + readOnlyBookies = getReadOnlyBookies(); + } catch (Exception e) { + log.error("Failed getReadOnlyBookies: ", e); } + + writableBookies = getBookies(); } /** @@ -289,10 +170,11 @@ public ArrayList newEnsemble(int ensembleSize, int writeQuo writeQuorumSize, ackQuorumSize, customMetadata, new HashSet( quarantinedBookies.asMap().keySet())); } catch (BKNotEnoughBookiesException e) { - if (logger.isDebugEnabled()) { - logger.debug("Not enough healthy bookies available, using quarantined bookies"); + if (log.isDebugEnabled()) { + log.debug("Not enough healthy bookies available, using quarantined bookies"); } - return placementPolicy.newEnsemble(ensembleSize, writeQuorumSize, ackQuorumSize, customMetadata, EMPTY_SET); + return placementPolicy.newEnsemble( + ensembleSize, writeQuorumSize, ackQuorumSize, customMetadata, Collections.emptySet()); } } @@ -318,8 +200,8 @@ public BookieSocketAddress replaceBookie(int ensembleSize, int writeQuorumSize, return placementPolicy.replaceBookie(ensembleSize, writeQuorumSize, ackQuorumSize, customMetadata, existingAndQuarantinedBookies, addr, excludeBookies); } catch (BKNotEnoughBookiesException e) { - if (logger.isDebugEnabled()) { - logger.debug("Not enough healthy bookies available, using quarantined bookies"); + if (log.isDebugEnabled()) { + log.debug("Not enough healthy bookies available, using quarantined bookies"); } return placementPolicy.replaceBookie(ensembleSize, writeQuorumSize, ackQuorumSize, customMetadata, new HashSet(existingBookies), addr, excludeBookies); @@ -334,112 +216,8 @@ public BookieSocketAddress replaceBookie(int ensembleSize, int writeQuorumSize, public void quarantineBookie(BookieSocketAddress bookie) { if (quarantinedBookies.getIfPresent(bookie) == null) { quarantinedBookies.put(bookie, Boolean.TRUE); - logger.warn("Bookie {} has been quarantined because of read/write errors.", bookie); + log.warn("Bookie {} has been quarantined because of read/write errors.", bookie); } } - /** - * Watcher implementation to watch the readonly bookies under - * <available>/readonly - */ - private static class ReadOnlyBookieWatcher implements Watcher, ChildrenCallback { - - private final static Logger LOG = LoggerFactory.getLogger(ReadOnlyBookieWatcher.class); - private volatile HashSet readOnlyBookies = new HashSet(); - private BookKeeper bk; - private String readOnlyBookieRegPath; - - public ReadOnlyBookieWatcher(ClientConfiguration conf, BookKeeper bk) throws KeeperException, - InterruptedException { - this.bk = bk; - readOnlyBookieRegPath = conf.getZkAvailableBookiesPath() + "/" - + BookKeeperConstants.READONLY; - if (null == bk.getZkHandle().exists(readOnlyBookieRegPath, false)) { - try { - List zkAcls = ZkUtils.getACLs(conf); - bk.getZkHandle().create(readOnlyBookieRegPath, new byte[0], zkAcls, - CreateMode.PERSISTENT); - } catch (NodeExistsException e) { - // this node is just now created by someone. - } - } - } - - @Override - public void process(WatchedEvent event) { - readROBookies(); - } - - // read the readonly bookies in blocking fashion. Used only for first - // time. - void readROBookiesBlocking() throws InterruptedException, KeeperException { - - final LinkedBlockingQueue queue = new LinkedBlockingQueue(); - readROBookies(new ChildrenCallback() { - public void processResult(int rc, String path, Object ctx, List children) { - try { - ReadOnlyBookieWatcher.this.processResult(rc, path, ctx, children); - queue.put(rc); - } catch (InterruptedException e) { - logger.error("Interruped when trying to read readonly bookies in a blocking fashion"); - throw new RuntimeException(e); - } - } - }); - int rc = queue.take(); - - if (rc != KeeperException.Code.OK.intValue()) { - throw KeeperException.create(Code.get(rc)); - } - } - - void notifyBookiesChanged(final BookiesListener listener) throws BKException { - try { - List children = bk.getZkHandle().getChildren(this.readOnlyBookieRegPath, new Watcher() { - public void process(WatchedEvent event) { - // listen children changed event from ZooKeeper - if (event.getType() == EventType.NodeChildrenChanged) { - listener.availableBookiesChanged(); - } - } - }); - - // Update the list of read-only bookies - HashSet newReadOnlyBookies = convertToBookieAddresses(children); - readOnlyBookies = newReadOnlyBookies; - } catch (KeeperException ke) { - logger.error("Error registering watcher with zookeeper", ke); - throw new BKException.ZKException(); - } catch (InterruptedException ie) { - Thread.currentThread().interrupt(); - logger.error("Interrupted registering watcher with zookeeper", ie); - throw new BKException.BKInterruptedException(); - } - } - - // Read children and register watcher for readonly bookies path - void readROBookies(ChildrenCallback callback) { - bk.getZkHandle().getChildren(this.readOnlyBookieRegPath, this, callback, null); - } - - void readROBookies() { - readROBookies(this); - } - - @Override - public void processResult(int rc, String path, Object ctx, List children) { - if (rc != Code.OK.intValue()) { - LOG.error("Not able to read readonly bookies : ", KeeperException.create(Code.get(rc))); - return; - } - - HashSet newReadOnlyBookies = convertToBookieAddresses(children); - readOnlyBookies = newReadOnlyBookies; - } - - // returns the readonly bookies - public HashSet getReadOnlyBookies() { - return readOnlyBookies; - } - } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/impl/BookKeeperBuilderImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/impl/BookKeeperBuilderImpl.java index d9223a4318c..7133c08ec89 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/impl/BookKeeperBuilderImpl.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/impl/BookKeeperBuilderImpl.java @@ -20,19 +20,16 @@ */ package org.apache.bookkeeper.client.impl; -import com.google.common.base.Preconditions; import io.netty.channel.EventLoopGroup; import io.netty.util.HashedWheelTimer; import java.io.IOException; -import org.apache.bookkeeper.client.BKException.ZKException; +import org.apache.bookkeeper.client.api.BKException; import org.apache.bookkeeper.client.api.BookKeeper; import org.apache.bookkeeper.client.api.BookKeeperBuilder; -import org.apache.bookkeeper.client.api.BKException; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.feature.FeatureProvider; import org.apache.bookkeeper.net.DNSToSwitchMapping; import org.apache.bookkeeper.stats.StatsLogger; -import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.ZooKeeper; /** @@ -86,13 +83,7 @@ public BookKeeperBuilder featureProvider(FeatureProvider featureProvider) { @Override public BookKeeper build() throws InterruptedException, BKException, IOException { - try { - return builder.build(); - } catch (KeeperException err) { - ZKException zkErr = new ZKException(); - zkErr.initCause(err); - throw zkErr; - } + return builder.build(); } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ClientConfiguration.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ClientConfiguration.java index 5a2175346f4..8725a6ce6e7 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ClientConfiguration.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ClientConfiguration.java @@ -29,6 +29,8 @@ import org.apache.bookkeeper.client.EnsemblePlacementPolicy; import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicy; +import org.apache.bookkeeper.discover.RegistrationClient; +import org.apache.bookkeeper.discover.ZKRegistrationClient; import org.apache.bookkeeper.replication.Auditor; import org.apache.bookkeeper.util.ReflectionUtils; import org.apache.commons.configuration.ConfigurationException; @@ -147,6 +149,9 @@ public class ClientConfiguration extends AbstractConfiguration { protected final static String TLS_TRUSTSTORE = "clientTrustStore"; protected final static String TLS_TRUSTSTORE_PASSWORD_PATH = "clientTrustStorePasswordPath"; + // Registration Client + protected final static String REGISTRATION_CLIENT_CLASS = "registrationClientClass"; + /** * Construct a default client-side configuration */ @@ -1620,4 +1625,27 @@ public ClientConfiguration setNettyUsePooledBuffers(boolean enabled) { setProperty(NETTY_USE_POOLED_BUFFERS, enabled); return this; } + + /** + * Set registration manager class + * + * @param regClientClass + * ClientClass + */ + public void setRegistrationClientClass( + Class regClientClass) { + setProperty(REGISTRATION_CLIENT_CLASS, regClientClass); + } + + /** + * Get Registration Client Class. + * + * @return registration manager class. + */ + public Class getRegistrationClientClass() + throws ConfigurationException { + return ReflectionUtils.getClass(this, REGISTRATION_CLIENT_CLASS, + ZKRegistrationClient.class, RegistrationClient.class, + defaultLoader); + } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/RegistrationClient.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/RegistrationClient.java index 7f6160adb63..068f1ccd8aa 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/RegistrationClient.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/RegistrationClient.java @@ -18,19 +18,98 @@ package org.apache.bookkeeper.discover; -import java.util.List; +import java.util.Optional; +import java.util.Set; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ScheduledExecutorService; +import org.apache.bookkeeper.client.BKException; +import org.apache.bookkeeper.common.annotation.InterfaceAudience.LimitedPrivate; +import org.apache.bookkeeper.common.annotation.InterfaceStability.Evolving; +import org.apache.bookkeeper.conf.ClientConfiguration; +import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.stats.StatsLogger; +import org.apache.bookkeeper.versioning.Versioned; +import org.apache.zookeeper.ZooKeeper; + /** * A registration client, which the bookkeeper client will use to interact with registration service. */ -public interface RegistrationClient { +@LimitedPrivate +@Evolving +public interface RegistrationClient extends AutoCloseable { /** - * Get the list of available bookie identifiers. + * Listener to receive changes from the registration service. + */ + interface RegistrationListener { + + void onBookiesChanged(Versioned> bookies); + + } + + /** + * Initialize the registration client with provided resources. + * + *

The existence of zkSupplier is for backward compatability. * - * @return a future represents the list of available bookies + * @param conf client configuration + * @param statsLogger stats logger + * @param zkOptional a supplier to supply zookeeper client. + * @return */ - CompletableFuture> getAvailableBookies(); + RegistrationClient initialize(ClientConfiguration conf, + ScheduledExecutorService scheduler, + StatsLogger statsLogger, + Optional zkOptional) + throws BKException; + @Override + void close(); + + /** + * Get the list of writable bookie identifiers. + * + * @return a future represents the list of writable bookies. + */ + CompletableFuture>> getWritableBookies(); + + /** + * Get the list of readonly bookie identifiers. + * + * @return a future represents the list of readonly bookies. + */ + CompletableFuture>> getReadOnlyBookies(); + + /** + * Watch the changes of bookies. + * + *

The topology changes of bookies will be propagated to the provided listener. + * + * @param listener listener to receive the topology changes of bookies. + */ + void watchWritableBookies(RegistrationListener listener); + + /** + * Unwatch the changes of bookies. + * + * @param listener listener to receive the topology changes of bookies. + */ + void unwatchWritableBookies(RegistrationListener listener); + + /** + * Watch the changes of bookies. + * + *

The topology changes of bookies will be propagated to the provided listener. + * + * @param listener listener to receive the topology changes of bookies. + */ + void watchReadOnlyBookies(RegistrationListener listener); + + /** + * Unwatch the changes of bookies. + * + * @param listener listener to receive the topology changes of bookies. + */ + void unwatchReadOnlyBookies(RegistrationListener listener); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/ZKRegistrationClient.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/ZKRegistrationClient.java new file mode 100644 index 00000000000..e214521a71c --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/ZKRegistrationClient.java @@ -0,0 +1,346 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.bookkeeper.discover; + +import static org.apache.bookkeeper.util.BookKeeperConstants.READONLY; + +import com.google.common.collect.Sets; +import java.io.IOException; +import java.util.HashSet; +import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CopyOnWriteArraySet; +import java.util.concurrent.RejectedExecutionException; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.function.BiConsumer; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.client.BKException; +import org.apache.bookkeeper.client.BKException.BKInterruptedException; +import org.apache.bookkeeper.client.BKException.Code; +import org.apache.bookkeeper.client.BKException.ZKException; +import org.apache.bookkeeper.common.concurrent.FutureUtils; +import org.apache.bookkeeper.common.util.SafeRunnable; +import org.apache.bookkeeper.conf.ClientConfiguration; +import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.stats.StatsLogger; +import org.apache.bookkeeper.util.ZkUtils; +import org.apache.bookkeeper.versioning.LongVersion; +import org.apache.bookkeeper.versioning.Version; +import org.apache.bookkeeper.versioning.Version.Occurred; +import org.apache.bookkeeper.versioning.Versioned; +import org.apache.bookkeeper.zookeeper.BoundExponentialBackoffRetryPolicy; +import org.apache.bookkeeper.zookeeper.ZooKeeperClient; +import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.WatchedEvent; +import org.apache.zookeeper.Watcher; +import org.apache.zookeeper.Watcher.Event.EventType; +import org.apache.zookeeper.Watcher.Event.KeeperState; +import org.apache.zookeeper.ZooKeeper; +import org.apache.zookeeper.data.ACL; + +/** + * ZooKeeper based {@link RegistrationClient}. + */ +@Slf4j +public class ZKRegistrationClient implements RegistrationClient { + + private static final int ZK_CONNECT_BACKOFF_MS = 200; + + private class WatchTask + implements SafeRunnable, + Watcher, + BiConsumer>, Throwable>, + AutoCloseable { + + private final String regPath; + private final Set listeners; + private boolean closed = false; + private Set bookies = null; + private Version version = Version.NEW; + + WatchTask(String regPath) { + this.regPath = regPath; + this.listeners = new CopyOnWriteArraySet<>(); + } + + public int getNumListeners() { + return listeners.size(); + } + + public boolean addListener(RegistrationListener listener) { + if (listeners.add(listener)) { + if (null != bookies) { + listener.onBookiesChanged(new Versioned<>(bookies, version)); + } + } + return true; + } + + public boolean removeListener(RegistrationListener listener) { + return listeners.remove(listener); + } + + void watch() { + scheduleWatchTask(0L); + } + + private void scheduleWatchTask(long delayMs) { + try { + scheduler.schedule(this, delayMs, TimeUnit.MILLISECONDS); + } catch (RejectedExecutionException ree) { + log.warn("Failed to schedule watch bookies task", ree); + } + } + + @Override + public void safeRun() { + if (isClosed()) { + return; + } + + getChildren(regPath, this) + .whenCompleteAsync(this, scheduler); + } + + @Override + public void accept(Versioned> bookieSet, Throwable throwable) { + if (throwable != null) { + scheduleWatchTask(ZK_CONNECT_BACKOFF_MS); + return; + } + + if (this.version.compare(bookieSet.getVersion()) == Occurred.BEFORE + || this.version.compare(bookieSet.getVersion()) == Occurred.CONCURRENTLY) { + this.version = bookieSet.getVersion(); + this.bookies = bookieSet.getValue(); + + for (RegistrationListener listener : listeners) { + listener.onBookiesChanged(bookieSet); + } + } + } + + @Override + public void process(WatchedEvent event) { + if (EventType.None == event.getType()) { + if (KeeperState.Expired == event.getState()) { + scheduleWatchTask(ZK_CONNECT_BACKOFF_MS); + } + return; + } + + // re-read the bookie list + scheduleWatchTask(0L); + } + + synchronized boolean isClosed() { + return closed; + } + + @Override + public synchronized void close() { + if (!closed) { + return; + } + closed = true; + } + } + + private ClientConfiguration conf; + private ZooKeeper zk = null; + // whether the zk handle is one we created, or is owned by whoever + // instantiated us + private boolean ownZKHandle = false; + private ScheduledExecutorService scheduler; + private WatchTask watchWritableBookiesTask = null; + private WatchTask watchReadOnlyBookiesTask = null; + + // registration paths + private String bookieRegistrationPath; + private String bookieReadonlyRegistrationPath; + + @Override + public RegistrationClient initialize(ClientConfiguration conf, + ScheduledExecutorService scheduler, + StatsLogger statsLogger, + Optional zkOptional) + throws BKException { + this.conf = conf; + this.scheduler = scheduler; + + this.bookieRegistrationPath = conf.getZkAvailableBookiesPath(); + this.bookieReadonlyRegistrationPath = this.bookieRegistrationPath + "/" + READONLY; + + // construct the zookeeper + if (zkOptional.isPresent()) { + this.zk = zkOptional.get(); + this.ownZKHandle = false; + } else { + try { + this.zk = ZooKeeperClient.newBuilder() + .connectString(conf.getZkServers()) + .sessionTimeoutMs(conf.getZkTimeout()) + .operationRetryPolicy(new BoundExponentialBackoffRetryPolicy(conf.getZkTimeout(), + conf.getZkTimeout(), 0)) + .statsLogger(statsLogger) + .build(); + + if (null == zk.exists(bookieReadonlyRegistrationPath, false)) { + try { + List zkAcls = ZkUtils.getACLs(conf); + zk.create(bookieReadonlyRegistrationPath, + new byte[0], + zkAcls, + CreateMode.PERSISTENT); + } catch (KeeperException.NodeExistsException e) { + // this node is just now created by someone. + } + } + } catch (IOException | KeeperException e) { + log.error("Failed to create zookeeper client to {}", conf.getZkServers(), e); + ZKException zke = new ZKException(); + zke.fillInStackTrace(); + throw zke; + } catch (InterruptedException e) { + throw new BKInterruptedException(); + } + this.ownZKHandle = true; + } + + return this; + } + + @Override + public void close() { + if (ownZKHandle && null != zk) { + try { + zk.close(); + } catch (InterruptedException e) { + log.warn("Interrupted on closing zookeeper client", e); + } + } + } + + public ZooKeeper getZk() { + return zk; + } + + @Override + public CompletableFuture>> getWritableBookies() { + return getChildren(bookieRegistrationPath, null); + } + + @Override + public CompletableFuture>> getReadOnlyBookies() { + return getChildren(bookieReadonlyRegistrationPath, null); + } + + private CompletableFuture>> getChildren(String regPath, Watcher watcher) { + CompletableFuture>> future = FutureUtils.createFuture(); + zk.getChildren(regPath, watcher, (rc, path, ctx, children, stat) -> { + if (Code.OK != rc) { + ZKException zke = new ZKException(); + zke.fillInStackTrace(); + future.completeExceptionally(zke); + return; + } + + Version version = new LongVersion(stat.getVersion()); + Set bookies = convertToBookieAddresses(children); + future.complete(new Versioned<>(bookies, version)); + }, null); + return future; + } + + + @Override + public synchronized void watchWritableBookies(RegistrationListener listener) { + if (null == watchWritableBookiesTask) { + watchWritableBookiesTask = new WatchTask(bookieRegistrationPath); + } + + watchWritableBookiesTask.addListener(listener); + if (watchWritableBookiesTask.getNumListeners() == 1) { + watchWritableBookiesTask.watch(); + } + } + + @Override + public synchronized void unwatchWritableBookies(RegistrationListener listener) { + if (null == watchWritableBookiesTask) { + return; + } + + watchWritableBookiesTask.removeListener(listener); + if (watchWritableBookiesTask.getNumListeners() == 0) { + watchWritableBookiesTask.close(); + watchWritableBookiesTask = null; + } + } + + @Override + public synchronized void watchReadOnlyBookies(RegistrationListener listener) { + if (null == watchReadOnlyBookiesTask) { + watchReadOnlyBookiesTask = new WatchTask(bookieReadonlyRegistrationPath); + } + + watchReadOnlyBookiesTask.addListener(listener); + if (watchReadOnlyBookiesTask.getNumListeners() == 1) { + watchReadOnlyBookiesTask.watch(); + } + } + + @Override + public synchronized void unwatchReadOnlyBookies(RegistrationListener listener) { + if (null == watchReadOnlyBookiesTask) { + return; + } + + watchReadOnlyBookiesTask.removeListener(listener); + if (watchReadOnlyBookiesTask.getNumListeners() == 0) { + watchReadOnlyBookiesTask.close(); + watchReadOnlyBookiesTask = null; + } + } + + private static HashSet convertToBookieAddresses(List children) { + // Read the bookie addresses into a set for efficient lookup + HashSet newBookieAddrs = Sets.newHashSet(); + for (String bookieAddrString : children) { + if (READONLY.equals(bookieAddrString)) { + continue; + } + + BookieSocketAddress bookieAddr; + try { + bookieAddr = new BookieSocketAddress(bookieAddrString); + } catch (IOException e) { + log.error("Could not parse bookie address: " + bookieAddrString + ", ignoring this bookie"); + continue; + } + newBookieAddrs.add(bookieAddr); + } + return newBookieAddrs; + } + +} diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/http/BKHttpServiceProvider.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/http/BKHttpServiceProvider.java index b31b5e72cff..38eb007c48e 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/http/BKHttpServiceProvider.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/http/BKHttpServiceProvider.java @@ -59,7 +59,7 @@ public class BKHttpServiceProvider implements HttpServiceProvider { private BKHttpServiceProvider(BookieServer bookieServer, AutoRecoveryMain autoRecovery, ServerConfiguration serverConf) - throws IOException, KeeperException, InterruptedException { + throws IOException, KeeperException, InterruptedException, BKException { this.bookieServer = bookieServer; this.autoRecovery = autoRecovery; this.serverConf = serverConf; @@ -129,7 +129,7 @@ public Builder setServerConfiguration(ServerConfiguration conf) { } public BKHttpServiceProvider build() - throws IOException, KeeperException, InterruptedException { + throws IOException, KeeperException, InterruptedException, BKException { return new BKHttpServiceProvider( bookieServer, autoRecovery, diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/http/ListBookiesService.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/http/ListBookiesService.java index 5523f6a88da..f7e5fe427fa 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/http/ListBookiesService.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/http/ListBookiesService.java @@ -70,7 +70,7 @@ public HttpServiceResponse handle(HttpServiceRequest request) throws Exception { params.get("print_hostnames").equals("true"); if (readOnly) { - bookies.addAll(bka.getReadOnlyBookiesAsync()); + bookies.addAll(bka.getReadOnlyBookies()); } else { bookies.addAll(bka.getAvailableBookies()); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieServer.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieServer.java index 00c41bcc9fb..253a8715945 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieServer.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieServer.java @@ -31,6 +31,7 @@ import org.apache.bookkeeper.bookie.BookieException; import org.apache.bookkeeper.bookie.ExitCode; import org.apache.bookkeeper.bookie.ReadOnlyBookie; +import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.processor.RequestProcessor; @@ -101,7 +102,7 @@ protected Bookie newBookie(ServerConfiguration conf) new Bookie(conf, statsLogger.scope(BOOKIE_SCOPE)); } - public void start() throws IOException, UnavailableException, InterruptedException, KeeperException { + public void start() throws IOException, UnavailableException, InterruptedException, BKException { this.bookie.start(); // fail fast, when bookie startup is not successful if (!this.bookie.isRunning()) { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/Auditor.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/Auditor.java index 4f84d03a289..7a10e0b52d4 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/Auditor.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/Auditor.java @@ -40,7 +40,6 @@ import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.BookKeeperAdmin; -import org.apache.bookkeeper.client.BookiesListener; import org.apache.bookkeeper.client.LedgerChecker; import org.apache.bookkeeper.client.LedgerFragment; import org.apache.bookkeeper.client.LedgerHandle; @@ -73,7 +72,7 @@ * re-replication activities by keeping all the corresponding ledgers of the * failed bookie as underreplicated znode in zk. */ -public class Auditor implements BookiesListener { +public class Auditor { private static final Logger LOG = LoggerFactory.getLogger(Auditor.class); private final ServerConfiguration conf; private BookKeeper bkc; @@ -157,12 +156,9 @@ private void initialize(ServerConfiguration conf, ZooKeeper zkc) } catch (CompatibilityException ce) { throw new UnavailableException( "CompatibilityException while initializing Auditor", ce); - } catch (IOException ioe) { + } catch (IOException | BKException | KeeperException ioe) { throw new UnavailableException( - "IOException while initializing Auditor", ioe); - } catch (KeeperException ke) { - throw new UnavailableException( - "KeeperException while initializing Auditor", ke); + "Exception while initializing Auditor", ioe); } catch (InterruptedException ie) { throw new UnavailableException( "Interrupted while initializing Auditor", ie); @@ -375,7 +371,7 @@ public void run() { LOG.info("Periodic checking disabled"); } try { - notifyBookieChanges(); + watchBookieChanges(); knownBookies = getAvailableBookies(); } catch (BKException bke) { LOG.error("Couldn't get bookie list, exiting", bke); @@ -427,7 +423,7 @@ private void waitIfLedgerReplicationDisabled() throws UnavailableException, private List getAvailableBookies() throws BKException { // Get the available bookies Collection availableBkAddresses = admin.getAvailableBookies(); - Collection readOnlyBkAddresses = admin.getReadOnlyBookiesSync(); + Collection readOnlyBkAddresses = admin.getReadOnlyBookies(); availableBkAddresses.addAll(readOnlyBkAddresses); List availableBookies = new ArrayList(); @@ -437,9 +433,9 @@ private List getAvailableBookies() throws BKException { return availableBookies; } - private void notifyBookieChanges() throws BKException { - admin.notifyBookiesChanged(this); - admin.notifyReadOnlyBookiesChanged(this); + private void watchBookieChanges() throws BKException { + admin.watchWritableBookiesChanged(bookies -> submitAuditTask()); + admin.watchReadOnlyBookiesChanged(bookies -> submitAuditTask()); } /** @@ -702,17 +698,6 @@ public void processResult(int rc, String s, Object obj) { } } - @Override - public void availableBookiesChanged() { - // since a watch is triggered, we need to watch again on the bookies - try { - notifyBookieChanges(); - } catch (BKException bke) { - LOG.error("Exception while registering for a bookie change notification", bke); - } - submitAuditTask(); - } - /** * Shutdown the auditor */ diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/ReplicationWorker.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/ReplicationWorker.java index 61205f764ff..cd79afd723a 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/ReplicationWorker.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/ReplicationWorker.java @@ -131,10 +131,14 @@ public ReplicationWorker(final ZooKeeper zkc, .newLedgerManagerFactory(this.conf, this.zkc); this.underreplicationManager = mFactory .newLedgerUnderreplicationManager(); - this.bkc = BookKeeper.forConfig(new ClientConfiguration(conf)) - .zk(zkc) - .statsLogger(statsLogger.scope(BK_CLIENT_SCOPE)) - .build(); + try { + this.bkc = BookKeeper.forConfig(new ClientConfiguration(conf)) + .zk(zkc) + .statsLogger(statsLogger.scope(BK_CLIENT_SCOPE)) + .build(); + } catch (BKException e) { + throw new IOException("Failed to instantiate replication worker", e); + } this.admin = new BookKeeperAdmin(bkc, statsLogger); this.ledgerChecker = new LedgerChecker(bkc); this.workerThread = new BookieThread(this, "ReplicationWorker"); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/service/BookieService.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/service/BookieService.java index 7b43122f5d8..ff9e9dcfb27 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/service/BookieService.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/service/BookieService.java @@ -19,12 +19,12 @@ package org.apache.bookkeeper.server.service; import java.io.IOException; +import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.proto.BookieServer; import org.apache.bookkeeper.replication.ReplicationException.UnavailableException; import org.apache.bookkeeper.server.component.ServerLifecycleComponent; import org.apache.bookkeeper.server.conf.BookieConfiguration; import org.apache.bookkeeper.stats.StatsLogger; -import org.apache.zookeeper.KeeperException; /** * A {@link ServerLifecycleComponent} that starts the core bookie server. @@ -50,7 +50,7 @@ public BookieServer getServer() { protected void doStart() { try { this.server.start(); - } catch (IOException | UnavailableException | InterruptedException | KeeperException e) { + } catch (IOException | UnavailableException | InterruptedException | BKException e) { throw new RuntimeException("Failed to start bookie server", e); } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/LocalBookKeeper.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/LocalBookKeeper.java index f03767ca77b..02f779ee3ff 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/LocalBookKeeper.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/LocalBookKeeper.java @@ -27,6 +27,7 @@ import java.util.ArrayList; import java.util.List; +import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.shims.zk.ZooKeeperServerShim; import org.apache.bookkeeper.shims.zk.ZooKeeperServerShimFactory; import org.apache.bookkeeper.zookeeper.ZooKeeperClient; @@ -132,7 +133,7 @@ private static void cleanupDirectories(List dirs) throws IOException { private List runBookies(ServerConfiguration baseConf, String dirSuffix) throws IOException, KeeperException, InterruptedException, BookieException, - UnavailableException, CompatibilityException, SecurityException { + UnavailableException, CompatibilityException, SecurityException, BKException { List tempDirs = new ArrayList(); try { runBookies(baseConf, tempDirs, dirSuffix); @@ -161,7 +162,7 @@ private List runBookies(ServerConfiguration baseConf, String dirSuffix) @SuppressWarnings("deprecation") private void runBookies(ServerConfiguration baseConf, List tempDirs, String dirSuffix) throws IOException, KeeperException, InterruptedException, BookieException, UnavailableException, - CompatibilityException, SecurityException { + CompatibilityException, SecurityException, BKException { LOG.info("Starting Bookie(s)"); // Create Bookie Servers (B1, B2, B3) diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/auth/TestAuth.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/auth/TestAuth.java index d36e29dd732..6799d3e00b7 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/auth/TestAuth.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/auth/TestAuth.java @@ -20,31 +20,33 @@ */ package org.apache.bookkeeper.auth; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + import java.io.IOException; import java.util.Arrays; import java.util.Enumeration; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; - import org.apache.bookkeeper.client.BKException; +import org.apache.bookkeeper.client.BKException.BKNotEnoughBookiesException; import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.BookKeeper.DigestType; import org.apache.bookkeeper.client.LedgerEntry; import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.proto.BookieConnectionPeer; import org.apache.bookkeeper.proto.BookieServer; +import org.apache.bookkeeper.proto.ClientConnectionPeer; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; +import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.junit.Assert.*; - -import org.junit.Test; -import org.apache.bookkeeper.client.BKException.BKNotEnoughBookiesException; -import org.apache.bookkeeper.proto.ClientConnectionPeer; -import org.apache.bookkeeper.proto.BookieConnectionPeer; - public class TestAuth extends BookKeeperClusterTestCase { static final Logger LOG = LoggerFactory.getLogger(TestAuth.class); @@ -113,12 +115,12 @@ private int entryCount(long ledgerId, ServerConfiguration bookieConf, public void testSingleMessageAuth() throws Exception { ServerConfiguration bookieConf = newServerConfiguration(); bookieConf.setBookieAuthProviderFactoryClass( - AlwaysSucceedBookieAuthProviderFactory.class.getName()); - + AlwaysSucceedBookieAuthProviderFactory.class.getName()); + ClientConfiguration clientConf = newClientConfiguration(); clientConf.setClientAuthProviderFactoryClass( - SendUntilCompleteClientAuthProviderFactory.class.getName()); - + SendUntilCompleteClientAuthProviderFactory.class.getName()); + startAndStoreBookie(bookieConf); AtomicLong ledgerId = new AtomicLong(-1); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperDiskSpaceWeightedLedgerPlacementTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperDiskSpaceWeightedLedgerPlacementTest.java index 7206b236862..8ae4caf9003 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperDiskSpaceWeightedLedgerPlacementTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperDiskSpaceWeightedLedgerPlacementTest.java @@ -35,7 +35,6 @@ import org.apache.bookkeeper.proto.BookieServer; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.apache.bookkeeper.test.annotations.FlakyTest; -import org.apache.zookeeper.KeeperException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -53,7 +52,7 @@ public BookKeeperDiskSpaceWeightedLedgerPlacementTest() { } class BookKeeperCheckInfoReader extends BookKeeper { - BookKeeperCheckInfoReader(ClientConfiguration conf) throws KeeperException, IOException, InterruptedException { + BookKeeperCheckInfoReader(ClientConfiguration conf) throws BKException, IOException, InterruptedException { super(conf); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperTest.java index c0ce2424aba..3527575a0d8 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperTest.java @@ -21,6 +21,7 @@ package org.apache.bookkeeper.client; import io.netty.util.IllegalReferenceCountException; +import java.io.IOException; import java.util.Collections; import java.util.Enumeration; import java.util.concurrent.CountDownLatch; @@ -33,8 +34,8 @@ import org.apache.bookkeeper.client.BKException.BKBookieHandleNotAvailableException; import org.apache.bookkeeper.client.BookKeeper.DigestType; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; +import org.apache.zookeeper.KeeperException.ConnectionLossException; import org.apache.zookeeper.ZooKeeper; -import org.apache.zookeeper.KeeperException; import org.junit.Assert; import org.junit.Test; import org.slf4j.Logger; @@ -88,8 +89,9 @@ public void testConstructionNotConnectedExplicitZk() throws Exception { try { BookKeeper bkc = new BookKeeper(conf, zk); fail("Shouldn't be able to construct with unconnected zk"); - } catch (KeeperException.ConnectionLossException cle) { + } catch (IOException cle) { // correct behaviour + assertTrue(cle.getCause() instanceof ConnectionLossException); } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperTestClient.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperTestClient.java index 5401fef8574..a42a50d4b56 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperTestClient.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperTestClient.java @@ -22,7 +22,6 @@ */ import java.io.IOException; - import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.proto.BookieClient; import org.apache.zookeeper.KeeperException; @@ -34,7 +33,7 @@ */ public class BookKeeperTestClient extends BookKeeper { public BookKeeperTestClient(ClientConfiguration conf) - throws IOException, InterruptedException, KeeperException { + throws IOException, InterruptedException, BKException { super(conf); } @@ -57,7 +56,7 @@ public BookieClient getBookieClient() { * @throws KeeperException */ public void readBookiesBlocking() - throws InterruptedException, KeeperException { + throws InterruptedException, BKException { bookieWatcher.readBookiesBlocking(); } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestBookieWatcher.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestBookieWatcher.java index f1534606ea5..b0cb8794518 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestBookieWatcher.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestBookieWatcher.java @@ -112,7 +112,7 @@ private void runBookieWatcherWhenSessionExpired(ZooKeeper zk, int timeout, boole } // make zookeeper session expired - expireZooKeeperSession(bkc.zk, timeout); + expireZooKeeperSession(bkc.getZkHandle(), timeout); TimeUnit.MILLISECONDS.sleep(3 * timeout); // start four new bookies diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadTimeout.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadTimeout.java index 9fe7bed8b47..272435e9a58 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadTimeout.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadTimeout.java @@ -25,7 +25,6 @@ import java.util.Set; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicBoolean; - import org.apache.bookkeeper.client.AsyncCallback.AddCallback; import org.apache.bookkeeper.client.BookKeeper.DigestType; import org.apache.bookkeeper.net.BookieSocketAddress; @@ -54,36 +53,37 @@ public TestReadTimeout() { public void testReadTimeout() throws Exception { final AtomicBoolean completed = new AtomicBoolean(false); - LedgerHandle writelh = bkc.createLedger(3,3,digestType, "testPasswd".getBytes()); + LedgerHandle writelh = bkc.createLedger(3, 3, digestType, "testPasswd".getBytes()); String tmp = "Foobar"; - + final int numEntries = 10; for (int i = 0; i < numEntries; i++) { writelh.addEntry(tmp.getBytes()); } - + Set beforeSet = new HashSet(); beforeSet.addAll(writelh.getLedgerMetadata().getEnsemble(numEntries)); final BookieSocketAddress bookieToSleep = writelh.getLedgerMetadata().getEnsemble(numEntries).get(0); - int sleeptime = baseClientConf.getReadTimeout()*3; + int sleeptime = baseClientConf.getReadTimeout() * 3; CountDownLatch latch = sleepBookie(bookieToSleep, sleeptime); latch.await(); - writelh.asyncAddEntry(tmp.getBytes(), - new AddCallback() { - public void addComplete(int rc, LedgerHandle lh, - long entryId, Object ctx) { - completed.set(true); - } - }, null); - Thread.sleep((baseClientConf.getReadTimeout()*3)*1000); + writelh.asyncAddEntry(tmp.getBytes(), + new AddCallback() { + public void addComplete(int rc, LedgerHandle lh, + long entryId, Object ctx) { + completed.set(true); + } + }, null); + Thread.sleep((baseClientConf.getReadTimeout() * 3) * 1000); Assert.assertTrue("Write request did not finish", completed.get()); Set afterSet = new HashSet(); afterSet.addAll(writelh.getLedgerMetadata().getEnsemble(numEntries + 1)); beforeSet.removeAll(afterSet); Assert.assertTrue("Bookie set should not match", beforeSet.size() != 0); + } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieFailureTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieFailureTest.java index 42f8ef518b2..b522fb28295 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieFailureTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieFailureTest.java @@ -35,7 +35,6 @@ import org.apache.bookkeeper.proto.BookieServer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.zookeeper.KeeperException; import org.junit.Before; import org.junit.Test; @@ -230,9 +229,6 @@ void auxTestReadWriteAsyncSingleClient(BookieServer bs) throws IOException { LOG.info("Verified that entries are ok, and now closing ledger"); lh.close(); - } catch (KeeperException e) { - LOG.error("Caught KeeperException", e); - fail(e.toString()); } catch (BKException e) { LOG.error("Caught BKException", e); fail(e.toString()); From c3b2287c869b04bed86b8263129a76fb210b2b30 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Thu, 2 Nov 2017 07:55:15 +0100 Subject: [PATCH 14/15] Recycle AddRequest/AddResponse objects This change was originally afd0ecb6 & 75bf0fa1 on the yahoo-4.3 branch Author: Matteo Merli Reviewers: Enrico Olivelli , Jia Zhai , Sijie Guo This closes #665 from ivankelly/yahoo-bp-2 --- .../bookkeeper/client/LedgerHandleAdv.java | 6 +- .../apache/bookkeeper/proto/AuthHandler.java | 2 +- .../bookkeeper/proto/BookieProtoEncoding.java | 54 ++++--- .../bookkeeper/proto/BookieProtocol.java | 139 ++++++++++++------ .../proto/PerChannelBookieClient.java | 11 +- .../bookkeeper/proto/ResponseBuilder.java | 4 +- .../bookkeeper/proto/WriteEntryProcessor.java | 10 +- 7 files changed, 148 insertions(+), 78 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandleAdv.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandleAdv.java index 28324b1b416..0ad9a9d3519 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandleAdv.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandleAdv.java @@ -175,7 +175,6 @@ protected void doAsyncAddEntry(final PendingAddOp op) { throttler.acquire(); } - final long currentLength; boolean wasClosed = false; synchronized (this) { // synchronized on this to ensure that @@ -183,12 +182,11 @@ protected void doAsyncAddEntry(final PendingAddOp op) { // updating lastAddPushed if (metadata.isClosed()) { wasClosed = true; - currentLength = 0; } else { - currentLength = addToLength(op.payload.readableBytes()); + long currentLength = addToLength(op.payload.readableBytes()); + op.setLedgerLength(currentLength); pendingAddOps.add(op); } - op.setLedgerLength(currentLength); } if (wasClosed) { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/AuthHandler.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/AuthHandler.java index 409fe4b4a6b..07807858d1e 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/AuthHandler.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/AuthHandler.java @@ -103,7 +103,7 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception BookieProtocol.Request req = (BookieProtocol.Request) msg; if (req.getOpCode() == BookieProtocol.ADDENTRY) { ctx.channel().writeAndFlush( - new BookieProtocol.AddResponse( + BookieProtocol.AddResponse.create( req.getProtocolVersion(), BookieProtocol.EUA, req.getLedgerId(), req.getEntryId())); } else if (req.getOpCode() == BookieProtocol.READENTRY) { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieProtoEncoding.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieProtoEncoding.java index 0fece29c521..df7f3b27160 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieProtoEncoding.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieProtoEncoding.java @@ -102,7 +102,9 @@ public Object encode(Object msg, ByteBufAllocator allocator) ByteBuf buf = allocator.buffer(totalHeaderSize); buf.writeInt(PacketHeader.toInt(r.getProtocolVersion(), r.getOpCode(), r.getFlags())); buf.writeBytes(r.getMasterKey(), 0, BookieProtocol.MASTER_KEY_LENGTH); - return DoubleByteBuf.get(buf, ar.getData()); + ByteBuf data = ar.getData(); + ar.recycle(); + return DoubleByteBuf.get(buf, data); } else if (r instanceof BookieProtocol.ReadRequest) { int totalHeaderSize = 4 // for request type + 8 // for ledgerId @@ -155,7 +157,9 @@ public Object decode(ByteBuf packet) // Read ledger and entry id without advancing the reader index ledgerId = packet.getLong(packet.readerIndex()); entryId = packet.getLong(packet.readerIndex() + 8); - return new BookieProtocol.AddRequest(version, ledgerId, entryId, flags, masterKey, packet.retain()); + return BookieProtocol.AddRequest.create( + version, ledgerId, entryId, flags, + masterKey, packet.retain()); } case BookieProtocol.READENTRY: @@ -223,29 +227,33 @@ public Object encode(Object msg, ByteBufAllocator allocator) buf.writeInt(PacketHeader.toInt(r.getProtocolVersion(), r.getOpCode(), (short) 0)); ServerStats.getInstance().incrementPacketsSent(); - if (msg instanceof BookieProtocol.ReadResponse) { - buf.writeInt(r.getErrorCode()); - buf.writeLong(r.getLedgerId()); - buf.writeLong(r.getEntryId()); + try { + if (msg instanceof BookieProtocol.ReadResponse) { + buf.writeInt(r.getErrorCode()); + buf.writeLong(r.getLedgerId()); + buf.writeLong(r.getEntryId()); + + BookieProtocol.ReadResponse rr = (BookieProtocol.ReadResponse) r; + if (rr.hasData()) { + return DoubleByteBuf.get(buf, rr.getData()); + } else { + return buf; + } + } else if (msg instanceof BookieProtocol.AddResponse) { + buf.writeInt(r.getErrorCode()); + buf.writeLong(r.getLedgerId()); + buf.writeLong(r.getEntryId()); - BookieProtocol.ReadResponse rr = (BookieProtocol.ReadResponse)r; - if (rr.hasData()) { - return DoubleByteBuf.get(buf, rr.getData()); - } else { return buf; + } else if (msg instanceof BookieProtocol.AuthResponse) { + BookkeeperProtocol.AuthMessage am = ((BookieProtocol.AuthResponse) r).getAuthMessage(); + return DoubleByteBuf.get(buf, Unpooled.wrappedBuffer(am.toByteArray())); + } else { + LOG.error("Cannot encode unknown response type {}", msg.getClass().getName()); + return msg; } - } else if (msg instanceof BookieProtocol.AddResponse) { - buf.writeInt(r.getErrorCode()); - buf.writeLong(r.getLedgerId()); - buf.writeLong(r.getEntryId()); - - return buf; - } else if (msg instanceof BookieProtocol.AuthResponse) { - BookkeeperProtocol.AuthMessage am = ((BookieProtocol.AuthResponse)r).getAuthMessage(); - return DoubleByteBuf.get(buf, Unpooled.wrappedBuffer(am.toByteArray())); - } else { - LOG.error("Cannot encode unknown response type {}", msg.getClass().getName()); - return msg; + } finally { + r.recycle(); } } @Override @@ -263,7 +271,7 @@ public Object decode(ByteBuf buffer) rc = buffer.readInt(); ledgerId = buffer.readLong(); entryId = buffer.readLong(); - return new BookieProtocol.AddResponse(version, rc, ledgerId, entryId); + return BookieProtocol.AddResponse.create(version, rc, ledgerId, entryId); case BookieProtocol.READENTRY: rc = buffer.readInt(); ledgerId = buffer.readLong(); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieProtocol.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieProtocol.java index 6fc91e51eef..2c7a82835db 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieProtocol.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieProtocol.java @@ -23,6 +23,9 @@ import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; +import io.netty.util.Recycler; +import io.netty.util.Recycler.Handle; +import io.netty.util.ReferenceCounted; import org.apache.bookkeeper.proto.BookkeeperProtocol.AuthMessage; @@ -40,7 +43,7 @@ public interface BookieProtocol { public static final byte LOWEST_COMPAT_PROTOCOL_VERSION = 0; /** - * Current version of the protocol, which client will use. + * Current version of the protocol, which client will use. */ public static final byte CURRENT_PROTOCOL_VERSION = 2; @@ -62,19 +65,19 @@ public interface BookieProtocol { */ public static final int MASTER_KEY_LENGTH = 20; - /** + /** * The first int of a packet is the header. * It contains the version, opCode and flags. * The initial versions of BK didn't have this structure - * and just had an int representing the opCode as the - * first int. This handles that case also. + * and just had an int representing the opCode as the + * first int. This handles that case also. */ final static class PacketHeader { public static int toInt(byte version, byte opCode, short flags) { if (version == 0) { return (int)opCode; } else { - return ((version & 0xFF) << 24) + return ((version & 0xFF) << 24) | ((opCode & 0xFF) << 16) | (flags & 0xFFFF); } @@ -177,20 +180,14 @@ public static short getFlags(int packetHeader) { public static final short FLAG_RECOVERY_ADD = 0x0002; static class Request { - - final byte protocolVersion; - final byte opCode; - final long ledgerId; - final long entryId; - final short flags; - final byte[] masterKey; - - protected Request(byte protocolVersion, byte opCode, long ledgerId, - long entryId, short flags) { - this(protocolVersion, opCode, ledgerId, entryId, flags, null); - } - - protected Request(byte protocolVersion, byte opCode, long ledgerId, + byte protocolVersion; + byte opCode; + long ledgerId; + long entryId; + short flags; + byte[] masterKey; + + protected void init(byte protocolVersion, byte opCode, long ledgerId, long entryId, short flags, byte[] masterKey) { this.protocolVersion = protocolVersion; this.opCode = opCode; @@ -233,15 +230,25 @@ byte[] getMasterKey() { public String toString() { return String.format("Op(%d)[Ledger:%d,Entry:%d]", opCode, ledgerId, entryId); } + + public void recycle() {} } static class AddRequest extends Request { - final ByteBuf data; - - public AddRequest(byte protocolVersion, long ledgerId, long entryId, - short flags, byte[] masterKey, ByteBuf data) { - super(protocolVersion, ADDENTRY, ledgerId, entryId, flags, masterKey); - this.data = data.retain(); + ByteBuf data; + + static AddRequest create(byte protocolVersion, long ledgerId, + long entryId, short flags, byte[] masterKey, + ByteBuf data) { + AddRequest add = RECYCLER.get(); + add.protocolVersion = protocolVersion; + add.opCode = ADDENTRY; + add.ledgerId = ledgerId; + add.entryId = entryId; + add.flags = flags; + add.masterKey = masterKey; + add.data = data.retain(); + return add; } ByteBuf getData() { @@ -255,16 +262,36 @@ boolean isRecoveryAdd() { void release() { data.release(); } + + private final Handle recyclerHandle; + private AddRequest(Handle recyclerHandle) { + this.recyclerHandle = recyclerHandle; + } + + private static final Recycler RECYCLER = new Recycler() { + protected AddRequest newObject(Handle handle) { + return new AddRequest(handle); + } + }; + + @Override + public void recycle() { + ledgerId = -1; + entryId = -1; + masterKey = null; + data = null; + recyclerHandle.recycle(this); + } } static class ReadRequest extends Request { ReadRequest(byte protocolVersion, long ledgerId, long entryId, short flags) { - super(protocolVersion, READENTRY, ledgerId, entryId, flags); + init(protocolVersion, READENTRY, ledgerId, entryId, flags, null); } ReadRequest(byte protocolVersion, long ledgerId, long entryId, short flags, byte[] masterKey) { - super(protocolVersion, READENTRY, ledgerId, entryId, flags, masterKey); + init(protocolVersion, READENTRY, ledgerId, entryId, flags, masterKey); } boolean isFencingRequest() { @@ -276,7 +303,7 @@ static class AuthRequest extends Request { final AuthMessage authMessage; AuthRequest(byte protocolVersion, AuthMessage authMessage) { - super(protocolVersion, AUTH, -1, -1, FLAG_NONE, null); + init(protocolVersion, AUTH, -1, -1, FLAG_NONE, null); this.authMessage = authMessage; } @@ -285,14 +312,14 @@ AuthMessage getAuthMessage() { } } - static class Response { - final byte protocolVersion; - final byte opCode; - final int errorCode; - final long ledgerId; - final long entryId; + static abstract class Response { + byte protocolVersion; + byte opCode; + int errorCode; + long ledgerId; + long entryId; - protected Response(byte protocolVersion, byte opCode, + protected void init(byte protocolVersion, byte opCode, int errorCode, long ledgerId, long entryId) { this.protocolVersion = protocolVersion; this.opCode = opCode; @@ -326,18 +353,20 @@ public String toString() { return String.format("Op(%d)[Ledger:%d,Entry:%d,errorCode=%d]", opCode, ledgerId, entryId, errorCode); } + + abstract void recycle(); } static class ReadResponse extends Response { final ByteBuf data; ReadResponse(byte protocolVersion, int errorCode, long ledgerId, long entryId) { - super(protocolVersion, READENTRY, errorCode, ledgerId, entryId); + init(protocolVersion, READENTRY, errorCode, ledgerId, entryId); this.data = Unpooled.EMPTY_BUFFER; } ReadResponse(byte protocolVersion, int errorCode, long ledgerId, long entryId, ByteBuf data) { - super(protocolVersion, READENTRY, errorCode, ledgerId, entryId); + init(protocolVersion, READENTRY, errorCode, ledgerId, entryId); this.data = data; } @@ -348,18 +377,41 @@ boolean hasData() { ByteBuf getData() { return data; } + + void recycle() { + } } static class AddResponse extends Response { - AddResponse(byte protocolVersion, int errorCode, long ledgerId, long entryId) { - super(protocolVersion, ADDENTRY, errorCode, ledgerId, entryId); + static AddResponse create(byte protocolVersion, int errorCode, long ledgerId, long entryId) { + AddResponse response = RECYCLER.get(); + response.init(protocolVersion, ADDENTRY, errorCode, ledgerId, entryId); + return response; + } + + private final Handle recyclerHandle; + private AddResponse(Handle recyclerHandle) { + this.recyclerHandle = recyclerHandle; + } + + private static final Recycler RECYCLER = new Recycler() { + protected AddResponse newObject(Handle handle) { + return new AddResponse(handle); + } + }; + + public void recycle() { + recyclerHandle.recycle(this); } } - + static class ErrorResponse extends Response { ErrorResponse(byte protocolVersion, byte opCode, int errorCode, long ledgerId, long entryId) { - super(protocolVersion, opCode, errorCode, ledgerId, entryId); + init(protocolVersion, opCode, errorCode, ledgerId, entryId); + } + + void recycle() { } } @@ -367,13 +419,16 @@ static class AuthResponse extends Response { final AuthMessage authMessage; AuthResponse(byte protocolVersion, AuthMessage authMessage) { - super(protocolVersion, AUTH, EOK, -1, -1); + init(protocolVersion, AUTH, EOK, -1, -1); this.authMessage = authMessage; } AuthMessage getAuthMessage() { return authMessage; } + + void recycle() { + } } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java index 01ef079b16d..c52453365b5 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java @@ -2,6 +2,7 @@ * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance @@ -532,11 +533,13 @@ void addEntry(final long ledgerId, byte[] masterKey, final long entryId, ByteBuf CompletionKey completionKey = null; if (useV2WireProtocol) { completionKey = acquireV2Key(ledgerId, entryId, OperationType.ADD_ENTRY); - request = new BookieProtocol.AddRequest(BookieProtocol.CURRENT_PROTOCOL_VERSION, ledgerId, entryId, + request = BookieProtocol.AddRequest.create( + BookieProtocol.CURRENT_PROTOCOL_VERSION, ledgerId, entryId, (short) options, masterKey, toSend); } else { final long txnId = getTxnId(); completionKey = new V3CompletionKey(txnId, OperationType.ADD_ENTRY); + // Build the request and calculate the total size to be included in the packet. BKPacketHeader.Builder headerBuilder = BKPacketHeader.newBuilder() .setVersion(ProtocolVersion.VERSION_THREE) @@ -854,6 +857,7 @@ private static String requestToString(Object request) { return request.toString(); } } + void errorOut(final CompletionKey key) { if (LOG.isDebugEnabled()) { LOG.debug("Removing completion key: {}", key); @@ -1010,6 +1014,7 @@ private void readV2Response(final BookieProtocol.Response response) { public void safeRun() { completionValue.handleV2Response(ledgerId, entryId, status, response); + response.recycle(); } }); } @@ -1734,7 +1739,6 @@ public void run(Timeout timeout) throws Exception { public void release() {} } - /** * Note : Helper functions follow */ @@ -1814,7 +1818,8 @@ public boolean equals(Object object) { } V2CompletionKey that = (V2CompletionKey) object; return this.entryId == that.entryId - && this.ledgerId == that.ledgerId; + && this.ledgerId == that.ledgerId + && this.operationType == that.operationType; } @Override diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ResponseBuilder.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ResponseBuilder.java index c0be1624714..342acd5df31 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ResponseBuilder.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ResponseBuilder.java @@ -25,7 +25,7 @@ class ResponseBuilder { static BookieProtocol.Response buildErrorResponse(int errorCode, BookieProtocol.Request r) { if (r.getOpCode() == BookieProtocol.ADDENTRY) { - return new BookieProtocol.AddResponse(r.getProtocolVersion(), errorCode, + return BookieProtocol.AddResponse.create(r.getProtocolVersion(), errorCode, r.getLedgerId(), r.getEntryId()); } else { assert(r.getOpCode() == BookieProtocol.READENTRY); @@ -35,7 +35,7 @@ static BookieProtocol.Response buildErrorResponse(int errorCode, BookieProtocol. } static BookieProtocol.Response buildAddResponse(BookieProtocol.Request r) { - return new BookieProtocol.AddResponse(r.getProtocolVersion(), BookieProtocol.EOK, r.getLedgerId(), + return BookieProtocol.AddResponse.create(r.getProtocolVersion(), BookieProtocol.EOK, r.getLedgerId(), r.getEntryId()); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteEntryProcessor.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteEntryProcessor.java index c4b28406b02..416a478e927 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteEntryProcessor.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteEntryProcessor.java @@ -17,6 +17,7 @@ */ package org.apache.bookkeeper.proto; +import io.netty.buffer.ByteBuf; import io.netty.channel.Channel; import io.netty.util.Recycler; import io.netty.util.Recycler.Handle; @@ -70,11 +71,12 @@ protected void processPacket() { startTimeNanos = MathUtils.nowInNano(); int rc = BookieProtocol.EOK; + ByteBuf addData = add.getData(); try { if (add.isRecoveryAdd()) { - requestProcessor.bookie.recoveryAddEntry(add.getData(), this, channel, add.getMasterKey()); + requestProcessor.bookie.recoveryAddEntry(addData, this, channel, add.getMasterKey()); } else { - requestProcessor.bookie.addEntry(add.getData(), this, channel, add.getMasterKey()); + requestProcessor.bookie.addEntry(addData, this, channel, add.getMasterKey()); } } catch (IOException e) { LOG.error("Error writing " + add, e); @@ -86,7 +88,7 @@ protected void processPacket() { LOG.error("Unauthorized access to ledger " + add.getLedgerId(), e); rc = BookieProtocol.EUA; } finally { - add.release(); + addData.release(); } if (rc != BookieProtocol.EOK) { @@ -95,6 +97,7 @@ protected void processPacket() { sendResponse(rc, ResponseBuilder.buildErrorResponse(rc, add), requestProcessor.addRequestStats); + add.recycle(); } } @@ -111,6 +114,7 @@ public void writeComplete(int rc, long ledgerId, long entryId, sendResponse(rc, ResponseBuilder.buildAddResponse(request), requestProcessor.addRequestStats); + request.recycle(); recycle(); } From 26aaaa92c117f64cb6ba295e9eff9d0b5721035f Mon Sep 17 00:00:00 2001 From: Ivan Kelly Date: Thu, 2 Nov 2017 11:52:50 +0100 Subject: [PATCH 15/15] fixup for blah blah --- .../org/apache/bookkeeper/proto/PerChannelBookieClient.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java index c52453365b5..6aaedcde527 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java @@ -24,7 +24,7 @@ import com.google.common.base.Joiner; import com.google.protobuf.ByteString; import com.google.protobuf.ExtensionRegistry; -import io.netty.bootstrap.Bootstrap; +import io.netty.bootstrap.Bootstrap; // comment A import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBufAllocator; import io.netty.buffer.PooledByteBufAllocator;